Merge branch 'master' of github.com:jboner/akka

Conflicts:
	akka-actor/src/main/scala/akka/actor/Actor.scala
	akka-actor/src/main/scala/akka/actor/ActorRef.scala
	akka-actor/src/main/scala/akka/actor/TypedActor.scala
	akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala
	akka-cluster/src/main/scala/akka/cluster/ReplicatedClusterRef.scala
	akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableDispatcher.scala
	akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala
	akka-tutorials/akka-tutorial-second/src/main/java/akka/tutorial/java/second/Pi.java
This commit is contained in:
Roland 2011-06-14 00:19:54 +02:00
commit ca592ef1b4
89 changed files with 7842 additions and 2929 deletions

View file

@ -260,7 +260,7 @@ class ActorRefSpec extends WordSpec with MustMatchers {
def receive = { case _ self reply nested }
}).start()
val nested = (a !! "any").get.asInstanceOf[ActorRef]
val nested = (a ? "any").as[ActorRef].get
a must not be null
nested must not be null
(a ne nested) must be === true
@ -268,13 +268,13 @@ class ActorRefSpec extends WordSpec with MustMatchers {
"support advanced nested actorOfs" in {
val a = Actor.actorOf(new OuterActor(Actor.actorOf(new InnerActor).start)).start
val inner = (a !! "innerself").get
val inner = (a ? "innerself").as[Any].get
(a !! a).get must be(a)
(a !! "self").get must be(a)
(a ? a).as[ActorRef].get must be(a)
(a ? "self").as[ActorRef].get must be(a)
inner must not be a
(a !! "msg").get must be === "msg"
(a ? "msg").as[String] must be === Some("msg")
}
"support reply via channel" in {
@ -310,11 +310,11 @@ class ActorRefSpec extends WordSpec with MustMatchers {
}
}).start()
val ffive: Future[String] = ref !!! 5
val fnull: Future[String] = ref !!! null
val ffive = (ref ? 5).mapTo[String]
val fnull = (ref ? null).mapTo[String]
intercept[ActorKilledException] {
ref !! PoisonPill
(ref ? PoisonPill).get
fail("shouldn't get here")
}

View file

@ -22,7 +22,9 @@ class DeployerSpec extends WordSpec with MustMatchers {
Clustered(
Node("node1"),
Replicate(3),
Stateless))))
Replication(
TransactionLog,
WriteThrough)))))
}
}
}

View file

@ -50,10 +50,7 @@ object ForwardActorSpec {
val latch = TestLatch()
val forwardActor = actorOf[ForwardActor]
forwardActor.start()
(forwardActor !! "SendBangBang") match {
case Some(_) latch.countDown()
case None {}
}
forwardActor ? "SendBangBang" onComplete { _ latch.countDown() }
def receive = {
case _ {}
}

View file

@ -66,7 +66,7 @@ object SupervisorSpec {
}
override def receive = {
case Die temp !! (Die, TimeoutMillis)
case Die (temp.?(Die)(timeout = TimeoutMillis)).get
}
}
@ -200,13 +200,13 @@ class SupervisorSpec extends WordSpec with MustMatchers with BeforeAndAfterEach
}
def ping(pingPongActor: ActorRef) = {
(pingPongActor !! (Ping, TimeoutMillis)).getOrElse("nil") must be(PongMessage)
messageLogPoll must be(PingMessage)
(pingPongActor.?(Ping)(timeout = TimeoutMillis)).as[String].getOrElse("nil") must be === PongMessage
messageLogPoll must be === PingMessage
}
def kill(pingPongActor: ActorRef) = {
intercept[RuntimeException] { pingPongActor !! (Die, TimeoutMillis) }
messageLogPoll must be(ExceptionMessage)
messageLogPoll must be === ExceptionMessage
}
"A supervisor" must {
@ -215,7 +215,7 @@ class SupervisorSpec extends WordSpec with MustMatchers with BeforeAndAfterEach
val master = actorOf[Master].start()
intercept[RuntimeException] {
master !! (Die, TimeoutMillis)
(master.?(Die)(timeout = TimeoutMillis)).get
}
sleepFor(1 second)
@ -226,7 +226,7 @@ class SupervisorSpec extends WordSpec with MustMatchers with BeforeAndAfterEach
val (temporaryActor, supervisor) = temporaryActorAllForOne
intercept[RuntimeException] {
temporaryActor !! (Die, TimeoutMillis)
(temporaryActor.?(Die)(timeout = TimeoutMillis)).get
}
sleepFor(1 second)
@ -374,13 +374,13 @@ class SupervisorSpec extends WordSpec with MustMatchers with BeforeAndAfterEach
Supervise(dyingActor, Permanent) :: Nil))
intercept[Exception] {
dyingActor !! (Die, TimeoutMillis)
(dyingActor.?(Die)(timeout = TimeoutMillis)).get
}
// give time for restart
sleepFor(3 seconds)
(dyingActor !! (Ping, TimeoutMillis)).getOrElse("nil") must be(PongMessage)
(dyingActor.?(Ping)(timeout = TimeoutMillis)).as[String].getOrElse("nil") must be === PongMessage
inits.get must be(3)

View file

@ -37,9 +37,8 @@ class ConfigSpec extends WordSpec with MustMatchers {
getInt("akka.actor.throughput") must equal(Some(5))
getInt("akka.actor.throughput-deadline-time") must equal(Some(-1))
getString("akka.remote.layer") must equal(Some("akka.remote.netty.NettyRemoteSupport"))
getString("akka.remote.server.hostname") must equal(Some("localhost"))
getInt("akka.remote.server.port") must equal(Some(2552))
getString("akka.cluster.layer") must equal(Some("akka.remote.netty.NettyRemoteSupport"))
getInt("akka.cluster.server.port") must equal(Some(2552))
}
}
}

View file

@ -348,9 +348,9 @@ abstract class ActorModelSpec extends JUnitSuite {
implicit val dispatcher = newInterceptedDispatcher
val a = newTestActor.start()
dispatcher.suspend(a)
val f1: Future[String] = a !!! Reply("foo")
val stopped = a !!! PoisonPill
val shouldBeCompleted = for (i 1 to 10) yield a !!! Reply(i)
val f1: Future[String] = a ? Reply("foo") mapTo manifest[String]
val stopped = a ? PoisonPill
val shouldBeCompleted = for (i 1 to 10) yield a ? Reply(i)
dispatcher.resume(a)
assert(f1.get === "foo")
stopped.await

View file

@ -45,7 +45,7 @@ class DispatcherActorSpec extends JUnitSuite {
@Test
def shouldSendReplySync = {
val actor = actorOf[TestActor].start()
val result = (actor !! ("Hello", 10000)).as[String]
val result = (actor.?("Hello")(timeout = 10000)).as[String]
assert("World" === result.get)
actor.stop()
}
@ -53,8 +53,8 @@ class DispatcherActorSpec extends JUnitSuite {
@Test
def shouldSendReplyAsync = {
val actor = actorOf[TestActor].start()
val result = actor !! "Hello"
assert("World" === result.get.asInstanceOf[String])
val result = (actor ? "Hello").as[String]
assert("World" === result.get)
actor.stop()
}
@ -62,7 +62,7 @@ class DispatcherActorSpec extends JUnitSuite {
def shouldSendReceiveException = {
val actor = actorOf[TestActor].start()
try {
actor !! "Failure"
(actor ? "Failure").get
fail("Should have thrown an exception")
} catch {
case e

View file

@ -40,7 +40,7 @@ class FutureSpec extends JUnitSuite {
def shouldActorReplyResultThroughExplicitFuture {
val actor = actorOf[TestActor]
actor.start()
val future = actor !!! "Hello"
val future = actor ? "Hello"
future.await
assert(future.result.isDefined)
assert("World" === future.result.get)
@ -51,7 +51,7 @@ class FutureSpec extends JUnitSuite {
def shouldActorReplyExceptionThroughExplicitFuture {
val actor = actorOf[TestActor]
actor.start()
val future = actor !!! "Failure"
val future = actor ? "Failure"
future.await
assert(future.exception.isDefined)
assert("Expected exception; to test fault-tolerance" === future.exception.get.getMessage)
@ -62,9 +62,9 @@ class FutureSpec extends JUnitSuite {
def shouldFutureCompose {
val actor1 = actorOf[TestActor].start()
val actor2 = actorOf(new Actor { def receive = { case s: String self reply s.toUpperCase } }).start()
val future1 = actor1 !!! "Hello" flatMap ((s: String) actor2 !!! s)
val future2 = actor1 !!! "Hello" flatMap (actor2 !!! (_: String))
val future3 = actor1 !!! "Hello" flatMap (actor2 !!! (_: Int))
val future1 = actor1 ? "Hello" mapTo manifest[String] flatMap ((s: String) actor2 ? s)
val future2 = actor1 ? "Hello" mapTo manifest[String] flatMap (actor2 ? (_: String))
val future3 = actor1 ? "Hello" mapTo manifest[Int] flatMap (actor2 ? (_: Int))
assert((future1.get: Any) === "WORLD")
assert((future2.get: Any) === "WORLD")
intercept[ClassCastException] { future3.get }
@ -76,8 +76,8 @@ class FutureSpec extends JUnitSuite {
def shouldFutureComposePatternMatch {
val actor1 = actorOf[TestActor].start()
val actor2 = actorOf(new Actor { def receive = { case s: String self reply s.toUpperCase } }).start()
val future1 = actor1 !!! "Hello" collect { case (s: String) s } flatMap (actor2 !!! _)
val future2 = actor1 !!! "Hello" collect { case (n: Int) n } flatMap (actor2 !!! _)
val future1 = actor1 ? "Hello" collect { case (s: String) s } flatMap (actor2 ? _)
val future2 = actor1 ? "Hello" collect { case (n: Int) n } flatMap (actor2 ? _)
assert((future1.get: Any) === "WORLD")
intercept[MatchError] { future2.get }
actor1.stop()
@ -93,18 +93,18 @@ class FutureSpec extends JUnitSuite {
}
}).start()
val future0 = actor !!! "Hello"
val future0 = actor ? "Hello"
val future1 = for {
a: Int future0 // returns 5
b: String actor !!! a // returns "10"
c: String actor !!! 7 // returns "14"
a: Int future0.mapTo[Int] // returns 5
b: String (actor ? a).mapTo[String] // returns "10"
c: String (actor ? 7).mapTo[String] // returns "14"
} yield b + "-" + c
val future2 = for {
a: Int future0
b: Int actor !!! a
c: String actor !!! 7
a: Int future0.mapTo[Int]
b: Int (actor ? a).mapTo[Int]
c: String (actor ? 7).mapTo[String]
} yield b + "-" + c
assert(future1.get === "10-14")
@ -124,15 +124,15 @@ class FutureSpec extends JUnitSuite {
}).start()
val future1 = for {
Res(a: Int) actor.!!![Res[Int]](Req("Hello"))
Res(b: String) actor.!!![Res[String]](Req(a))
Res(c: String) actor.!!![Res[String]](Req(7))
Res(a: Int) actor.?(Req("Hello")).mapTo[Res[Int]]
Res(b: String) actor.?(Req(a)).mapTo[Res[String]]
Res(c: String) actor.?(Req(7)).mapTo[Res[String]]
} yield b + "-" + c
val future2 = for {
Res(a: Int) actor.!!![Any](Req("Hello"))
Res(b: Int) actor.!!![Res[Int]](Req(a))
Res(c: Int) actor.!!![Res[Int]](Req(7))
Res(a: Int) actor.?(Req("Hello"))
Res(b: Int) actor.?(Req(a)).mapTo[Res[Int]]
Res(c: Int) actor.?(Req(7)).mapTo[Res[Int]]
} yield b + "-" + c
assert(future1.get === "10-14")
@ -162,14 +162,14 @@ class FutureSpec extends JUnitSuite {
val actor = actorOf[TestActor].start()
val future8 = actor !!! "Failure"
val future9 = actor !!! "Failure" recover {
val future8 = actor ? "Failure"
val future9 = actor ? "Failure" recover {
case e: RuntimeException "FAIL!"
}
val future10 = actor !!! "Hello" recover {
val future10 = actor ? "Hello" recover {
case e: RuntimeException "FAIL!"
}
val future11 = actor !!! "Failure" recover { case _ "Oops!" }
val future11 = actor ? "Failure" recover { case _ "Oops!" }
assert(future1.get === 5)
intercept[ArithmeticException] { future2.get }
@ -194,7 +194,7 @@ class FutureSpec extends JUnitSuite {
}).start()
}
val timeout = 10000
def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) actor.!!![Int]((idx, idx * 200), timeout) }
def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) actor.?((idx, idx * 200))(timeout = timeout).mapTo[Int] }
assert(Futures.fold(0, timeout)(futures)(_ + _).await.result.get === 45)
}
@ -205,7 +205,7 @@ class FutureSpec extends JUnitSuite {
def receive = { case (add: Int, wait: Int) Thread.sleep(wait); self reply_? add }
}).start()
}
def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) actor.!!![Int]((idx, idx * 200), 10000) }
def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) actor.?((idx, idx * 200))(timeout = 10000).mapTo[Int] }
assert(futures.foldLeft(Future(0))((fr, fa) for (r fr; a fa) yield (r + a)).get === 45)
}
@ -222,7 +222,7 @@ class FutureSpec extends JUnitSuite {
}).start()
}
val timeout = 10000
def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) actor.!!![Int]((idx, idx * 100), timeout) }
def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) actor.?((idx, idx * 100))(timeout = timeout).mapTo[Int] }
assert(Futures.fold(0, timeout)(futures)(_ + _).await.exception.get.getMessage === "shouldFoldResultsWithException: expected")
}
@ -239,7 +239,7 @@ class FutureSpec extends JUnitSuite {
}).start()
}
val timeout = 10000
def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) actor.!!![Int]((idx, idx * 200), timeout) }
def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) actor.?((idx, idx * 200))(timeout = timeout).mapTo[Int] }
assert(Futures.reduce(futures, timeout)(_ + _).get === 45)
}
@ -256,7 +256,7 @@ class FutureSpec extends JUnitSuite {
}).start()
}
val timeout = 10000
def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) actor.!!![Int]((idx, idx * 100), timeout) }
def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) actor.?((idx, idx * 100))(timeout = timeout).mapTo[Int] }
assert(Futures.reduce(futures, timeout)(_ + _).await.exception.get.getMessage === "shouldFoldResultsWithException: expected")
}
@ -269,7 +269,7 @@ class FutureSpec extends JUnitSuite {
def receiveShouldExecuteOnComplete {
val latch = new StandardLatch
val actor = actorOf[TestActor].start()
actor !!! "Hello" onResult { case "World" latch.open }
actor ? "Hello" onResult { case "World" latch.open }
assert(latch.tryAwait(5, TimeUnit.SECONDS))
actor.stop()
}
@ -285,7 +285,7 @@ class FutureSpec extends JUnitSuite {
}
}).start()
val oddFutures: List[Future[Int]] = List.fill(100)(oddActor !!! 'GetNext)
val oddFutures = List.fill(100)(oddActor ? 'GetNext mapTo manifest[Int])
assert(Future.sequence(oddFutures).get.sum === 10000)
oddActor.stop()
@ -342,9 +342,9 @@ class FutureSpec extends JUnitSuite {
val actor = actorOf[TestActor].start
val x = Future("Hello")
val y = x flatMap (actor !!! _)
val y = x flatMap (actor ? _) mapTo manifest[String]
val r = flow(x() + " " + y[String]() + "!")
val r = flow(x() + " " + y() + "!")
assert(r.get === "Hello World!")
@ -370,9 +370,9 @@ class FutureSpec extends JUnitSuite {
val actor = actorOf[TestActor].start
val x = Future(3)
val y = actor !!! "Hello"
val y = (actor ? "Hello").mapTo[Int]
val r = flow(x() + y[Int](), 100)
val r = flow(x() + y(), 100)
intercept[ClassCastException](r.get)
}
@ -384,7 +384,7 @@ class FutureSpec extends JUnitSuite {
val actor = actorOf[TestActor].start
val x = Future("Hello")
val y = actor !!! "Hello"
val y = actor ? "Hello" mapTo manifest[Nothing]
val r = flow(x() + y())

View file

@ -8,7 +8,7 @@ import akka.dispatch.Dispatchers
import akka.actor.Actor
import Actor._
object ThreadBasedActorSpec {
object PinnedActorSpec {
class TestActor extends Actor {
self.dispatcher = Dispatchers.newPinnedDispatcher(self)
@ -21,8 +21,8 @@ object ThreadBasedActorSpec {
}
}
class ThreadBasedActorSpec extends JUnitSuite {
import ThreadBasedActorSpec._
class PinnedActorSpec extends JUnitSuite {
import PinnedActorSpec._
private val unit = TimeUnit.MILLISECONDS
@ -43,7 +43,7 @@ class ThreadBasedActorSpec extends JUnitSuite {
@Test
def shouldSendReplySync = {
val actor = actorOf[TestActor].start()
val result = (actor !! ("Hello", 10000)).as[String]
val result = (actor.?("Hello")(timeout = 10000)).as[String]
assert("World" === result.get)
actor.stop()
}
@ -51,8 +51,8 @@ class ThreadBasedActorSpec extends JUnitSuite {
@Test
def shouldSendReplyAsync = {
val actor = actorOf[TestActor].start()
val result = actor !! "Hello"
assert("World" === result.get.asInstanceOf[String])
val result = (actor ? "Hello").as[String]
assert("World" === result.get)
actor.stop()
}
@ -60,7 +60,7 @@ class ThreadBasedActorSpec extends JUnitSuite {
def shouldSendReceiveException = {
val actor = actorOf[TestActor].start()
try {
actor !! "Failure"
(actor ? "Failure").get
fail("Should have thrown an exception")
} catch {
case e

View file

@ -44,7 +44,7 @@ class PriorityDispatcherSpec extends WordSpec with MustMatchers {
dispatcher.resume(actor) //Signal the actor to start treating it's message backlog
actor.!!![List[Int]]('Result).await.result.get must be === (msgs.reverse)
actor.?('Result).as[List[Int]].get must be === (msgs.reverse)
}
}

View file

@ -89,7 +89,7 @@ class ActorRegistrySpec extends JUnitSuite {
val actor2 = actorOf[TestActor]("test-actor-2").start
val results = new ConcurrentLinkedQueue[Future[String]]
Actor.registry.local.foreach(actor results.add(actor.!!![String]("ping")))
Actor.registry.local.foreach(actor results.add(actor.?("ping").mapTo[String]))
assert(results.size === 2)
val i = results.iterator

View file

@ -7,12 +7,12 @@ import akka.testing._
import akka.testing.Testing.{ sleepFor, testMillis }
import akka.util.duration._
import akka.actor._
import akka.actor.Actor._
import akka.routing._
import java.util.concurrent.atomic.AtomicInteger
import akka.dispatch.{ KeptPromise, Future }
import akka.actor.{ TypedActor, Actor }
object RoutingSpec {
trait Foo {
@ -55,10 +55,11 @@ class RoutingSpec extends WordSpec with MustMatchers {
case Test3 t2
}.start()
implicit val timeout = Actor.Timeout(testMillis(5 seconds))
val result = for {
a (d !! (Test1, testMillis(5 seconds))).as[Int]
b (d !! (Test2, testMillis(5 seconds))).as[Int]
c (d !! (Test3, testMillis(5 seconds))).as[Int]
a (d ? (Test1)).as[Int]
b (d ? (Test2)).as[Int]
c (d ? (Test3)).as[Int]
} yield a + b + c
result.isDefined must be(true)
@ -208,7 +209,7 @@ class RoutingSpec extends WordSpec with MustMatchers {
count.get must be(2)
(pool !! ActorPool.Stat).asInstanceOf[Option[ActorPool.Stats]].get.size must be(2)
(pool ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(2)
pool.stop()
}
@ -237,7 +238,7 @@ class RoutingSpec extends WordSpec with MustMatchers {
}).start()
try {
(for (count 1 to 500) yield pool.!!![String]("Test", 20000)) foreach {
(for (count 1 to 500) yield pool.?("Test")(timeout = 20000)) foreach {
_.await.resultOrException.get must be("Response")
}
} finally {
@ -276,14 +277,14 @@ class RoutingSpec extends WordSpec with MustMatchers {
pool ! 1
(pool !! ActorPool.Stat).asInstanceOf[Option[ActorPool.Stats]].get.size must be(2)
(pool ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(2)
var loops = 0
def loop(t: Int) = {
latch = TestLatch(loops)
count.set(0)
for (m 0 until loops) {
pool !!! t
pool ? t
sleepFor(50 millis)
}
}
@ -296,7 +297,7 @@ class RoutingSpec extends WordSpec with MustMatchers {
latch.await
count.get must be(loops)
(pool !! ActorPool.Stat).asInstanceOf[Option[ActorPool.Stats]].get.size must be(2)
(pool ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(2)
// a whole bunch should max it out
@ -305,7 +306,7 @@ class RoutingSpec extends WordSpec with MustMatchers {
latch.await
count.get must be(loops)
(pool !! ActorPool.Stat).asInstanceOf[Option[ActorPool.Stats]].get.size must be(4)
(pool ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(4)
pool.stop()
}
@ -353,7 +354,7 @@ class RoutingSpec extends WordSpec with MustMatchers {
latch.await
count.get must be(loops)
(pool !! ActorPool.Stat).asInstanceOf[Option[ActorPool.Stats]].get.size must be(2)
(pool ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(2)
// send a bunch over the theshold and observe an increment
loops = 15
@ -362,7 +363,7 @@ class RoutingSpec extends WordSpec with MustMatchers {
latch.await(10 seconds)
count.get must be(loops)
(pool !! ActorPool.Stat).asInstanceOf[Option[ActorPool.Stats]].get.size must be >= (3)
(pool ? ActorPool.Stat).as[ActorPool.Stats].get.size must be >= (3)
pool.stop()
}
@ -458,7 +459,7 @@ class RoutingSpec extends WordSpec with MustMatchers {
sleepFor(5 millis)
val z = (pool !! ActorPool.Stat).asInstanceOf[Option[ActorPool.Stats]].get.size
val z = (pool ? ActorPool.Stat).as[ActorPool.Stats].get.size
z must be >= (2)
@ -469,7 +470,7 @@ class RoutingSpec extends WordSpec with MustMatchers {
sleepFor(500 millis)
}
(pool !! ActorPool.Stat).asInstanceOf[Option[ActorPool.Stats]].get.size must be <= (z)
(pool ? ActorPool.Stat).as[ActorPool.Stats].get.size must be <= (z)
pool.stop()
}

View file

@ -8,7 +8,7 @@ import org.scalatest.matchers.MustMatchers
class Ticket703Spec extends WordSpec with MustMatchers {
"A !!! call to an actor pool" should {
"A ? call to an actor pool" should {
"reuse the proper timeout" in {
val actorPool = actorOf(
new Actor with DefaultActorPool with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with BasicNoBackoffFilter {
@ -28,7 +28,7 @@ class Ticket703Spec extends WordSpec with MustMatchers {
}
})
}).start()
(actorPool.!!![String]("Ping", 7000)).await.result must be === Some("Response")
(actorPool.?("Ping")(timeout = 7000)).await.result must be === Some("Response")
}
}
}
}

View file

@ -10,7 +10,6 @@ import akka.config._
import Config._
import akka.util.{ ListenerManagement, ReflectiveAccess, Duration, Helpers }
import ReflectiveAccess._
import Helpers.{ narrow, narrowSilently }
import akka.remoteinterface.RemoteSupport
import akka.japi.{ Creator, Procedure }
import akka.AkkaException
@ -74,10 +73,10 @@ case object Kill extends AutoReceivedMessage with LifeCycleMessage
case object ReceiveTimeout extends LifeCycleMessage
case class MaximumNumberOfRestartsWithinTimeRangeReached(
@BeanProperty val victim: ActorRef,
@BeanProperty val maxNrOfRetries: Option[Int],
@BeanProperty val withinTimeRange: Option[Int],
@BeanProperty val lastExceptionCausingRestart: Throwable) extends LifeCycleMessage
@BeanProperty victim: ActorRef,
@BeanProperty maxNrOfRetries: Option[Int],
@BeanProperty withinTimeRange: Option[Int],
@BeanProperty lastExceptionCausingRestart: Throwable) extends LifeCycleMessage
// Exceptions for Actors
class ActorStartException private[akka] (message: String, cause: Throwable = null) extends AkkaException(message, cause)
@ -130,7 +129,7 @@ object Actor extends ListenerManagement {
subclassAudits synchronized { subclassAudits.clear() }
}
}
Runtime.getRuntime.addShutdownHook(new Thread(hook))
Runtime.getRuntime.addShutdownHook(new Thread(hook, "akka-shutdown-hook"))
hook
}
@ -146,6 +145,8 @@ object Actor extends ListenerManagement {
def apply(timeout: Long) = new Timeout(timeout)
def apply(length: Long, unit: TimeUnit) = new Timeout(length, unit)
implicit def durationToTimeout(duration: Duration) = new Timeout(duration)
implicit def intToTimeout(timeout: Int) = new Timeout(timeout)
implicit def longToTimeout(timeout: Long) = new Timeout(timeout)
}
private[akka] val TIMEOUT = Duration(config.getInt("akka.actor.timeout", 5), TIME_UNIT).toMillis
@ -282,7 +283,7 @@ object Actor extends ListenerManagement {
* </pre>
*/
def actorOf[T <: Actor](creator: T, address: String): ActorRef = {
createActor(address, () new LocalActorRef(() creator, address))
createActor(address, () new LocalActorRef(() creator, address, Transient))
}
/**
@ -305,7 +306,7 @@ object Actor extends ListenerManagement {
* JAVA API
*/
def actorOf[T <: Actor](creator: Creator[T], address: String): ActorRef = {
createActor(address, () new LocalActorRef(() creator.create, address))
createActor(address, () new LocalActorRef(() creator.create, address, Transient))
}
/**
@ -333,25 +334,6 @@ object Actor extends ListenerManagement {
}).start() ! Spawn
}
/**
* Implicitly converts the given Option[Any] to a AnyOptionAsTypedOption which offers the method <code>as[T]</code>
* to convert an Option[Any] to an Option[T].
*/
implicit def toAnyOptionAsTypedOption(anyOption: Option[Any]) = new AnyOptionAsTypedOption(anyOption)
/**
* Implicitly converts the given Future[_] to a AnyOptionAsTypedOption which offers the method <code>as[T]</code>
* to convert an Option[Any] to an Option[T].
* This means that the following code is equivalent:
* (actor !! "foo").as[Int] (Deprecated)
* and
* (actor !!! "foo").as[Int] (Recommended)
*/
implicit def futureToAnyOptionAsTypedOption(anyFuture: Future[_]) = new AnyOptionAsTypedOption({
try { anyFuture.await } catch { case t: FutureTimeoutException }
anyFuture.resultOrException
})
private[akka] def createActor(address: String, actorFactory: () ActorRef): ActorRef = {
Address.validate(address)
registry.actorFor(address) match { // check if the actor for the address is already in the registry
@ -387,12 +369,18 @@ object Actor extends ListenerManagement {
"\nif so put it outside the class/trait, f.e. in a companion object," +
"\nOR try to change: 'actorOf[MyActor]' to 'actorOf(new MyActor)'.", cause)
}
}, address)
}, address, Transient)
}
private def newClusterActorRef(factory: () ActorRef, address: String, deploy: Deploy): ActorRef = {
deploy match {
case Deploy(configAdress, router, serializerClassName, Clustered(home, replication: Replication, state: State))
case Deploy(
configAdress, router, serializerClassName,
Clustered(
home,
replicas,
replication))
ClusterModule.ensureEnabled()
if (configAdress != address) throw new IllegalStateException(
@ -401,57 +389,68 @@ object Actor extends ListenerManagement {
"Remote server is not running")
val isHomeNode = DeploymentConfig.isHomeNode(home)
val replicas = DeploymentConfig.replicaValueFor(replication)
val nrOfReplicas = DeploymentConfig.replicaValueFor(replicas)
def serializerErrorDueTo(reason: String) =
throw new akka.config.ConfigurationException(
"Could not create Serializer object [" + serializerClassName +
"] for serialization of actor [" + address +
"] since " + reason)
val serializer: Serializer = serializerClassName match {
case null | "" | Format.`defaultSerializerName` Format.Default
case specialSerializer
ReflectiveAccess.getClassFor(specialSerializer) match {
case Right(clazz)
clazz.newInstance match {
case s: Serializer s
case other serializerErrorDueTo("class must be of type [akka.serialization.Serializer]")
}
case Left(exception)
val cause = exception match {
case i: InvocationTargetException i.getTargetException
case _ exception
}
serializerErrorDueTo(cause.toString)
}
}
val isStateful = state match {
case _: Stateless | Stateless false
case _: Stateful | Stateful true
}
if (isStateful && isHomeNode) { // stateful actor's home node
cluster
.use(address, serializer)
.getOrElse(throw new ConfigurationException(
"Could not check out actor [" + address + "] from cluster registry as a \"local\" actor"))
} else {
if (!cluster.isClustered(address)) { // add actor to cluster registry (if not already added)
cluster.store(factory().start(), replicas, false, serializer)
}
def storeActorAndGetClusterRef(replicationScheme: ReplicationScheme, serializer: Serializer): ActorRef = {
// add actor to cluster registry (if not already added)
if (!cluster.isClustered(address))
cluster.store(factory().start(), nrOfReplicas, replicationScheme, false, serializer)
// remote node (not home node), check out as ClusterActorRef
cluster.ref(address, DeploymentConfig.routerTypeFor(router))
}
val serializer = serializerFor(address, serializerClassName)
replication match {
case _: Transient | Transient
storeActorAndGetClusterRef(Transient, serializer)
case replication: Replication
if (isHomeNode) { // stateful actor's home node
cluster
.use(address, serializer)
.getOrElse(throw new ConfigurationException(
"Could not check out actor [" + address + "] from cluster registry as a \"local\" actor"))
} else {
// FIXME later manage different 'storage' (data grid) as well
storeActorAndGetClusterRef(replication, serializer)
}
}
case invalid throw new IllegalActorStateException(
"Could not create actor with address [" + address +
"], not bound to a valid deployment scheme [" + invalid + "]")
}
}
// FIXME move serializerFor method to ...?
def serializerFor(address: String, serializerClassName: String): Serializer = {
def serializerErrorDueTo(reason: String) =
throw new akka.config.ConfigurationException(
"Could not create Serializer object [" + serializerClassName +
"] for serialization of actor [" + address +
"] since " + reason)
val serializer: Serializer = serializerClassName match {
case null | "" | Format.`defaultSerializerName` Format.Default
case specialSerializer
ReflectiveAccess.getClassFor(specialSerializer) match {
case Right(clazz)
clazz.newInstance match {
case s: Serializer s
case other serializerErrorDueTo("class must be of type [akka.serialization.Serializer]")
}
case Left(exception)
val cause = exception match {
case i: InvocationTargetException i.getTargetException
case _ exception
}
serializerErrorDueTo(cause.toString)
}
}
serializer
}
}
/**
@ -470,7 +469,7 @@ object Actor extends ListenerManagement {
*
* <p/>
* Here you find functions like:
* - !, !!, !!! and forward
* - !, ? and forward
* - link, unlink, startLink etc
* - start, stop
* - etc.
@ -517,8 +516,7 @@ trait Actor {
val someSelf: Some[ActorRef] = {
val refStack = Actor.actorRefInCreation.get
if (refStack.isEmpty) throw new ActorInitializationException(
"ActorRef for instance of actor [" + getClass.getName + "] is not in scope." +
"\n\tYou can not create an instance of an actor explicitly using 'new MyActor'." +
"\n\tYou can not create an instance of an " + getClass.getName + " explicitly using 'new MyActor'." +
"\n\tYou have to use one of the factory methods in the 'Actor' object to create a new actor." +
"\n\tEither use:" +
"\n\t\t'val actor = Actor.actorOf[MyActor]', or" +
@ -527,9 +525,10 @@ trait Actor {
val ref = refStack.head
if (ref eq null)
throw new ActorInitializationException("Trying to create an instance of an Actor outside of a wrapping 'actorOf'")
throw new ActorInitializationException("Trying to create an instance of " + getClass.getName + " outside of a wrapping 'actorOf'")
else {
Actor.actorRefInCreation.set(refStack.push(null)) //Push a null marker so any subsequent calls to new Actor doesn't reuse this actor ref
// Push a null marker so any subsequent calls to new Actor doesn't reuse this actor ref
Actor.actorRefInCreation.set(refStack.push(null))
Some(ref)
}
}
@ -538,7 +537,7 @@ trait Actor {
* Option[ActorRef] representation of the 'self' ActorRef reference.
* <p/>
* Mainly for internal use, functions as the implicit sender references when invoking
* one of the message send functions ('!', '!!' and '!!!').
* one of the message send functions ('!' and '?').
*/
def optionSelf: Option[ActorRef] = someSelf
@ -661,17 +660,10 @@ trait Actor {
val behaviorStack = self.hotswap
msg match {
case l: AutoReceivedMessage
autoReceiveMessage(l)
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) //This is the only line that differs from processingbehavior
case l: AutoReceivedMessage autoReceiveMessage(l)
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) //This is the only line that differs from processingbehavior
}
}
@ -694,18 +686,3 @@ trait Actor {
private lazy val processingBehavior = receive //ProcessingBehavior is the original behavior
}
private[actor] class AnyOptionAsTypedOption(anyOption: Option[Any]) {
/**
* Convenience helper to cast the given Option of Any to an Option of the given type. Will throw a ClassCastException
* if the actual type is not assignable from the given one.
*/
def as[T]: Option[T] = narrow[T](anyOption)
/**
* Convenience helper to cast the given Option of Any to an Option of the given type. Will swallow a possible
* ClassCastException and return None in that case.
*/
def asSilently[T: Manifest]: Option[T] = narrowSilently[T](anyOption)
}

View file

@ -6,10 +6,13 @@ package akka.actor
import akka.event.EventHandler
import akka.dispatch._
import akka.config.Config
import akka.config._
import akka.config.Supervision._
import akka.util._
import akka.serialization.{ Format, Serializer }
import ReflectiveAccess._
import ClusterModule._
import DeploymentConfig.{ ReplicationScheme, Replication, Transient, WriteThrough, WriteBehind }
import java.net.InetSocketAddress
import java.util.concurrent.atomic.AtomicReference
@ -77,8 +80,8 @@ trait ActorRef extends ActorRefShared with ForwardableChannel with java.lang.Com
/**
* User overridable callback/setting.
* <p/>
* Defines the default timeout for '!!' and '!!!' invocations,
* e.g. the timeout for the future returned by the call to '!!' and '!!!'.
* Defines the default timeout for '?'/'ask' invocations,
* e.g. the timeout for the future returned by the call to '?'/'ask'.
*/
@deprecated("Will be replaced by implicit-scoped timeout on all methods that needs it, will default to timeout specified in config", "1.1")
@BeanProperty
@ -186,7 +189,7 @@ trait ActorRef extends ActorRefShared with ForwardableChannel with java.lang.Com
/**
* Akka Java API. <p/>
* The reference sender future of the last received message.
* Is defined if the message was sent with sent with '!!' or '!!!', else None.
* Is defined if the message was sent with sent with '?'/'ask', else None.
*/
def getSenderFuture: Option[Promise[Any]] = senderFuture
@ -222,60 +225,17 @@ trait ActorRef extends ActorRefShared with ForwardableChannel with java.lang.Com
/**
* Akka Java API. <p/>
* @see sendRequestReply(message: AnyRef, timeout: Long, sender: ActorRef)
* Uses the default timeout of the Actor (setTimeout()) and omits the sender reference
*/
def sendRequestReply(message: AnyRef): AnyRef = {
!!(message, timeout).getOrElse(throw new ActorTimeoutException(
"Message [" + message +
"]\n\tfrom [nowhere]\n\twith timeout [" + timeout +
"]\n\ttimed out."))
.asInstanceOf[AnyRef]
}
/**
* Akka Java API. <p/>
* @see sendRequestReply(message: AnyRef, timeout: Long, sender: ActorRef)
* Uses the default timeout of the Actor (setTimeout())
*/
def sendRequestReply(message: AnyRef, sender: ActorRef): AnyRef = sendRequestReply(message, timeout, sender)
/**
* Akka Java API. <p/>
* Sends a message asynchronously and waits on a future for a reply message under the hood.
* <p/>
* It waits on the reply either until it receives it or until the timeout expires
* (which will throw an ActorTimeoutException). E.g. send-and-receive-eventually semantics.
* <p/>
* <b>NOTE:</b>
* Use this method with care. In most cases it is better to use 'sendOneWay' together with 'getContext().getSender()' to
* implement request/response message exchanges.
* <p/>
* If you are sending messages using <code>sendRequestReply</code> then you <b>have to</b> use <code>getContext().reply(..)</code>
* to send a reply message to the original sender. If not then the sender will block until the timeout expires.
*/
def sendRequestReply(message: AnyRef, timeout: Long, sender: ActorRef): AnyRef = {
?(message)(sender, Actor.Timeout(timeout)).as[AnyRef].getOrElse(throw new ActorTimeoutException(
"Message [" + message +
"]\n\tfrom [" + (if (sender ne null) sender.address else "nowhere") +
"]\n\twith timeout [" + timeout +
"]\n\ttimed out."))
.asInstanceOf[AnyRef]
}
/**
* Akka Java API. <p/>
* @see sendRequestReplyFuture(message: AnyRef, sender: ActorRef): Future[_]
* @see ask(message: AnyRef, sender: ActorRef): Future[_]
* Uses the Actors default timeout (setTimeout()) and omits the sender
*/
def sendRequestReplyFuture(message: AnyRef): Future[Any] = ?(message)
def ask(message: AnyRef): Future[AnyRef] = ask(message, timeout, null)
/**
* Akka Java API. <p/>
* @see sendRequestReplyFuture(message: AnyRef, sender: ActorRef): Future[_]
* @see ask(message: AnyRef, sender: ActorRef): Future[_]
* Uses the Actors default timeout (setTimeout())
*/
def sendRequestReplyFuture(message: AnyRef, sender: ActorRef): Future[Any] = ?(message)(sender)
def ask(message: AnyRef, sender: ActorRef): Future[AnyRef] = ask(message, timeout, sender)
/**
* Akka Java API. <p/>
@ -285,10 +245,10 @@ trait ActorRef extends ActorRefShared with ForwardableChannel with java.lang.Com
* Use this method with care. In most cases it is better to use 'sendOneWay' together with the 'getContext().getSender()' to
* implement request/response message exchanges.
* <p/>
* If you are sending messages using <code>sendRequestReplyFuture</code> then you <b>have to</b> use <code>getContext().reply(..)</code>
* If you are sending messages using <code>ask</code> then you <b>have to</b> use <code>getContext().reply(..)</code>
* to send a reply message to the original sender. If not then the sender will block until the timeout expires.
*/
def sendRequestReplyFuture(message: AnyRef, timeout: Long, sender: ActorRef): Future[Any] = ?(message)(sender, Actor.Timeout(timeout))
def ask(message: AnyRef, timeout: Long, sender: ActorRef): Future[AnyRef] = ?(message)(sender, Actor.Timeout(timeout)).asInstanceOf[Future[AnyRef]]
/**
* Akka Java API. <p/>
@ -367,17 +327,6 @@ trait ActorRef extends ActorRefShared with ForwardableChannel with java.lang.Com
*/
def startLink(actorRef: ActorRef): ActorRef
/**
* Returns the mailbox size.
*/
def mailboxSize = dispatcher.mailboxSize(this)
/**
* Akka Java API. <p/>
* Returns the mailbox size.
*/
def getMailboxSize: Int = mailboxSize
/**
* Returns the supervisor, if there is one.
*/
@ -458,7 +407,7 @@ trait ActorRef extends ActorRefShared with ForwardableChannel with java.lang.Com
that.asInstanceOf[ActorRef].uuid == uuid
}
override def toString = "Actor[" + address + ":" + uuid + "]"
override def toString = "Actor[%s:%s]".format(address, uuid)
}
/**
@ -466,27 +415,72 @@ trait ActorRef extends ActorRefShared with ForwardableChannel with java.lang.Com
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class LocalActorRef private[akka] (private[this] val actorFactory: () Actor, val address: String)
class LocalActorRef private[akka] (
private[this] val actorFactory: () Actor,
val address: String,
replicationScheme: ReplicationScheme)
extends ActorRef with ScalaActorRef {
protected[akka] val guard = new ReentrantGuard
@volatile
protected[akka] var _futureTimeout: Option[ScheduledFuture[AnyRef]] = None
@volatile
private[akka] lazy val _linkedActors = new ConcurrentHashMap[Uuid, ActorRef]
@volatile
private[akka] var _supervisor: Option[ActorRef] = None
@volatile
private var maxNrOfRetriesCount: Int = 0
@volatile
private var restartTimeWindowStartNanos: Long = 0L
@volatile
private var _mailbox: AnyRef = _
@volatile
private[akka] var _dispatcher: MessageDispatcher = Dispatchers.defaultGlobalDispatcher
protected[akka] val actorInstance = guard.withGuard { new AtomicReference[Actor](newActor) }
private val isReplicated: Boolean = replicationScheme match {
case _: Transient | Transient false
case _ true
}
// FIXME how to get the matching serializerClassName? Now default is used. Needed for transaction log snapshot
private val serializer = Actor.serializerFor(address, Format.defaultSerializerName)
private lazy val replicationStorage: Either[TransactionLog, AnyRef] = {
replicationScheme match {
case _: Transient | Transient
throw new IllegalStateException("Can not replicate 'transient' actor [" + toString + "]")
case Replication(storage, strategy)
val isWriteBehind = strategy match {
case _: WriteBehind | WriteBehind true
case _: WriteThrough | WriteThrough false
}
storage match {
case _: DeploymentConfig.TransactionLog | DeploymentConfig.TransactionLog
EventHandler.debug(this,
"Creating a transaction log for Actor [%s] with replication strategy [%s]"
.format(address, replicationScheme))
Left(transactionLog.newLogFor(_uuid.toString, isWriteBehind, replicationScheme, serializer))
case _: DeploymentConfig.DataGrid | DeploymentConfig.DataGrid
throw new ConfigurationException("Replication storage type \"data-grid\" is not yet supported")
case unknown
throw new ConfigurationException("Unknown replication storage type [" + unknown + "]")
}
}
}
//If it was started inside "newActor", initialize it
if (isRunning) initializeActorInstance
@ -499,8 +493,11 @@ class LocalActorRef private[akka] (private[this] val actorFactory: () ⇒ Actor,
__lifeCycle: LifeCycle,
__supervisor: Option[ActorRef],
__hotswap: Stack[PartialFunction[Any, Unit]],
__factory: () Actor) = {
this(__factory, __address)
__factory: () Actor,
__replicationStrategy: ReplicationScheme) = {
this(__factory, __address, __replicationStrategy)
_uuid = __uuid
timeout = __timeout
receiveTimeout = __receiveTimeout
@ -572,6 +569,10 @@ class LocalActorRef private[akka] (private[this] val actorFactory: () ⇒ Actor,
setActorSelfFields(actorInstance.get, null)
}
} //else if (isBeingRestarted) throw new ActorKilledException("Actor [" + toString + "] is being restarted.")
if (isReplicated) {
if (replicationStorage.isLeft) replicationStorage.left.get.delete()
}
}
}
@ -608,7 +609,6 @@ class LocalActorRef private[akka] (private[this] val actorFactory: () ⇒ Actor,
guard.withGuard {
if (_linkedActors.remove(actorRef.uuid) eq null)
throw new IllegalActorStateException("Actor [" + actorRef + "] is not a linked actor, can't unlink")
actorRef.supervisor = None
}
}
@ -692,7 +692,12 @@ class LocalActorRef private[akka] (private[this] val actorFactory: () ⇒ Actor,
throw e
}
}
} finally { guard.lock.unlock() }
} finally {
guard.lock.unlock()
if (isReplicated) {
if (replicationStorage.isLeft) replicationStorage.left.get.recordEntry(messageHandle, this)
}
}
}
protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable) {
@ -910,7 +915,7 @@ class LocalActorRef private[akka] (private[this] val actorFactory: () ⇒ Actor,
protected[akka] def checkReceiveTimeout() {
cancelReceiveTimeout()
if (receiveTimeout.isDefined && dispatcher.mailboxSize(this) <= 0) { //Only reschedule if desired and there are currently no more messages to be processed
if (receiveTimeout.isDefined && dispatcher.mailboxIsEmpty(this)) { //Only reschedule if desired and there are currently no more messages to be processed
_futureTimeout = Some(Scheduler.scheduleOnce(this, ReceiveTimeout, receiveTimeout.get, TimeUnit.MILLISECONDS))
}
}
@ -949,8 +954,6 @@ private[akka] case class RemoteActorRef private[akka] (
timeout = _timeout
// FIXME BAD, we should not have different ActorRefs
start()
def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit = {
@ -992,15 +995,12 @@ private[akka] case class RemoteActorRef private[akka] (
}
}
// ==== NOT SUPPORTED ====
@throws(classOf[java.io.ObjectStreamException])
private def writeReplace(): AnyRef = {
SerializedActorRef(uuid, address, remoteAddress.getAddress.getHostAddress, remoteAddress.getPort, timeout)
}
@deprecated("Will be removed without replacement, doesn't make any sense to have in the face of `become` and `unbecome`", "1.1")
def actorClass: Class[_ <: Actor] = unsupported
// ==== NOT SUPPORTED ====
def dispatcher_=(md: MessageDispatcher) {
unsupported
}
@ -1113,7 +1113,7 @@ trait ScalaActorRef extends ActorRefShared with ForwardableChannel { ref: ActorR
/**
* The reference sender future of the last received message.
* Is defined if the message was sent with sent with '!!' or '!!!', else None.
* Is defined if the message was sent with sent with '?'/'ask', else None.
*/
@deprecated("will be removed in 2.0, use channel instead", "1.2")
def senderFuture(): Option[Promise[Any]] = {
@ -1167,19 +1167,6 @@ trait ScalaActorRef extends ActorRefShared with ForwardableChannel { ref: ActorR
"Actor has not been started, you need to invoke 'actor.start()' before using it")
}
/**
* Sends a message asynchronously returns a future holding the eventual reply message.
* <p/>
* <b>NOTE:</b>
* Use this method with care. In most cases it is better to use '!' together with the 'sender' member field to
* implement request/response message exchanges.
* If you are sending messages using <code>!!!</code> then you <b>have to</b> use <code>self.reply(..)</code>
* to send a reply message to the original sender. If not then the sender will block until the timeout expires.
*/
@deprecated("return type is an illusion, use the more honest ? method", "1.2")
def !!![T](message: Any, timeout: Long = this.timeout)(implicit channel: UntypedChannel = NullChannel): Future[T] =
this.?(message)(channel, Actor.Timeout(timeout)).asInstanceOf[Future[T]]
/**
* Sends a message asynchronously, returning a future which may eventually hold the reply.
*/
@ -1192,7 +1179,7 @@ trait ScalaActorRef extends ActorRefShared with ForwardableChannel { ref: ActorR
/**
* Forwards the message and passes the original sender actor as the sender.
* <p/>
* Works with '!', '!!' and '!!!'.
* Works with '!' and '?'/'ask'.
*/
def forward(message: Any)(implicit channel: ForwardableChannel) = {
if (isRunning) {
@ -1229,6 +1216,8 @@ case class SerializedActorRef(val uuid: Uuid,
if (ReflectiveAccess.RemoteModule.isEnabled)
RemoteActorRef(new InetSocketAddress(hostname, port), address, timeout, None)
else
throw new IllegalStateException("Trying to deserialize ActorRef (" + this + ") but it's not found in the local registry and remoting is not enabled!")
throw new IllegalStateException(
"Trying to deserialize ActorRef (" + this +
") but it's not found in the local registry and remoting is not enabled!")
}
}

View file

@ -115,7 +115,7 @@ private[actor] final class ActorRegistry private[actor] () extends ListenerManag
}
/**
* View over the local actor registry.
* Projection over the local actor registry.
*/
class LocalActorRegistry(
private val actorsByAddress: ConcurrentHashMap[String, ActorRef],

View file

@ -62,8 +62,8 @@ object DeploymentConfig {
sealed trait Scope
case class Clustered(
home: Home = Host("localhost"),
replication: Replication = NoReplicas,
state: State = Stateful) extends Scope
replicas: Replicas = NoReplicas,
replication: ReplicationScheme = Transient) extends Scope
// For Java API
case class Local() extends Scope
@ -80,33 +80,60 @@ object DeploymentConfig {
case class IP(ipAddress: String) extends Home
// --------------------------------
// --- Replication
// --- Replicas
// --------------------------------
sealed trait Replication
case class Replicate(factor: Int) extends Replication {
if (factor < 1) throw new IllegalArgumentException("Replication factor can not be negative or zero")
sealed trait Replicas
case class Replicate(factor: Int) extends Replicas {
if (factor < 1) throw new IllegalArgumentException("Replicas factor can not be negative or zero")
}
// For Java API
case class AutoReplicate() extends Replication
case class NoReplicas() extends Replication
case class AutoReplicate() extends Replicas
case class NoReplicas() extends Replicas
// For Scala API
case object AutoReplicate extends Replication
case object NoReplicas extends Replication
case object AutoReplicate extends Replicas
case object NoReplicas extends Replicas
// --------------------------------
// --- State
// --- Replication
// --------------------------------
sealed trait State
sealed trait ReplicationScheme
// For Java API
case class Stateless() extends State
case class Stateful() extends State
case class Transient() extends ReplicationScheme
// For Scala API
case object Stateless extends State
case object Stateful extends State
case object Transient extends ReplicationScheme
case class Replication(
storage: ReplicationStorage,
strategy: ReplicationStrategy) extends ReplicationScheme
// --------------------------------
// --- ReplicationStorage
// --------------------------------
sealed trait ReplicationStorage
// For Java API
case class TransactionLog() extends ReplicationStorage
case class DataGrid() extends ReplicationStorage
// For Scala API
case object TransactionLog extends ReplicationStorage
case object DataGrid extends ReplicationStorage
// --------------------------------
// --- ReplicationStrategy
// --------------------------------
sealed trait ReplicationStrategy
// For Java API
case class WriteBehind() extends ReplicationStrategy
case class WriteThrough() extends ReplicationStrategy
// For Scala API
case object WriteBehind extends ReplicationStrategy
case object WriteThrough extends ReplicationStrategy
// --------------------------------
// --- Helper methods for parsing
@ -114,11 +141,11 @@ object DeploymentConfig {
def isHomeNode(home: Home): Boolean = home match {
case Host(hostname) hostname == Config.hostname
case IP(address) address == "0.0.0.0" // FIXME checking if IP address is on home node is missing
case IP(address) address == "0.0.0.0" || address == "127.0.0.1" // FIXME look up IP address from the system
case Node(nodename) nodename == Config.nodename
}
def replicaValueFor(replication: Replication): Int = replication match {
def replicaValueFor(replicas: Replicas): Int = replicas match {
case Replicate(replicas) replicas
case AutoReplicate -1
case AutoReplicate() -1
@ -141,6 +168,11 @@ object DeploymentConfig {
case LeastMessages() RouterType.LeastMessages
case c: CustomRouter throw new UnsupportedOperationException("routerTypeFor: " + c)
}
def isReplicationAsync(strategy: ReplicationStrategy): Boolean = strategy match {
case _: WriteBehind | WriteBehind true
case _: WriteThrough | WriteThrough false
}
}
/**
@ -346,13 +378,31 @@ object Deployer {
}
// --------------------------------
// akka.actor.deployment.<address>.clustered.stateless
// akka.actor.deployment.<address>.clustered.replication
// --------------------------------
val state =
if (clusteredConfig.getBool("stateless", false)) Stateless
else Stateful
clusteredConfig.getSection("replication") match {
case None
Some(Deploy(address, router, format, Clustered(home, replicas, Transient)))
Some(Deploy(address, router, format, Clustered(home, replicas, state)))
case Some(replicationConfig)
val storage = replicationConfig.getString("storage", "transaction-log") match {
case "transaction-log" TransactionLog
case "data-grid" DataGrid
case unknown
throw new ConfigurationException("Config option [" + addressPath +
".clustered.replication.storage] needs to be either [\"transaction-log\"] or [\"data-grid\"] - was [" +
unknown + "]")
}
val strategy = replicationConfig.getString("strategy", "write-through") match {
case "write-through" WriteThrough
case "write-behind" WriteBehind
case unknown
throw new ConfigurationException("Config option [" + addressPath +
".clustered.replication.strategy] needs to be either [\"write-through\"] or [\"write-behind\"] - was [" +
unknown + "]")
}
Some(Deploy(address, router, format, Clustered(home, replicas, Replication(storage, strategy))))
}
}
}
}
@ -415,7 +465,8 @@ object Address {
def validate(address: String) {
if (validAddressPattern.matcher(address).matches) true
else {
val e = new IllegalArgumentException("Address [" + address + "] is not valid, need to follow pattern [0-9a-zA-Z\\-\\_\\$]+")
val e = new IllegalArgumentException(
"Address [" + address + "] is not valid, need to follow pattern [0-9a-zA-Z\\-\\_\\$]+")
EventHandler.error(e, this, e.getMessage)
throw e
}

View file

@ -5,7 +5,7 @@ package akka.actor
*/
import akka.japi.{ Creator, Option JOption }
import akka.actor.Actor.{ actorOf, futureToAnyOptionAsTypedOption }
import akka.actor.Actor._
import akka.dispatch.{ MessageDispatcher, Dispatchers, Future, FutureTimeoutException }
import java.lang.reflect.{ InvocationTargetException, Method, InvocationHandler, Proxy }
import akka.util.{ Duration }
@ -47,7 +47,7 @@ object TypedActor {
actor ! m
null
case m if m.returnsFuture_?
actor !!! m
actor ? m
case m if m.returnsJOption_? || m.returnsOption_?
val f = actor ? m
try { f.await } catch { case _: FutureTimeoutException }
@ -57,7 +57,7 @@ object TypedActor {
case Some(Left(ex)) throw ex
}
case m
(actor !!! m).get
(actor ? m).get.asInstanceOf[AnyRef]
}
}
}

View file

@ -7,6 +7,7 @@ package akka.cluster
import akka.remoteinterface.RemoteSupport
import akka.serialization.Serializer
import akka.actor._
import DeploymentConfig._
import akka.dispatch.Future
import akka.config.Config
import akka.util._
@ -179,6 +180,13 @@ trait ClusterNode {
*/
def store[T <: Actor](address: String, actorClass: Class[T], format: Serializer): ClusterNode
/**
* Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store[T <: Actor](address: String, actorClass: Class[T], replicationScheme: ReplicationScheme, format: Serializer): ClusterNode
/**
* Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
@ -186,6 +194,13 @@ trait ClusterNode {
*/
def store[T <: Actor](address: String, actorClass: Class[T], replicationFactor: Int, format: Serializer): ClusterNode
/**
* Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store[T <: Actor](address: String, actorClass: Class[T], replicationFactor: Int, replicationScheme: ReplicationScheme, format: Serializer): ClusterNode
/**
* Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
@ -193,6 +208,13 @@ trait ClusterNode {
*/
def store[T <: Actor](address: String, actorClass: Class[T], serializeMailbox: Boolean, format: Serializer): ClusterNode
/**
* Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store[T <: Actor](address: String, actorClass: Class[T], replicationScheme: ReplicationScheme, serializeMailbox: Boolean, format: Serializer): ClusterNode
/**
* Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
@ -200,6 +222,13 @@ trait ClusterNode {
*/
def store[T <: Actor](address: String, actorClass: Class[T], replicationFactor: Int, serializeMailbox: Boolean, format: Serializer): ClusterNode
/**
* Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store[T <: Actor](address: String, actorClass: Class[T], replicationFactor: Int, replicationScheme: ReplicationScheme, serializeMailbox: Boolean, format: Serializer): ClusterNode
/**
* Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
@ -207,6 +236,13 @@ trait ClusterNode {
*/
def store(actorRef: ActorRef, format: Serializer): ClusterNode
/**
* Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store(actorRef: ActorRef, replicationScheme: ReplicationScheme, format: Serializer): ClusterNode
/**
* Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
@ -214,6 +250,13 @@ trait ClusterNode {
*/
def store(actorRef: ActorRef, replicationFactor: Int, format: Serializer): ClusterNode
/**
* Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store(actorRef: ActorRef, replicationFactor: Int, replicationScheme: ReplicationScheme, format: Serializer): ClusterNode
/**
* Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
@ -221,11 +264,23 @@ trait ClusterNode {
*/
def store(actorRef: ActorRef, serializeMailbox: Boolean, format: Serializer): ClusterNode
/**
* Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store(actorRef: ActorRef, replicationScheme: ReplicationScheme, serializeMailbox: Boolean, format: Serializer): ClusterNode
/**
* Needed to have reflection through structural typing work.
*/
def store(actorRef: ActorRef, replicationFactor: Int, serializeMailbox: Boolean, format: AnyRef): ClusterNode
/**
* Needed to have reflection through structural typing work.
*/
def store(actorRef: ActorRef, replicationFactor: Int, replicationScheme: ReplicationScheme, serializeMailbox: Boolean, format: AnyRef): ClusterNode
/**
* Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
@ -233,6 +288,13 @@ trait ClusterNode {
*/
def store(actorRef: ActorRef, replicationFactor: Int, serializeMailbox: Boolean, format: Serializer): ClusterNode
/**
* Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store(actorRef: ActorRef, replicationFactor: Int, replicationScheme: ReplicationScheme, serializeMailbox: Boolean, format: Serializer): ClusterNode
/**
* Removes actor with uuid from the cluster.
*/
@ -262,13 +324,13 @@ trait ClusterNode {
* Checks out an actor for use on this node, e.g. checked out as a 'LocalActorRef' but it makes it available
* for remote access through lookup by its UUID.
*/
def use[T <: Actor](actorAddress: String): Option[LocalActorRef]
def use[T <: Actor](actorAddress: String): Option[ActorRef]
/**
* Checks out an actor for use on this node, e.g. checked out as a 'LocalActorRef' but it makes it available
* for remote access through lookup by its UUID.
*/
def use[T <: Actor](actorAddress: String, format: Serializer): Option[LocalActorRef]
def use[T <: Actor](actorAddress: String, format: Serializer): Option[ActorRef]
/**
* Using (checking out) all actors with a specific UUID on all nodes in the cluster.

View file

@ -110,6 +110,8 @@ class Dispatcher(
*/
protected def getMailbox(receiver: ActorRef) = receiver.mailbox.asInstanceOf[MessageQueue with ExecutableMailbox]
def mailboxIsEmpty(actorRef: ActorRef): Boolean = getMailbox(actorRef).isEmpty
override def mailboxSize(actorRef: ActorRef) = getMailbox(actorRef).size
def createMailbox(actorRef: ActorRef): AnyRef = mailboxType match {

View file

@ -278,7 +278,7 @@ sealed trait Future[+T] {
* continuation until the result is available.
*
* If this Future is untyped (a Future[Nothing]), a type parameter must be explicitly provided or
* execution will fail. The normal result of getting a Future from an ActorRef using !!! will return
* execution will fail. The normal result of getting a Future from an ActorRef using ? will return
* an untyped Future.
*/
def apply[A >: T](): A @cps[Future[Any]] = shift(this flatMap (_: A Future[Any]))
@ -419,9 +419,9 @@ sealed trait Future[+T] {
* Example:
* <pre>
* val future1 = for {
* a <- actor !!! Req("Hello") collect { case Res(x: Int) => x }
* b <- actor !!! Req(a) collect { case Res(x: String) => x }
* c <- actor !!! Req(7) collect { case Res(x: String) => x }
* a <- actor ? Req("Hello") collect { case Res(x: Int) => x }
* b <- actor ? Req(a) collect { case Res(x: String) => x }
* c <- actor ? Req(7) collect { case Res(x: String) => x }
* } yield b + "-" + c
* </pre>
*/
@ -484,9 +484,9 @@ sealed trait Future[+T] {
* Example:
* <pre>
* val future1 = for {
* a: Int <- actor !!! "Hello" // returns 5
* b: String <- actor !!! a // returns "10"
* c: String <- actor !!! 7 // returns "14"
* a: Int <- actor ? "Hello" // returns 5
* b: String <- actor ? a // returns "10"
* c: String <- actor ? 7 // returns "14"
* } yield b + "-" + c
* </pre>
*/
@ -540,9 +540,9 @@ sealed trait Future[+T] {
* Example:
* <pre>
* val future1 = for {
* a: Int <- actor !!! "Hello" // returns 5
* b: String <- actor !!! a // returns "10"
* c: String <- actor !!! 7 // returns "14"
* a: Int <- actor ? "Hello" // returns 5
* b: String <- actor ? a // returns "10"
* c: String <- actor ? 7 // returns "14"
* } yield b + "-" + c
* </pre>
*/

View file

@ -234,6 +234,11 @@ trait MessageDispatcher {
*/
def mailboxSize(actorRef: ActorRef): Int
/**
* Returns the "current" emptiness status of the mailbox for the specified actor
*/
def mailboxIsEmpty(actorRef: ActorRef): Boolean
/**
* Returns the amount of futures queued for execution
*/

View file

@ -154,7 +154,7 @@ class MonitorableThreadFactory(val name: String) extends ThreadFactory {
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object MonitorableThread {
val DEFAULT_NAME = "MonitorableThread"
val DEFAULT_NAME = "MonitorableThread".intern
// FIXME use MonitorableThread.created and MonitorableThread.alive in monitoring
val created = new AtomicInteger

View file

@ -0,0 +1,39 @@
/**
* Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se>
*/
import akka.dispatch.{ FutureTimeoutException, Future }
import akka.util.Helpers.{ narrow, narrowSilently }
package object akka {
/**
* Implicitly converts the given Option[Any] to a AnyOptionAsTypedOption which offers the method <code>as[T]</code>
* to convert an Option[Any] to an Option[T].
*/
implicit def toAnyOptionAsTypedOption(anyOption: Option[Any]) = new AnyOptionAsTypedOption(anyOption)
/**
* Implicitly converts the given Future[_] to a AnyOptionAsTypedOption which offers the method <code>as[T]</code>
* to convert an Option[Any] to an Option[T].
* This means that the following code is equivalent:
* (actor ? "foo").as[Int] (Recommended)
*/
implicit def futureToAnyOptionAsTypedOption(anyFuture: Future[_]) = new AnyOptionAsTypedOption({
try { anyFuture.await } catch { case t: FutureTimeoutException }
anyFuture.resultOrException
})
private[akka] class AnyOptionAsTypedOption(anyOption: Option[Any]) {
/**
* Convenience helper to cast the given Option of Any to an Option of the given type. Will throw a ClassCastException
* if the actual type is not assignable from the given one.
*/
def as[T]: Option[T] = narrow[T](anyOption)
/**
* Convenience helper to cast the given Option of Any to an Option of the given type. Will swallow a possible
* ClassCastException and return None in that case.
*/
def asSilently[T: Manifest]: Option[T] = narrowSilently[T](anyOption)
}
}

View file

@ -117,7 +117,7 @@ trait SmallestMailboxSelector {
var take = if (partialFill) math.min(selectionCount, delegates.length) else selectionCount
while (take > 0) {
set = delegates.sortWith(_.mailboxSize < _.mailboxSize).take(take) ++ set //Question, doesn't this risk selecting the same actor multiple times?
set = delegates.sortWith((a, b) a.dispatcher.mailboxSize(a) < b.dispatcher.mailboxSize(b)).take(take) ++ set //Question, doesn't this risk selecting the same actor multiple times?
take -= set.size
}
@ -188,7 +188,7 @@ trait BoundedCapacitor {
trait MailboxPressureCapacitor {
def pressureThreshold: Int
def pressure(delegates: Seq[ActorRef]): Int =
delegates count { _.mailboxSize > pressureThreshold }
delegates count { a a.dispatcher.mailboxSize(a) > pressureThreshold }
}
/**

View file

@ -206,7 +206,7 @@ case class SmallestMailboxFirstIterator(val items: Seq[ActorRef]) extends Infini
def this(items: java.util.List[ActorRef]) = this(items.toList)
def hasNext = items != Nil
def next = items.reduceLeft((a1, a2) if (a1.mailboxSize < a2.mailboxSize) a1 else a2)
def next = items.reduceLeft((a1, a2) if (a1.dispatcher.mailboxSize(a1) < a2.dispatcher.mailboxSize(a2)) a1 else a2)
override def exists(f: ActorRef Boolean): Boolean = items.exists(f)
}

View file

@ -8,7 +8,7 @@ import akka.dispatch.{ Future, Promise, MessageInvocation }
import akka.config.{ Config, ModuleNotAvailableException }
import akka.remoteinterface.RemoteSupport
import akka.actor._
import DeploymentConfig.Deploy
import DeploymentConfig.{ Deploy, ReplicationScheme, ReplicationStrategy }
import akka.event.EventHandler
import akka.serialization.Format
import akka.cluster.ClusterNode
@ -62,6 +62,13 @@ object ReflectiveAccess {
None
}
lazy val transactionLogInstance: Option[TransactionLogObject] = getObjectFor("akka.cluster.TransactionLog$") match {
case Right(value) Some(value)
case Left(exception)
EventHandler.debug(this, exception.toString)
None
}
lazy val node: ClusterNode = {
ensureEnabled()
clusterInstance.get.node
@ -72,6 +79,11 @@ object ReflectiveAccess {
clusterDeployerInstance.get
}
lazy val transactionLog: TransactionLogObject = {
ensureEnabled()
transactionLogInstance.get
}
type ClusterDeployer = {
def init(deployments: List[Deploy])
def shutdown()
@ -94,6 +106,35 @@ object ReflectiveAccess {
def toBinary(obj: AnyRef): Array[Byte]
def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef
}
type TransactionLogObject = {
def newLogFor(
id: String,
isAsync: Boolean,
replicationScheme: ReplicationScheme,
format: Serializer): TransactionLog
def logFor(
id: String,
isAsync: Boolean,
replicationScheme: ReplicationScheme,
format: Serializer): TransactionLog
def shutdown()
}
type TransactionLog = {
def recordEntry(messageHandle: MessageInvocation, actorRef: ActorRef)
def recordEntry(entry: Array[Byte])
def recordSnapshot(snapshot: Array[Byte])
def entries: Vector[Array[Byte]]
def entriesFromLatestSnapshot: Tuple2[Array[Byte], Vector[Array[Byte]]]
def entriesInRange(from: Long, to: Long): Vector[Array[Byte]]
def latestEntryId: Long
def latestSnapshotId: Long
def delete()
def close()
}
}
/**

View file

@ -40,10 +40,10 @@ class TypedConsumerPublishRequestorTest extends JUnitSuite {
@Test
def shouldReceiveOneConsumerMethodRegisteredEvent = {
Actor.registry.addListener(requestor)
val latch = (publisher !! SetExpectedTestMessageCount(1)).as[CountDownLatch].get
val latch = (publisher ? SetExpectedTestMessageCount(1)).as[CountDownLatch].get
val obj = TypedActor.typedActorOf(classOf[SampleTypedSingleConsumer], classOf[SampleTypedSingleConsumerImpl], defaultConfiguration)
assert(latch.await(5000, TimeUnit.MILLISECONDS))
val event = (publisher !! GetRetainedMessage).as[ConsumerMethodRegistered].get
val event = (publisher ? GetRetainedMessage).as[ConsumerMethodRegistered].get
assert(event.endpointUri === "direct:foo")
assert(event.typedActor === obj)
assert(event.methodName === "foo")
@ -52,11 +52,11 @@ class TypedConsumerPublishRequestorTest extends JUnitSuite {
@Test
def shouldReceiveOneConsumerMethodUnregisteredEvent = {
val obj = TypedActor.typedActorOf(classOf[SampleTypedSingleConsumer], classOf[SampleTypedSingleConsumerImpl], defaultConfiguration)
val latch = (publisher !! SetExpectedTestMessageCount(1)).as[CountDownLatch].get
val latch = (publisher ? SetExpectedTestMessageCount(1)).as[CountDownLatch].get
Actor.registry.addListener(requestor)
TypedActor.stop(obj)
assert(latch.await(5000, TimeUnit.MILLISECONDS))
val event = (publisher !! GetRetainedMessage).as[ConsumerMethodUnregistered].get
val event = (publisher ? GetRetainedMessage).as[ConsumerMethodUnregistered].get
assert(event.endpointUri === "direct:foo")
assert(event.typedActor === obj)
assert(event.methodName === "foo")
@ -65,23 +65,23 @@ class TypedConsumerPublishRequestorTest extends JUnitSuite {
@Test
def shouldReceiveThreeConsumerMethodRegisteredEvents = {
Actor.registry.addListener(requestor)
val latch = (publisher !! SetExpectedTestMessageCount(3)).as[CountDownLatch].get
val latch = (publisher ? SetExpectedTestMessageCount(3)).as[CountDownLatch].get
val obj = TypedActor.typedActorOf(classOf[SampleTypedConsumer], classOf[SampleTypedConsumerImpl], defaultConfiguration)
assert(latch.await(5000, TimeUnit.MILLISECONDS))
val request = GetRetainedMessages(_.isInstanceOf[ConsumerMethodRegistered])
val events = (publisher !! request).as[List[ConsumerMethodRegistered]].get
val events = (publisher ? request).as[List[ConsumerMethodRegistered]].get
assert(events.map(_.method.getName).sortWith(_ < _) === List("m2", "m3", "m4"))
}
@Test
def shouldReceiveThreeConsumerMethodUnregisteredEvents = {
val obj = TypedActor.typedActorOf(classOf[SampleTypedConsumer], classOf[SampleTypedConsumerImpl], defaultConfiguration)
val latch = (publisher !! SetExpectedTestMessageCount(3)).as[CountDownLatch].get
val latch = (publisher ? SetExpectedTestMessageCount(3)).as[CountDownLatch].get
Actor.registry.addListener(requestor)
TypedActor.stop(obj)
assert(latch.await(5000, TimeUnit.MILLISECONDS))
val request = GetRetainedMessages(_.isInstanceOf[ConsumerMethodUnregistered])
val events = (publisher !! request).as[List[ConsumerMethodUnregistered]].get
val events = (publisher ? request).as[List[ConsumerMethodUnregistered]].get
assert(events.map(_.method.getName).sortWith(_ < _) === List("m2", "m3", "m4"))
}
}

File diff suppressed because it is too large Load diff

View file

@ -29,6 +29,7 @@ import Helpers._
import akka.actor._
import Actor._
import Status._
import DeploymentConfig.{ ReplicationScheme, ReplicationStrategy, Transient, WriteThrough, WriteBehind }
import akka.event.EventHandler
import akka.dispatch.{ Dispatchers, Future }
import akka.remoteinterface._
@ -463,7 +464,15 @@ class DefaultClusterNode private[akka] (
* available durable store.
*/
def store[T <: Actor](address: String, actorClass: Class[T], format: Serializer): ClusterNode =
store(Actor.actorOf(actorClass, address).start, 0, false, format)
store(Actor.actorOf(actorClass, address).start, 0, Transient, false, format)
/**
* Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store[T <: Actor](address: String, actorClass: Class[T], replicationScheme: ReplicationScheme, format: Serializer): ClusterNode =
store(Actor.actorOf(actorClass, address).start, 0, replicationScheme, false, format)
/**
* Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated
@ -471,7 +480,15 @@ class DefaultClusterNode private[akka] (
* available durable store.
*/
def store[T <: Actor](address: String, actorClass: Class[T], replicationFactor: Int, format: Serializer): ClusterNode =
store(Actor.actorOf(actorClass, address).start, replicationFactor, false, format)
store(Actor.actorOf(actorClass, address).start, replicationFactor, Transient, false, format)
/**
* Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store[T <: Actor](address: String, actorClass: Class[T], replicationFactor: Int, replicationScheme: ReplicationScheme, format: Serializer): ClusterNode =
store(Actor.actorOf(actorClass, address).start, replicationFactor, replicationScheme, false, format)
/**
* Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated
@ -479,7 +496,15 @@ class DefaultClusterNode private[akka] (
* available durable store.
*/
def store[T <: Actor](address: String, actorClass: Class[T], serializeMailbox: Boolean, format: Serializer): ClusterNode =
store(Actor.actorOf(actorClass, address).start, 0, serializeMailbox, format)
store(Actor.actorOf(actorClass, address).start, 0, Transient, serializeMailbox, format)
/**
* Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store[T <: Actor](address: String, actorClass: Class[T], replicationScheme: ReplicationScheme, serializeMailbox: Boolean, format: Serializer): ClusterNode =
store(Actor.actorOf(actorClass, address).start, 0, replicationScheme, serializeMailbox, format)
/**
* Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated
@ -487,7 +512,15 @@ class DefaultClusterNode private[akka] (
* available durable store.
*/
def store[T <: Actor](address: String, actorClass: Class[T], replicationFactor: Int, serializeMailbox: Boolean, format: Serializer): ClusterNode =
store(Actor.actorOf(actorClass, address).start, replicationFactor, serializeMailbox, format)
store(Actor.actorOf(actorClass, address).start, replicationFactor, Transient, serializeMailbox, format)
/**
* Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store[T <: Actor](address: String, actorClass: Class[T], replicationFactor: Int, replicationScheme: ReplicationScheme, serializeMailbox: Boolean, format: Serializer): ClusterNode =
store(Actor.actorOf(actorClass, address).start, replicationFactor, replicationScheme, serializeMailbox, format)
/**
* Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated
@ -495,7 +528,15 @@ class DefaultClusterNode private[akka] (
* available durable store.
*/
def store(actorRef: ActorRef, format: Serializer): ClusterNode =
store(actorRef, 0, false, format)
store(actorRef, 0, Transient, false, format)
/**
* Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store(actorRef: ActorRef, replicationScheme: ReplicationScheme, format: Serializer): ClusterNode =
store(actorRef, 0, replicationScheme, false, format)
/**
* Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated
@ -503,7 +544,15 @@ class DefaultClusterNode private[akka] (
* available durable store.
*/
def store(actorRef: ActorRef, replicationFactor: Int, format: Serializer): ClusterNode =
store(actorRef, replicationFactor, false, format)
store(actorRef, replicationFactor, Transient, false, format)
/**
* Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store(actorRef: ActorRef, replicationFactor: Int, replicationScheme: ReplicationScheme, format: Serializer): ClusterNode =
store(actorRef, replicationFactor, replicationScheme, false, format)
/**
* Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated
@ -511,20 +560,47 @@ class DefaultClusterNode private[akka] (
* available durable store.
*/
def store(actorRef: ActorRef, serializeMailbox: Boolean, format: Serializer): ClusterNode =
store(actorRef, 0, serializeMailbox, format)
/**
* Needed to have reflection through structural typing work.
*/
def store(actorRef: ActorRef, replicationFactor: Int, serializeMailbox: Boolean, format: AnyRef): ClusterNode =
store(actorRef, replicationFactor, serializeMailbox, format.asInstanceOf[Serializer])
store(actorRef, 0, Transient, serializeMailbox, format)
/**
* Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store(actorRef: ActorRef, replicationFactor: Int, serializeMailbox: Boolean, format: Serializer): ClusterNode = if (isConnected.isOn) {
def store(actorRef: ActorRef, replicationFactor: Int, serializeMailbox: Boolean, format: Serializer): ClusterNode =
store(actorRef, replicationFactor, Transient, serializeMailbox, format)
/**
* Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store(actorRef: ActorRef, replicationScheme: ReplicationScheme, serializeMailbox: Boolean, format: Serializer): ClusterNode =
store(actorRef, 0, replicationScheme, serializeMailbox, format)
/**
* Needed to have reflection through structural typing work.
*/
def store(actorRef: ActorRef, replicationFactor: Int, replicationScheme: ReplicationScheme, serializeMailbox: Boolean, format: AnyRef): ClusterNode =
store(actorRef, replicationFactor, replicationScheme, serializeMailbox, format.asInstanceOf[Serializer])
/**
* Needed to have reflection through structural typing work.
*/
def store(actorRef: ActorRef, replicationFactor: Int, serializeMailbox: Boolean, format: AnyRef): ClusterNode =
store(actorRef, replicationFactor, Transient, serializeMailbox, format)
/**
* Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store(
actorRef: ActorRef,
replicationFactor: Int,
replicationScheme: ReplicationScheme,
serializeMailbox: Boolean,
format: Serializer): ClusterNode = if (isConnected.isOn) {
import akka.serialization.ActorSerialization._
@ -535,12 +611,14 @@ class DefaultClusterNode private[akka] (
EventHandler.debug(this,
"Storing actor [%s] with UUID [%s] in cluster".format(actorRef.address, uuid))
val actorBytes = if (shouldCompressData) LZF.compress(toBinary(actorRef, serializeMailbox)(format))
else toBinary(actorRef)(format)
val actorBytes =
if (shouldCompressData) LZF.compress(toBinary(actorRef, serializeMailbox, replicationScheme)(format))
else toBinary(actorRef, serializeMailbox, replicationScheme)(format)
val actorRegistryPath = actorRegistryPathFor(uuid)
// create UUID -> Array[Byte] for actor registry
if (zkClient.exists(actorRegistryPath)) zkClient.writeData(actorRegistryPath, actorBytes) // FIXME check for size and warn if too big
if (zkClient.exists(actorRegistryPath)) zkClient.writeData(actorRegistryPath, actorBytes) // FIXME Store actor bytes in Data Grid not ZooKeeper
else {
zkClient.retryUntilConnected(new Callable[Either[String, Exception]]() {
def call: Either[String, Exception] = {
@ -587,12 +665,10 @@ class DefaultClusterNode private[akka] (
.build
replicaConnectionsForReplicationFactor(replicationFactor) foreach { connection
(connection !! (command, remoteDaemonAckTimeout)) match {
(connection ? (command, remoteDaemonAckTimeout)).as[Status] match {
case Some(Success)
EventHandler.debug(this,
"Replica for [%s] successfully created on [%s]"
.format(actorRef.address, connection))
EventHandler.debug(this, "Replica for [%s] successfully created".format(actorRef.address))
case Some(Failure(cause))
EventHandler.error(cause, this, cause.toString)
@ -616,8 +692,9 @@ class DefaultClusterNode private[akka] (
releaseActorOnAllNodes(uuid)
locallyCheckedOutActors.remove(uuid)
// warning: ordering matters here
ignore[ZkNoNodeException](zkClient.deleteRecursive(actorAddressToUuidsPathFor(actorAddressForUuid(uuid)))) // remove ADDRESS to UUID mapping
ignore[ZkNoNodeException](zkClient.deleteRecursive(actorAddressToUuidsPathFor(actorAddressForUuid(uuid)))) // FIXME remove ADDRESS to UUID mapping?
ignore[ZkNoNodeException](zkClient.deleteRecursive(actorAtNodePathFor(nodeAddress.nodeName, uuid)))
ignore[ZkNoNodeException](zkClient.deleteRecursive(actorRegistryPathFor(uuid)))
ignore[ZkNoNodeException](zkClient.deleteRecursive(actorLocationsPathFor(uuid)))
@ -662,20 +739,17 @@ class DefaultClusterNode private[akka] (
* Checks out an actor for use on this node, e.g. checked out as a 'LocalActorRef' but it makes it available
* for remote access through lookup by its UUID.
*/
def use[T <: Actor](actorAddress: String): Option[LocalActorRef] = use(actorAddress, formatForActor(actorAddress))
def use[T <: Actor](actorAddress: String): Option[ActorRef] = use(actorAddress, formatForActor(actorAddress))
/**
* Checks out an actor for use on this node, e.g. checked out as a 'LocalActorRef' but it makes it available
* for remote access through lookup by its UUID.
*/
def use[T <: Actor](actorAddress: String, format: Serializer): Option[LocalActorRef] = if (isConnected.isOn) {
def use[T <: Actor](actorAddress: String, format: Serializer): Option[ActorRef] = if (isConnected.isOn) {
import akka.serialization.ActorSerialization._
actorUuidsForActorAddress(actorAddress) map { uuid
EventHandler.debug(this,
"Checking out actor with UUID [%s] to be used on node [%s] as local actor"
.format(uuid, nodeAddress.nodeName))
ignore[ZkNodeExistsException](zkClient.createPersistent(actorAtNodePathFor(nodeAddress.nodeName, uuid), true))
ignore[ZkNodeExistsException](zkClient.createEphemeral(actorLocationsPathFor(uuid, nodeAddress)))
@ -697,12 +771,12 @@ class DefaultClusterNode private[akka] (
}) match {
case Left(bytes)
locallyCheckedOutActors += (uuid -> bytes)
// FIXME switch to ReplicatedActorRef here
// val actor = new ReplicatedActorRef(fromBinary[T](bytes, remoteServerAddress)(format))
val actor = fromBinary[T](bytes, remoteServerAddress)(format)
// remoteService.register(UUID_PREFIX + uuid, actor) // FIXME is Actor.remote.register(UUID, ..) correct here?
EventHandler.debug(this,
"Checking out actor [%s] to be used on node [%s] as local actor"
.format(actor, nodeAddress.nodeName))
actor.start()
actor.asInstanceOf[LocalActorRef]
actor
case Right(exception) throw exception
}
} headOption // FIXME should not be an array at all coming here but an Option[ActorRef]
@ -715,14 +789,15 @@ class DefaultClusterNode private[akka] (
isConnected ifOn {
EventHandler.debug(this,
"Using (checking out) all actors with UUID [%s] on all nodes in cluster".format(uuid))
val command = RemoteDaemonMessageProtocol.newBuilder
.setMessageType(USE)
.setActorUuid(uuidToUuidProtocol(uuid))
.build
membershipNodes foreach { node
replicaConnections.get(node) foreach {
case (_, connection)
connection ! command
case (_, connection) connection ! command
}
}
}
@ -786,8 +861,8 @@ class DefaultClusterNode private[akka] (
def ref(actorAddress: String, router: RouterType): ActorRef = if (isConnected.isOn) {
val addresses = addressesForActor(actorAddress)
EventHandler.debug(this,
"Checking out cluster actor ref with address [%s] and router [%s] connected to [\n\t%s]"
.format(actorAddress, router, addresses.mkString("\n\t")))
"Checking out cluster actor ref with address [%s] and router [%s] on [%s] connected to [\n\t%s]"
.format(actorAddress, router, remoteServerAddress, addresses.map(_._2).mkString("\n\t")))
val actorRef = Router newRouter (router, addresses, actorAddress, Actor.TIMEOUT)
addresses foreach { case (_, address) clusterActorRefs.put(address, actorRef) }
@ -981,7 +1056,7 @@ class DefaultClusterNode private[akka] (
.setMessageType(FUNCTION_FUN0_ANY)
.setPayload(ByteString.copyFrom(Serializers.Java.toBinary(f)))
.build
val results = replicaConnectionsForReplicationFactor(replicationFactor) map (_ !!! message)
val results = replicaConnectionsForReplicationFactor(replicationFactor) map (_ ? message)
results.toList.asInstanceOf[List[Future[Any]]]
}
@ -1007,7 +1082,7 @@ class DefaultClusterNode private[akka] (
.setMessageType(FUNCTION_FUN1_ARG_ANY)
.setPayload(ByteString.copyFrom(Serializers.Java.toBinary((f, arg))))
.build
val results = replicaConnectionsForReplicationFactor(replicationFactor) map (_ !!! message)
val results = replicaConnectionsForReplicationFactor(replicationFactor) map (_ ? message)
results.toList.asInstanceOf[List[Future[Any]]]
}
@ -1230,7 +1305,7 @@ class DefaultClusterNode private[akka] (
homeAddress.setAccessible(true)
homeAddress.set(actor, Some(remoteServerAddress))
remoteService.register(uuid, actor) // FIXME is Actor.remote.register(UUID, ..) correct here?
remoteService.register(actorAddress, actor)
}
}
@ -1473,8 +1548,6 @@ object RemoteClusterDaemon {
val computeGridDispatcher = Dispatchers.newDispatcher("akka:cloud:cluster:compute-grid").build
}
// FIXME supervise RemoteClusterDaemon
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/

View file

@ -6,12 +6,17 @@ package akka.cluster
import Cluster._
import akka.actor._
import akka.actor.Actor._
import Actor._
import akka.dispatch._
import akka.util._
import ReflectiveAccess._
import ClusterModule._
import akka.event.EventHandler
import akka.dispatch.Future
import java.net.InetSocketAddress
import java.util.concurrent.atomic.AtomicReference
import java.util.{ Map JMap }
import com.eaio.uuid.UUID
@ -20,21 +25,20 @@ import com.eaio.uuid.UUID
*/
class ClusterActorRef private[akka] (
inetSocketAddresses: Array[Tuple2[UUID, InetSocketAddress]],
actorAddress: String,
timeout: Long,
val replicationStrategy: ReplicationStrategy)
extends RemoteActorRef(null, actorAddress, timeout, None) { // FIXME UGLY HACK - should not extend RemoteActorRef
this: ClusterActorRef with Router.Router
val address: String,
_timeout: Long)
extends ActorRef with ScalaActorRef { this: Router.Router
EventHandler.debug(this,
"Creating a ClusterActorRef for actor with address [%s] with connections [\n\t%s]"
.format(actorAddress, inetSocketAddresses.mkString("\n\t")))
timeout = _timeout
private[akka] val inetSocketAddressToActorRefMap = new AtomicReference[Map[InetSocketAddress, ActorRef]](
(Map[InetSocketAddress, ActorRef]() /: inetSocketAddresses) {
case (map, (uuid, inetSocketAddress)) map + (inetSocketAddress -> createRemoteActorRef(actorAddress, inetSocketAddress))
case (map, (uuid, inetSocketAddress)) map + (inetSocketAddress -> createRemoteActorRef(address, inetSocketAddress))
})
ClusterModule.ensureEnabled()
start()
def connections: Map[InetSocketAddress, ActorRef] = inetSocketAddressToActorRefMap.get
override def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit = {
@ -68,4 +72,53 @@ class ClusterActorRef private[akka] (
private def createRemoteActorRef(actorAddress: String, inetSocketAddress: InetSocketAddress) = {
RemoteActorRef(inetSocketAddress, actorAddress, Actor.TIMEOUT, None)
}
def start(): ActorRef = synchronized {
_status = ActorRefInternals.RUNNING
this
}
def stop() {
synchronized {
if (_status == ActorRefInternals.RUNNING) {
_status = ActorRefInternals.SHUTDOWN
postMessageToMailbox(RemoteActorSystemMessage.Stop, None)
}
}
}
// ==== NOT SUPPORTED ====
// FIXME move these methods and the same ones in RemoteActorRef to a base class - now duplicated
def dispatcher_=(md: MessageDispatcher) {
unsupported
}
def dispatcher: MessageDispatcher = unsupported
def link(actorRef: ActorRef) {
unsupported
}
def unlink(actorRef: ActorRef) {
unsupported
}
def startLink(actorRef: ActorRef): ActorRef = unsupported
def supervisor: Option[ActorRef] = unsupported
def linkedActors: JMap[Uuid, ActorRef] = unsupported
protected[akka] def mailbox: AnyRef = unsupported
protected[akka] def mailbox_=(value: AnyRef): AnyRef = unsupported
protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable) {
unsupported
}
protected[akka] def restart(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]) {
unsupported
}
protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]) {
unsupported
}
protected[akka] def invoke(messageHandle: MessageInvocation) {
unsupported
}
protected[akka] def supervisor_=(sup: Option[ActorRef]) {
unsupported
}
protected[akka] def actorInstance: AtomicReference[Actor] = unsupported
private def unsupported = throw new UnsupportedOperationException("Not supported for RemoteActorRef")
}

View file

@ -27,7 +27,7 @@ import java.util.concurrent.atomic.AtomicReference
/**
* A ClusterDeployer is responsible for deploying a Deploy.
*
* big question is: what does Deploy mean?
* FIXME Document: what does Deploy mean?
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
@ -35,11 +35,16 @@ object ClusterDeployer {
val clusterName = Cluster.name
val nodeName = Config.nodename
val clusterPath = "/%s" format clusterName
val clusterDeploymentLockPath = clusterPath + "/deployment-lock"
val deploymentPath = clusterPath + "/deployment"
val baseNodes = List(clusterPath, clusterDeploymentLockPath, deploymentPath)
val deploymentAddressPath = deploymentPath + "/%s"
val deploymentCoordinationPath = clusterPath + "/deployment-coordination"
val deploymentInProgressLockPath = deploymentCoordinationPath + "/in-progress"
val isDeploymentCompletedInClusterLockPath = deploymentCoordinationPath + "/completed" // should not be part of baseNodes
val baseNodes = List(clusterPath, deploymentPath, deploymentCoordinationPath, deploymentInProgressLockPath)
private val isConnected = new Switch(false)
private val deploymentCompleted = new CountDownLatch(1)
@ -49,7 +54,7 @@ object ClusterDeployer {
Cluster.connectionTimeout,
Cluster.defaultSerializer)
private val clusterDeploymentLockListener = new LockListener {
private val deploymentInProgressLockListener = new LockListener {
def lockAcquired() {
EventHandler.debug(this, "Clustered deployment started")
}
@ -60,13 +65,11 @@ object ClusterDeployer {
}
}
private val deploymentLock = new WriteLock(
zkClient.connection.getZookeeper, clusterDeploymentLockPath, null, clusterDeploymentLockListener) {
private val ownerIdField = classOf[WriteLock].getDeclaredField("ownerId")
ownerIdField.setAccessible(true)
def leader: String = ownerIdField.get(this).asInstanceOf[String]
}
private val deploymentInProgressLock = new WriteLock(
zkClient.connection.getZookeeper,
deploymentInProgressLockPath,
null,
deploymentInProgressLockListener)
private val systemDeployments: List[Deploy] = Nil
@ -79,6 +82,7 @@ object ClusterDeployer {
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))
@ -124,8 +128,6 @@ object ClusterDeployer {
}
private[akka] def init(deployments: List[Deploy]) {
println("===============================================================")
println("------------ INIT 1")
isConnected switchOn {
EventHandler.info(this, "Initializing cluster deployer")
@ -141,31 +143,21 @@ object ClusterDeployer {
}
}
println("------------ INIT 2")
val allDeployments = deployments ::: systemDeployments
///===========================================================
// FIXME need a flag 'deploymentDone' in ZK and to wrap the deployment in 'if (!deploymentDone) { .. }', since now the deployment is only protected by lock during the actual deployment, if node comes in later then deployment is repeated on that node again
///===========================================================
if (!isDeploymentCompletedInCluster) {
if (deploymentInProgressLock.lock()) {
// try to be the one doing the clustered deployment
EventHandler.info(this, "Deploying to cluster [\n" + allDeployments.mkString("\n\t") + "\n]")
allDeployments foreach (deploy(_)) // deploy
markDeploymentCompletedInCluster()
deploymentInProgressLock.unlock() // signal deployment complete
if (deploymentLock.lock()) {
println("------------ INIT 3")
// try to be the one doing the clustered deployment
EventHandler.info(this, "Deploying to cluster [\n" + allDeployments.mkString("\n\t") + "\n]")
println("------------ INIT 4")
allDeployments foreach (deploy(_)) // deploy
println("------------ INIT 5")
// FIXME need to set deployment done flag
deploymentLock.unlock() // signal deployment complete
} else {
println("------------ INIT WAITING")
deploymentCompleted.await() // wait until deployment is completed by other "master" node
} else {
deploymentCompleted.await() // wait until deployment is completed by other "master" node
}
}
println("------------ INIT 6")
// fetch clustered deployments and deploy them locally
fetchDeploymentsFromCluster foreach (LocalDeployer.deploy(_))
}
@ -183,14 +175,29 @@ object ClusterDeployer {
zkClient.writeData(path, deployment)
} catch {
case e: NullPointerException
handleError(new DeploymentException("Could not store deployment data [" + deployment + "] in ZooKeeper since client session is closed"))
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))
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")

View file

@ -1,104 +0,0 @@
package akka.cluster
/**
* Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se>
*/
import Cluster._
import akka.actor._
import akka.remote.MessageSerializer
import akka.event.EventHandler
import akka.config.Supervision._
import akka.dispatch._
import java.net.InetSocketAddress
import java.util.concurrent.atomic.AtomicReference
import java.util.{ Map JMap }
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
trait Replicable { this: Actor
}
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
sealed trait ReplicationStrategy
object ReplicationStrategy {
case object Transient extends ReplicationStrategy
case object WriteThrough extends ReplicationStrategy
case object WriteBehind extends ReplicationStrategy
}
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class ReplicatedActorRef private[akka] (actorRef: ActorRef, val address: String) extends ActorRef with ScalaActorRef {
private lazy val txLog = {
EventHandler.debug(this, "Creating a ReplicatedActorRef for Actor [%s]".format(address))
TransactionLog.newLogFor(uuid.toString)
}
def invoke(messageHandle: MessageInvocation) {
actorRef.invoke(messageHandle)
txLog.recordEntry(MessageSerializer.serialize(messageHandle.message).toByteArray)
}
def start(): ActorRef = {
EventHandler.debug(this, "Starting ReplicatedActorRef for Actor [%s] with transaction log [%s]"
.format(address, txLog.logId))
actorRef.start()
}
def stop() {
txLog.delete()
actorRef.stop()
}
override def setFaultHandler(handler: FaultHandlingStrategy) {
actorRef.setFaultHandler(handler)
}
override def getFaultHandler: FaultHandlingStrategy = actorRef.getFaultHandler()
override def setLifeCycle(lifeCycle: LifeCycle) {
actorRef.setLifeCycle(lifeCycle)
}
override def getLifeCycle: LifeCycle = actorRef.getLifeCycle
def dispatcher_=(md: MessageDispatcher) {
actorRef.dispatcher_=(md)
}
def dispatcher: MessageDispatcher = actorRef.dispatcher
def link(actorRef: ActorRef) {
actorRef.link(actorRef)
}
def unlink(actorRef: ActorRef) {
actorRef.unlink(actorRef)
}
def startLink(actorRef: ActorRef): ActorRef = actorRef.startLink(actorRef)
def supervisor: Option[ActorRef] = actorRef.supervisor
def linkedActors: JMap[Uuid, ActorRef] = actorRef.linkedActors
protected[akka] def postMessageToMailbox(message: Any, channel: UntypedChannel) {
actorRef.postMessageToMailbox(message, channel)
}
protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout(
message: Any,
timeout: Long,
channel: UntypedChannel): Future[Any] = actorRef.postMessageToMailboxAndCreateFutureResultWithTimeout(message, timeout, channel)
protected[akka] def actorInstance: AtomicReference[Actor] = actorRef.actorInstance
protected[akka] def supervisor_=(sup: Option[ActorRef]) {
actorRef.supervisor_=(sup)
}
protected[akka] def mailbox: AnyRef = actorRef.mailbox
protected[akka] def mailbox_=(value: AnyRef): AnyRef = actorRef.mailbox_=(value)
protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable) {
actorRef.handleTrapExit(dead, reason)
}
protected[akka] def restart(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]) {
actorRef.restart(reason, maxNrOfRetries, withinTimeRange)
}
protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]) {
actorRef.restartLinkedActors(reason, maxNrOfRetries, withinTimeRange)
}
}

View file

@ -27,12 +27,11 @@ object Router {
routerType: RouterType,
inetSocketAddresses: Array[Tuple2[UUID, InetSocketAddress]],
actorAddress: String,
timeout: Long,
replicationStrategy: ReplicationStrategy = ReplicationStrategy.WriteThrough): ClusterActorRef = {
timeout: Long): ClusterActorRef = {
routerType match {
case Direct new ClusterActorRef(inetSocketAddresses, actorAddress, timeout, replicationStrategy) with Direct
case Random new ClusterActorRef(inetSocketAddresses, actorAddress, timeout, replicationStrategy) with Random
case RoundRobin new ClusterActorRef(inetSocketAddresses, actorAddress, timeout, replicationStrategy) with RoundRobin
case Direct new ClusterActorRef(inetSocketAddresses, actorAddress, timeout) with Direct
case Random new ClusterActorRef(inetSocketAddresses, actorAddress, timeout) with Random
case RoundRobin new ClusterActorRef(inetSocketAddresses, actorAddress, timeout) with RoundRobin
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")
@ -57,7 +56,7 @@ object Router {
}
def route[T](message: Any, timeout: Long)(implicit sender: Option[ActorRef]): Future[T] = next match {
case Some(actor) actor.!!!(message, timeout)(sender)
case Some(actor) actor.?(message, timeout)(sender).asInstanceOf[Future[T]]
case _ throwNoConnectionsError()
}

182
akka-cluster/src/main/scala/akka/cluster/Storage.scala Normal file → Executable file
View file

@ -5,9 +5,8 @@ import akka.AkkaException
import org.apache.zookeeper.{ KeeperException, CreateMode }
import org.apache.zookeeper.data.Stat
import java.util.concurrent.ConcurrentHashMap
import org.apache.zookeeper.KeeperException.NoNodeException
import java.lang.UnsupportedOperationException
import annotation.tailrec
import java.lang.{ UnsupportedOperationException, RuntimeException }
/**
* Simple abstraction to store an Array of bytes based on some String key.
@ -15,36 +14,37 @@ import annotation.tailrec
* 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.
*
* TODO: Perhaps add a version to the store to prevent lost updates using optimistic locking.
* (This is supported by ZooKeeper).
* 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 given entry.
* Loads the VersionedData for the given key.
*
* @param key: the key of the data to load.
* @return the VersionedData for the given key.
* @throws NoNodeExistsException if the data with the given key doesn't exist.
* @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 data for the given key and version.
* Loads the VersionedData for the given key and version.
*
* @param key: the key of the data to load
* @param version the version of the data to load
* @throws NoNodeExistsException if the data with the given key doesn't exist.
* @throws VersioningMismatchStorageException if the version of the data is not the same as the given data.
* @param key: the key of the VersionedData to load
* @param version the version of the VersionedData to load
* @throws MissingDataException if the data with the given key doesn't exist.
* @throws VersioningException if the version of the data is not the same as the given data.
* @throws StorageException if anything goes wrong while accessing the storage
*/
def load(key: String, version: Long): VersionedData
/**
* Checks if a value with the given key exists.
* 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.
@ -57,19 +57,34 @@ trait Storage {
*
* @param key the key of the Data to insert.
* @param bytes the data to insert.
* @return the version of the inserted data
* @throws NodeExistsException when a Node with the given Key already exists.
* @return the VersionedData
* @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]): VersionedData
/**
* Stores a array of bytes based on some key.
* Inserts the data if there is no data for that key, or overwrites it if it is there.
*
* @throws MissingNodeException when the Node with the given key doesn't exist.
* 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 VersionedData that was stored.
* @throws StorageException if anything goes wrong while accessing the storage
*/
def update(key: String, bytes: Array[Byte]): VersionedData
def insertOrOverwrite(key: String, bytes: Array[Byte]): VersionedData
/**
* Overwrites the current data for the given key.
*
* @param key the key of the data to overwrite
* @param bytes the data to insert.
* @throws ` 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]): VersionedData
/**
* @throws StorageException if anything goes wrong while accessing the storage
@ -98,17 +113,17 @@ class StorageException(msg: String = null, cause: java.lang.Throwable = null) ex
* *
* A StorageException thrown when an operation is done on a non existing node.
*/
class MissingNodeException(msg: String = null, cause: java.lang.Throwable = null) extends StorageException(msg, cause)
class MissingDataException(msg: String = null, cause: java.lang.Throwable = null) extends StorageException(msg, cause)
/**
* A StorageException thrown when an operation is done on an existing node, but no node was expected.
*/
class NodeExistsException(msg: String = null, cause: java.lang.Throwable = null) extends StorageException(msg, cause)
class DataExistsException(msg: String = null, cause: java.lang.Throwable = null) extends StorageException(msg, cause)
/**
* A StorageException thrown when an operation causes an optimistic locking failure.
*/
class VersioningMismatchStorageException(msg: String = null, cause: java.lang.Throwable = null) extends StorageException(msg, cause)
class VersioningException(msg: String = null, cause: java.lang.Throwable = null) extends StorageException(msg, cause)
/**
* A Storage implementation based on ZooKeeper.
@ -117,59 +132,89 @@ class VersioningMismatchStorageException(msg: String = null, cause: java.lang.Th
* - so everything is written or nothing is written
* - is isolated, so threadsafe,
* but it will not participate in any transactions.
* //todo: unclear, is only a single connection used in the JVM??
*
*/
class ZooKeeperStorage(zkClient: AkkaZkClient) extends Storage {
def load(key: String) = try {
val arrayOfBytes: Array[Byte] = zkClient.connection.readData(key, new Stat, false)
//Some(arrayOfBytes)
throw new UnsupportedOperationException()
val stat = new Stat
val arrayOfBytes = zkClient.connection.readData(key, stat, false)
new VersionedData(arrayOfBytes, stat.getVersion)
} catch {
//todo: improved error messaged
case e: KeeperException.NoNodeException throw new MissingNodeException("Failed to load key", e)
case e: KeeperException throw new StorageException("failed to load key " + key, e)
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, version: Long) = {
throw new UnsupportedOperationException()
def load(key: String, expectedVersion: Long) = try {
val stat = new Stat
val arrayOfBytes = zkClient.connection.readData(key, stat, false)
if (stat.getVersion != expectedVersion) throw new VersioningException(
"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]): VersionedData = {
try {
zkClient.connection.create(key, bytes, CreateMode.PERSISTENT);
throw new UnsupportedOperationException()
zkClient.connection.create(key, bytes, CreateMode.PERSISTENT)
//todo: how to get hold of the reference.
val version: Long = 0
new VersionedData(bytes, version)
} catch {
//todo: improved error messaged
case e: KeeperException.NodeExistsException throw new NodeExistsException("failed to insert key " + key, e)
case e: KeeperException throw new StorageException("failed to insert key " + key, e)
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(key, false)
} catch {
//todo: improved error messaged
case e: KeeperException throw new StorageException("failed to check for existance on key " + key, e)
case e: KeeperException throw new StorageException(
String.format("Failed to check existance for key [%s]", key), e)
}
def update(key: String, versionedData: VersionedData) = try {
zkClient.connection.writeData(key, versionedData.data, versionedData.version.asInstanceOf[Int])
} catch {
//todo: improved error messaged
case e: KeeperException.BadVersionException throw new VersioningMismatchStorageException()
case e: KeeperException throw new StorageException("failed to check for existance on key " + key, e)
def update(key: String, versionedData: VersionedData) {
try {
zkClient.connection.writeData(key, versionedData.data, versionedData.version.asInstanceOf[Int])
} catch {
case e: KeeperException.BadVersionException throw new VersioningException(
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 update(key: String, bytes: Array[Byte]): VersionedData = {
def overwrite(key: String, bytes: Array[Byte]): VersionedData = {
try {
zkClient.connection.writeData(key, bytes)
throw new RuntimeException()
} catch {
//todo: improved error messaged
case e: KeeperException.NoNodeException throw new MissingNodeException("failed to update key ", e)
case e: KeeperException throw new StorageException("failed to update key ", e)
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)
}
}
}
@ -188,8 +233,8 @@ final class InMemoryStorage extends Storage {
def load(key: String) = {
val result = map.get(key)
if (result == null) throw new MissingNodeException(
String.format("Failed to load data for key [%s]: no data was found", key))
if (result == null) throw new MissingDataException(
String.format("Failed to load key [%s]: no data was found", key))
result
}
@ -197,8 +242,8 @@ final class InMemoryStorage extends Storage {
def load(key: String, expectedVersion: Long) = {
val result = load(key)
if (result.version != expectedVersion) throw new VersioningMismatchStorageException(
"Failed to load data for key [" + key + "]: version mismatch, expected [" + result.version + "] " +
if (result.version != expectedVersion) throw new VersioningException(
"Failed to load key [" + key + "]: version mismatch, expected [" + result.version + "] " +
"but found [" + expectedVersion + "]")
result
@ -211,7 +256,7 @@ final class InMemoryStorage extends Storage {
val result = new VersionedData(bytes, version)
val previous = map.putIfAbsent(key, result)
if (previous != null) throw new NodeExistsException(
if (previous != null) throw new DataExistsException(
String.format("Failed to insert key [%s]: the key already has been inserted previously", key))
result
@ -221,23 +266,36 @@ final class InMemoryStorage extends Storage {
def update(key: String, updatedData: VersionedData) {
val currentData = map.get(key)
if (currentData == null) throw new MissingNodeException(
String.format("Failed to update data for key [%s], no previous entry exist", key))
if (currentData == null) throw new MissingDataException(
String.format("Failed to update key [%s], no previous entry exist", key))
val expectedVersion = currentData.version + 1
if (expectedVersion != updatedData.version) throw new VersioningMismatchStorageException(
"Failed to update data for key [" + key + "]: version mismatch, expected [" + expectedVersion + "]" +
if (expectedVersion != updatedData.version) throw new VersioningException(
"Failed to update key [" + key + "]: version mismatch, expected [" + expectedVersion + "]" +
" but found [" + updatedData.version + "]")
if (!map.replace(key, currentData, updatedData)) update(key, updatedData)
}
def update(key: String, bytes: Array[Byte]): VersionedData = {
if (map.get(key) == null) throw new NoNodeException(
String.format("Failed to update key [%s]: no previous insert of this key exists", key))
@tailrec
def overwrite(key: String, bytes: Array[Byte]): VersionedData = {
val currentData = map.get(key)
//smap.put(key, bytes)
throw new UnsupportedOperationException()
if (currentData == null) throw new MissingDataException(
String.format("Failed to overwrite key [%s], no previous entry exist", key))
val newData = currentData.createUpdate(bytes)
if (map.replace(key, currentData, newData)) newData else overwrite(key, bytes)
}
def insertOrOverwrite(key: String, bytes: Array[Byte]): VersionedData = {
val version = InMemoryStorage.InitialVersion
val result = new VersionedData(bytes, version)
val previous = map.putIfAbsent(key, result)
if (previous == null) result
else overwrite(key, bytes)
}
}

View file

@ -9,16 +9,23 @@ 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, ReplicationStrategy, Transient, WriteThrough, WriteBehind }
import akka.event.EventHandler
import akka.dispatch.{ DefaultPromise, Promise }
import akka.AkkaException
import akka.dispatch.{ DefaultPromise, Promise, MessageInvocation }
import akka.remote.MessageSerializer
import akka.serialization.ActorSerialization._
import akka.cluster.zookeeper._
import akka.serialization.{ Serializer, Compression }
import Compression.LZF
import akka.serialization.ActorSerialization._
import java.util.Enumeration
import java.util.concurrent.atomic.AtomicLong
// 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
@ -41,25 +48,47 @@ class ReplicationException(message: String) extends AkkaException(message)
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class TransactionLog private (ledger: LedgerHandle, val id: String, val isAsync: Boolean) {
class TransactionLog private (
ledger: LedgerHandle,
val id: String,
val isAsync: Boolean,
replicationScheme: ReplicationScheme,
format: Serializer) {
import TransactionLog._
val logId = ledger.getId
val txLogPath = transactionLogNode + "/" + id
val snapshotPath = txLogPath + "/snapshot"
val nrOfEntries = new AtomicLong(0)
private val isOpen = new Switch(true)
/**
* TODO document method
*/
def recordEntry(messageHandle: MessageInvocation, actorRef: ActorRef) {
if (nrOfEntries.incrementAndGet % snapshotFrequency == 0) {
val snapshot =
// FIXME ReplicationStrategy Transient is always used
if (Cluster.shouldCompressData) LZF.compress(toBinary(actorRef, false, replicationScheme)(format))
else toBinary(actorRef, false, replicationScheme)(format)
recordSnapshot(snapshot)
}
recordEntry(MessageSerializer.serialize(messageHandle.message).toByteArray)
}
/**
* TODO document method
*/
def recordEntry(entry: Array[Byte]) {
if (isOpen.isOn) {
val bytes = if (Cluster.shouldCompressData) LZF.compress(entry)
else entry
try {
if (isAsync) {
ledger.asyncAddEntry(
entry,
bytes,
new AsyncCallback.AddCallback {
def addComplete(
returnCode: Int,
@ -73,7 +102,7 @@ class TransactionLog private (ledger: LedgerHandle, val id: String, val isAsync:
},
null)
} else {
handleReturnCode(ledger.addEntry(entry))
handleReturnCode(ledger.addEntry(bytes))
val entryId = ledger.getLastAddPushed
EventHandler.debug(this, "Writing entry [%s] to log [%s]".format(entryId, logId))
}
@ -88,10 +117,12 @@ class TransactionLog private (ledger: LedgerHandle, val id: String, val isAsync:
*/
def recordSnapshot(snapshot: Array[Byte]) {
if (isOpen.isOn) {
val bytes = if (Cluster.shouldCompressData) LZF.compress(snapshot)
else snapshot
try {
if (isAsync) {
ledger.asyncAddEntry(
snapshot,
bytes,
new AsyncCallback.AddCallback {
def addComplete(
returnCode: Int,
@ -104,7 +135,7 @@ class TransactionLog private (ledger: LedgerHandle, val id: String, val isAsync:
},
null)
} else {
handleReturnCode(ledger.addEntry(snapshot))
handleReturnCode(ledger.addEntry(bytes))
storeSnapshotMetaDataInZooKeeper(ledger.getLastAddPushed)
}
} catch {
@ -121,7 +152,7 @@ class TransactionLog private (ledger: LedgerHandle, val id: String, val isAsync:
/**
* TODO document method
*/
def entriesFromLatestSnapshot: Tuple2[Array[Byte], Vector[Array[Byte]]] = {
def toByteArraysLatestSnapshot: (Array[Byte], Vector[Array[Byte]]) = {
val snapshotId = latestSnapshotId
EventHandler.debug(this,
"Reading entries from snapshot id [%s] for log [%s]".format(snapshotId, logId))
@ -133,9 +164,9 @@ class TransactionLog private (ledger: LedgerHandle, val id: String, val isAsync:
*/
def entriesInRange(from: Long, to: Long): Vector[Array[Byte]] = if (isOpen.isOn) {
try {
if (from < 0) throw new IllegalArgumentException("'from' can't be negative [" + from + "]")
if (to < 0) throw new IllegalArgumentException("'to' can't be negative [" + from + "]")
if (to < from) throw new IllegalArgumentException("'to' can't be smaller than 'from' [" + from + "," + to + "]")
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))
@ -150,10 +181,7 @@ class TransactionLog private (ledger: LedgerHandle, val id: String, val isAsync:
enumeration: Enumeration[LedgerEntry],
ctx: AnyRef) {
val future = ctx.asInstanceOf[Promise[Vector[Array[Byte]]]]
var entries = Vector[Array[Byte]]()
while (enumeration.hasMoreElements) {
entries = entries :+ enumeration.nextElement.getEntry
}
val entries = toByteArrays(enumeration)
if (returnCode == BKException.Code.OK) future.completeWithResult(entries)
else future.completeWithException(BKException.create(returnCode))
}
@ -161,12 +189,7 @@ class TransactionLog private (ledger: LedgerHandle, val id: String, val isAsync:
future)
await(future)
} else {
val enumeration = ledger.readEntries(from, to)
var entries = Vector[Array[Byte]]()
while (enumeration.hasMoreElements) {
entries = entries :+ enumeration.nextElement.getEntry
}
entries
toByteArrays(ledger.readEntries(from, to))
}
} catch {
case e handleError(e)
@ -190,8 +213,7 @@ class TransactionLog private (ledger: LedgerHandle, val id: String, val isAsync:
} catch {
case e: ZkNoNodeException
handleError(new ReplicationException(
"Transaction log for UUID [" + id +
"] does not have a snapshot recorded in ZooKeeper"))
"Transaction log for UUID [" + id + "] does not have a snapshot recorded in ZooKeeper"))
case e handleError(e)
}
}
@ -208,7 +230,7 @@ class TransactionLog private (ledger: LedgerHandle, val id: String, val isAsync:
logId,
new AsyncCallback.DeleteCallback {
def deleteComplete(returnCode: Int, ctx: AnyRef) {
handleReturnCode(returnCode)
(returnCode)
}
},
null)
@ -248,6 +270,18 @@ class TransactionLog private (ledger: LedgerHandle, val id: String, val isAsync:
}
}
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 (Cluster.shouldCompressData) LZF.uncompress(bytes)
else bytes
entries = entries :+ entry
}
entries
}
private def storeSnapshotMetaDataInZooKeeper(snapshotId: Long) {
if (isOpen.isOn) {
try {
@ -265,8 +299,7 @@ class TransactionLog private (ledger: LedgerHandle, val id: String, val isAsync:
"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))
EventHandler.debug(this, "Writing snapshot [%s] to log [%s]".format(snapshotId, logId))
} else transactionClosedError
}
@ -292,12 +325,13 @@ object TransactionLog {
case "CRC32" BookKeeper.DigestType.CRC32
case "MAC" BookKeeper.DigestType.MAC
case unknown throw new ConfigurationException(
"akka.cluster.replication.digest-type is invalid [" + unknown + "]")
"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 timeout = 5000 // FIXME make configurable
val snapshotFrequency = config.getInt("akka.cluster.replication.snapshot-frequency", 1000)
val timeout = Duration(config.getInt("akka.cluster.replication.timeout", 30), TIME_UNIT).toMillis
private[akka] val transactionLogNode = "/transaction-log-ids"
@ -333,8 +367,13 @@ object TransactionLog {
(bk, zk)
}
private[akka] def apply(ledger: LedgerHandle, id: String, isAsync: Boolean = false) =
new TransactionLog(ledger, id, isAsync)
private[akka] def apply(
ledger: LedgerHandle,
id: String,
isAsync: Boolean,
replicationScheme: ReplicationScheme,
format: Serializer) =
new TransactionLog(ledger, id, isAsync, replicationScheme, format)
/**
* Shuts down the transaction log.
@ -355,7 +394,12 @@ object TransactionLog {
/**
* TODO document method
*/
def newLogFor(id: String, isAsync: Boolean = false): TransactionLog = {
def newLogFor(
id: String,
isAsync: Boolean,
replicationScheme: ReplicationScheme,
format: Serializer): TransactionLog = {
val txLogPath = transactionLogNode + "/" + id
val ledger = try {
@ -399,13 +443,18 @@ object TransactionLog {
}
EventHandler.info(this, "Created new transaction log [%s] for UUID [%s]".format(logId, id))
TransactionLog(ledger, id, isAsync)
TransactionLog(ledger, id, isAsync, replicationScheme, format)
}
/**
* TODO document method
*/
def logFor(id: String, isAsync: Boolean = false): TransactionLog = {
def logFor(
id: String,
isAsync: Boolean,
replicationScheme: ReplicationScheme,
format: Serializer): TransactionLog = {
val txLogPath = transactionLogNode + "/" + id
val logId = try {
@ -444,7 +493,7 @@ object TransactionLog {
case e handleError(e)
}
TransactionLog(ledger, id, isAsync)
TransactionLog(ledger, id, isAsync, replicationScheme, format)
}
private[akka] def await[T](future: Promise[T]): T = {
@ -489,15 +538,10 @@ object LocalBookKeeperEnsemble {
def shutdown() {
isRunning switchOff {
EventHandler.info(this, "Shutting down LocalBookKeeperEnsemble...")
println("***************************** 1")
localBookKeeper.bs.foreach(_.shutdown()) // stop bookies
println("***************************** 2")
localBookKeeper.zkc.close() // stop zk client
println("***************************** 3")
localBookKeeper.zks.shutdown() // stop zk server
println("***************************** 4")
localBookKeeper.serverFactory.shutdown() // stop zk NIOServer
println("***************************** 5")
EventHandler.info(this, "LocalBookKeeperEnsemble shut down successfully")
}
}

View file

@ -42,7 +42,7 @@ object PingPong {
count += 1
self reply Ball
} else {
self.sender.foreach(_ !! Stop)
self.sender.foreach(s (s ? Stop).await)
gameOverLatch.countDown
self.stop
}

View file

@ -0,0 +1,251 @@
package akka.cluster
import org.scalatest.matchers.MustMatchers
import org.scalatest.WordSpec
import akka.cluster.StorageTestUtils._
class InMemoryStorageSpec extends WordSpec with MustMatchers {
"unversioned load" must {
"throw MissingDataException if non existing key" in {
val store = new InMemoryStorage()
try {
store.load("foo")
fail()
} catch {
case e: MissingDataException
}
}
"return VersionedData if key existing" in {
val storage = new InMemoryStorage()
val key = "somekey"
val value = "somevalue".getBytes
storage.insert(key, value)
val result = storage.load(key)
//todo: strange that the implicit store is not found
assertContent(key, value, result.version)(storage)
}
}
"exist" must {
"return true if value exists" in {
val store = new InMemoryStorage()
val key = "somekey"
store.insert(key, "somevalue".getBytes)
store.exists(key) must be(true)
}
"return false if value not exists" in {
val store = new InMemoryStorage()
store.exists("somekey") must be(false)
}
}
"versioned load" must {
"throw MissingDataException if non existing key" in {
val store = new InMemoryStorage()
try {
store.load("foo", 1)
fail()
} catch {
case e: MissingDataException
}
}
"return VersionedData if key existing and exact version match" in {
val storage = new InMemoryStorage()
val key = "somekey"
val value = "somevalue".getBytes
val stored = storage.insert(key, value)
val result = storage.load(key, stored.version)
assert(result.version == stored.version)
assert(result.data == stored.data)
}
"throw VersioningException is version too new" in {
val storage = new InMemoryStorage()
val key = "somekey"
val value = "somevalue".getBytes
val stored = storage.insert(key, value)
try {
storage.load(key, stored.version + 1)
fail()
} catch {
case e: VersioningException
}
}
"throw VersioningException is version too old" in {
val storage = new InMemoryStorage()
val key = "somekey"
val value = "somevalue".getBytes
val stored = storage.insert(key, value)
try {
storage.load(key, stored.version - 1)
fail()
} catch {
case e: VersioningException
}
}
}
"insert" must {
"place a new value when non previously existed" in {
val storage = new InMemoryStorage()
val key = "somekey"
val oldValue = "oldvalue".getBytes
storage.insert(key, oldValue)
val result = storage.load(key)
assertContent(key, oldValue)(storage)
assert(InMemoryStorage.InitialVersion == result.version)
}
"throw MissingDataException when there already exists an entry with the same key" in {
val storage = new InMemoryStorage()
val key = "somekey"
val oldValue = "oldvalue".getBytes
val oldVersionedData = storage.insert(key, oldValue)
val newValue = "newValue".getBytes
try {
storage.insert(key, newValue)
fail()
} catch {
case e: DataExistsException
}
//make sure that the old value was not changed
assert(oldVersionedData == storage.load(key))
}
}
"update" must {
"throw MissingDataException when no node exists" in {
val storage = new InMemoryStorage()
val key = "somekey"
try {
storage.update(key, new VersionedData("somevalue".getBytes, 1))
fail()
} catch {
case e: MissingDataException
}
}
"replace if previous value exists and no other updates have been done" in {
val storage = new InMemoryStorage()
//do the initial insert
val key = "foo"
val oldValue = "insert".getBytes
val insert = storage.insert(key, oldValue)
//do the update the will be the cause of the conflict.
val updateValue = "update".getBytes
val update = insert.createUpdate(updateValue)
storage.update(key, update)
assertContent(key, update.data, update.version)(storage)
}
"throw VersioningException when already overwritten" in {
val storage = new InMemoryStorage()
//do the initial insert
val key = "foo"
val oldValue = "insert".getBytes
val insert = storage.insert(key, oldValue)
//do the update the will be the cause of the conflict.
val otherUpdateValue = "otherupdate".getBytes
val otherUpdate = insert.createUpdate(otherUpdateValue)
storage.update(key, otherUpdate)
val update = insert.createUpdate("update".getBytes)
try {
storage.update(key, update)
fail()
} catch {
case e: VersioningException
}
assertContent(key, otherUpdate.data, otherUpdate.version)(storage)
}
}
"overwrite" must {
"throw MissingDataException when no node exists" in {
val storage = new InMemoryStorage()
val key = "somekey"
try {
storage.overwrite(key, "somevalue".getBytes)
fail()
} catch {
case e: MissingDataException
}
storage.exists(key) must be(false)
}
"succeed if previous value exist" in {
val storage = new InMemoryStorage()
val key = "somekey"
val oldValue = "oldvalue".getBytes
val newValue: Array[Byte] = "somevalue".getBytes
val initialInsert: VersionedData = storage.insert(key, oldValue)
val result: VersionedData = storage.overwrite(key, newValue)
assert(result.version == initialInsert.version + 1)
assert(result.data == newValue)
storage.load(key) must be eq (result)
}
}
"insertOrOverwrite" must {
"insert if nothing was inserted before" in {
val storage = new InMemoryStorage()
val key = "somekey"
val value = "somevalue".getBytes
val result = storage.insertOrOverwrite(key, value)
assert(result.version == InMemoryStorage.InitialVersion)
assert(result.data == value)
storage.load(key) must be eq (result)
}
"overwrite of something existed before" in {
val storage = new InMemoryStorage()
val key = "somekey"
val oldValue = "oldvalue".getBytes
val newValue = "somevalue".getBytes
val initialInsert = storage.insert(key, oldValue)
val result = storage.insertOrOverwrite(key, newValue)
assert(result.version == initialInsert.version + 1)
assert(result.data == newValue)
storage.load(key) must be eq (result)
}
}
}

View file

@ -32,31 +32,31 @@ class ReplicationSpec extends WordSpec with MustMatchers with BeforeAndAfterAll
"A Transaction Log" should {
"be able to record entries - synchronous" in {
val uuid = (new UUID).toString
val txlog = TransactionLog.newLogFor(uuid)
val txlog = TransactionLog.newLogFor(uuid, false, null, Format.Default)
val entry = "hello".getBytes("UTF-8")
txlog.recordEntry(entry)
}
"be able to record and delete entries - synchronous" in {
val uuid = (new UUID).toString
val txlog1 = TransactionLog.newLogFor(uuid)
val txlog1 = TransactionLog.newLogFor(uuid, false, null, Format.Default)
val entry = "hello".getBytes("UTF-8")
txlog1.recordEntry(entry)
txlog1.recordEntry(entry)
txlog1.delete
txlog1.close
intercept[BKNoSuchLedgerExistsException](TransactionLog.logFor(uuid))
intercept[BKNoSuchLedgerExistsException](TransactionLog.logFor(uuid, false, null, Format.Default))
}
"be able to record entries and read entries with 'entriesInRange' - synchronous" in {
val uuid = (new UUID).toString
val txlog1 = TransactionLog.newLogFor(uuid)
val txlog1 = TransactionLog.newLogFor(uuid, false, null, Format.Default)
val entry = "hello".getBytes("UTF-8")
txlog1.recordEntry(entry)
txlog1.recordEntry(entry)
txlog1.close
val txlog2 = TransactionLog.logFor(uuid)
val txlog2 = TransactionLog.logFor(uuid, false, null, Format.Default)
val entries = txlog2.entriesInRange(0, 1).map(bytes new String(bytes, "UTF-8"))
entries.size must equal(2)
entries(0) must equal("hello")
@ -66,7 +66,7 @@ class ReplicationSpec extends WordSpec with MustMatchers with BeforeAndAfterAll
"be able to record entries and read entries with 'entries' - synchronous" in {
val uuid = (new UUID).toString
val txlog1 = TransactionLog.newLogFor(uuid)
val txlog1 = TransactionLog.newLogFor(uuid, false, null, Format.Default)
val entry = "hello".getBytes("UTF-8")
txlog1.recordEntry(entry)
txlog1.recordEntry(entry)
@ -74,7 +74,7 @@ class ReplicationSpec extends WordSpec with MustMatchers with BeforeAndAfterAll
txlog1.recordEntry(entry)
txlog1.close
val txlog2 = TransactionLog.logFor(uuid)
val txlog2 = TransactionLog.logFor(uuid, false, null, Format.Default)
val entries = txlog2.entries.map(bytes new String(bytes, "UTF-8"))
entries.size must equal(4)
entries(0) must equal("hello")
@ -86,7 +86,7 @@ class ReplicationSpec extends WordSpec with MustMatchers with BeforeAndAfterAll
"be able to record a snapshot - synchronous" in {
val uuid = (new UUID).toString
val txlog1 = TransactionLog.newLogFor(uuid)
val txlog1 = TransactionLog.newLogFor(uuid, false, null, Format.Default)
val snapshot = "snapshot".getBytes("UTF-8")
txlog1.recordSnapshot(snapshot)
txlog1.close
@ -94,7 +94,7 @@ class ReplicationSpec extends WordSpec with MustMatchers with BeforeAndAfterAll
"be able to record and read a snapshot and following entries - synchronous" in {
val uuid = (new UUID).toString
val txlog1 = TransactionLog.newLogFor(uuid)
val txlog1 = TransactionLog.newLogFor(uuid, false, null, Format.Default)
val snapshot = "snapshot".getBytes("UTF-8")
txlog1.recordSnapshot(snapshot)
@ -105,8 +105,8 @@ class ReplicationSpec extends WordSpec with MustMatchers with BeforeAndAfterAll
txlog1.recordEntry(entry)
txlog1.close
val txlog2 = TransactionLog.logFor(uuid)
val (snapshotAsBytes, entriesAsBytes) = txlog2.entriesFromLatestSnapshot
val txlog2 = TransactionLog.logFor(uuid, false, null, Format.Default)
val (snapshotAsBytes, entriesAsBytes) = txlog2.toByteArraysLatestSnapshot
new String(snapshotAsBytes, "UTF-8") must equal("snapshot")
val entries = entriesAsBytes.map(bytes new String(bytes, "UTF-8"))
@ -120,7 +120,7 @@ class ReplicationSpec extends WordSpec with MustMatchers with BeforeAndAfterAll
"be able to record entries then a snapshot then more entries - and then read from the snapshot and the following entries - synchronous" in {
val uuid = (new UUID).toString
val txlog1 = TransactionLog.newLogFor(uuid)
val txlog1 = TransactionLog.newLogFor(uuid, false, null, Format.Default)
val entry = "hello".getBytes("UTF-8")
txlog1.recordEntry(entry)
@ -134,8 +134,8 @@ class ReplicationSpec extends WordSpec with MustMatchers with BeforeAndAfterAll
txlog1.recordEntry(entry)
txlog1.close
val txlog2 = TransactionLog.logFor(uuid)
val (snapshotAsBytes, entriesAsBytes) = txlog2.entriesFromLatestSnapshot
val txlog2 = TransactionLog.logFor(uuid, false, null, Format.Default)
val (snapshotAsBytes, entriesAsBytes) = txlog2.toByteArraysLatestSnapshot
new String(snapshotAsBytes, "UTF-8") must equal("snapshot")
val entries = entriesAsBytes.map(bytes new String(bytes, "UTF-8"))
@ -149,7 +149,7 @@ class ReplicationSpec extends WordSpec with MustMatchers with BeforeAndAfterAll
"A Transaction Log" should {
"be able to record entries - asynchronous" in {
val uuid = (new UUID).toString
val txlog = TransactionLog.newLogFor(uuid, true)
val txlog = TransactionLog.newLogFor(uuid, true, null, Format.Default)
val entry = "hello".getBytes("UTF-8")
txlog.recordEntry(entry)
Thread.sleep(100)
@ -158,24 +158,24 @@ class ReplicationSpec extends WordSpec with MustMatchers with BeforeAndAfterAll
"be able to record and delete entries - asynchronous" in {
val uuid = (new UUID).toString
val txlog1 = TransactionLog.newLogFor(uuid, true)
val txlog1 = TransactionLog.newLogFor(uuid, true, null, Format.Default)
val entry = "hello".getBytes("UTF-8")
txlog1.recordEntry(entry)
txlog1.recordEntry(entry)
txlog1.delete
Thread.sleep(100)
intercept[BKNoSuchLedgerExistsException](TransactionLog.logFor(uuid, true))
intercept[BKNoSuchLedgerExistsException](TransactionLog.logFor(uuid, true, null, Format.Default))
}
"be able to record entries and read entries with 'entriesInRange' - asynchronous" in {
val uuid = (new UUID).toString
val txlog1 = TransactionLog.newLogFor(uuid, true)
val txlog1 = TransactionLog.newLogFor(uuid, true, null, Format.Default)
val entry = "hello".getBytes("UTF-8")
txlog1.recordEntry(entry)
txlog1.recordEntry(entry)
Thread.sleep(100)
txlog1.close
val txlog2 = TransactionLog.logFor(uuid, true)
val txlog2 = TransactionLog.logFor(uuid, true, null, Format.Default)
val entries = txlog2.entriesInRange(0, 1).map(bytes new String(bytes, "UTF-8"))
entries.size must equal(2)
entries(0) must equal("hello")
@ -186,7 +186,7 @@ class ReplicationSpec extends WordSpec with MustMatchers with BeforeAndAfterAll
"be able to record entries and read entries with 'entries' - asynchronous" in {
val uuid = (new UUID).toString
val txlog1 = TransactionLog.newLogFor(uuid, true)
val txlog1 = TransactionLog.newLogFor(uuid, true, null, Format.Default)
val entry = "hello".getBytes("UTF-8")
txlog1.recordEntry(entry)
txlog1.recordEntry(entry)
@ -195,7 +195,7 @@ class ReplicationSpec extends WordSpec with MustMatchers with BeforeAndAfterAll
Thread.sleep(100)
txlog1.close
val txlog2 = TransactionLog.logFor(uuid, true)
val txlog2 = TransactionLog.logFor(uuid, true, null, Format.Default)
val entries = txlog2.entries.map(bytes new String(bytes, "UTF-8"))
entries.size must equal(4)
entries(0) must equal("hello")
@ -208,7 +208,7 @@ class ReplicationSpec extends WordSpec with MustMatchers with BeforeAndAfterAll
"be able to record a snapshot - asynchronous" in {
val uuid = (new UUID).toString
val txlog1 = TransactionLog.newLogFor(uuid, true)
val txlog1 = TransactionLog.newLogFor(uuid, true, null, Format.Default)
val snapshot = "snapshot".getBytes("UTF-8")
txlog1.recordSnapshot(snapshot)
Thread.sleep(100)
@ -217,7 +217,7 @@ class ReplicationSpec extends WordSpec with MustMatchers with BeforeAndAfterAll
"be able to record and read a snapshot and following entries - asynchronous" in {
val uuid = (new UUID).toString
val txlog1 = TransactionLog.newLogFor(uuid, true)
val txlog1 = TransactionLog.newLogFor(uuid, true, null, Format.Default)
val snapshot = "snapshot".getBytes("UTF-8")
txlog1.recordSnapshot(snapshot)
@ -229,8 +229,8 @@ class ReplicationSpec extends WordSpec with MustMatchers with BeforeAndAfterAll
Thread.sleep(100)
txlog1.close
val txlog2 = TransactionLog.logFor(uuid, true)
val (snapshotAsBytes, entriesAsBytes) = txlog2.entriesFromLatestSnapshot
val txlog2 = TransactionLog.logFor(uuid, true, null, Format.Default)
val (snapshotAsBytes, entriesAsBytes) = txlog2.toByteArraysLatestSnapshot
new String(snapshotAsBytes, "UTF-8") must equal("snapshot")
val entries = entriesAsBytes.map(bytes new String(bytes, "UTF-8"))
@ -245,7 +245,7 @@ class ReplicationSpec extends WordSpec with MustMatchers with BeforeAndAfterAll
"be able to record entries then a snapshot then more entries - and then read from the snapshot and the following entries - asynchronous" in {
val uuid = (new UUID).toString
val txlog1 = TransactionLog.newLogFor(uuid, true)
val txlog1 = TransactionLog.newLogFor(uuid, true, null, Format.Default)
val entry = "hello".getBytes("UTF-8")
txlog1.recordEntry(entry)
@ -258,8 +258,8 @@ class ReplicationSpec extends WordSpec with MustMatchers with BeforeAndAfterAll
Thread.sleep(100)
txlog1.close
val txlog2 = TransactionLog.logFor(uuid, true)
val (snapshotAsBytes, entriesAsBytes) = txlog2.entriesFromLatestSnapshot
val txlog2 = TransactionLog.logFor(uuid, true, null, Format.Default)
val (snapshotAsBytes, entriesAsBytes) = txlog2.toByteArraysLatestSnapshot
new String(snapshotAsBytes, "UTF-8") must equal("snapshot")
val entries = entriesAsBytes.map(bytes new String(bytes, "UTF-8"))
entries.size must equal(2)

View file

@ -1,165 +0,0 @@
package akka.cluster
import org.scalatest.matchers.MustMatchers
import org.scalatest.WordSpec
class InMemoryStorageSpec extends WordSpec with MustMatchers {
"unversioned load" must {
"throw MissingNodeException if non existing key" in {
val store = new InMemoryStorage()
try {
store.load("foo")
fail()
} catch {
case e: MissingNodeException
}
}
"return VersionedData if key existing" in {
val storage = new InMemoryStorage()
val key = "somekey"
val value = "somevalue".getBytes
storage.insert(key, value)
val result = storage.load(key)
//todo: strange that the implicit store is not found
assertContent(key, value, result.version)(storage)
}
}
"exist" must {
"return true if value exists" in {
val store = new InMemoryStorage()
val key = "somekey"
store.insert(key, "somevalue".getBytes)
store.exists(key) must be(true)
}
"return false if value not exists" in {
val store = new InMemoryStorage()
store.exists("somekey") must be(false)
}
}
"versioned load" must {
"throw MissingNodeException if non existing key" in {
val store = new InMemoryStorage()
try {
store.load("foo", 1)
fail()
} catch {
case e: MissingNodeException
}
}
"return VersionedData if key existing and exact version match" in {
val storage = new InMemoryStorage()
val key = "somekey"
val value = "somevalue".getBytes
val stored = storage.insert(key, value)
val result = storage.load(key, stored.version)
assert(result.version == stored.version)
assert(result.data == stored.data)
}
"throw VersioningMismatchStorageException is version too new" in {
val storage = new InMemoryStorage()
val key = "somekey"
val value = "somevalue".getBytes
val stored = storage.insert(key, value)
try {
storage.load(key, stored.version + 1)
fail()
} catch {
case e: VersioningMismatchStorageException
}
}
"throw VersioningMismatchStorageException is version too old" in {
val storage = new InMemoryStorage()
val key = "somekey"
val value = "somevalue".getBytes
val stored = storage.insert(key, value)
try {
storage.load(key, stored.version - 1)
fail()
} catch {
case e: VersioningMismatchStorageException
}
}
}
"insert" must {
"place a new value when non previously existed" in {
val storage = new InMemoryStorage()
val key = "somekey"
val oldValue = "oldvalue".getBytes
storage.insert(key, oldValue)
val result = storage.load(key)
assertContent(key, oldValue)(storage)
assert(InMemoryStorage.InitialVersion == result.version)
}
"throw MissingNodeException when there already exists an entry with the same key" in {
val storage = new InMemoryStorage()
val key = "somekey"
val oldValue = "oldvalue".getBytes
val oldVersionedData = storage.insert(key, oldValue)
val newValue = "newValue".getBytes
try {
storage.insert(key, newValue)
fail()
} catch {
case e: NodeExistsException
}
//make sure that the old value was not changed
assert(oldVersionedData == storage.load(key))
}
}
"update with versioning" must {
"throw NoNodeException when no node exists" in {
val storage = new InMemoryStorage()
val key = "somekey"
try {
storage.update(key, new VersionedData("somevalue".getBytes, 1))
fail()
} catch {
case e: MissingNodeException
}
}
"throw OptimisticLockException when ..." in {
}
"replace" in {
}
}
def assertContent(key: String, expectedData: Array[Byte], expectedVersion: Long)(implicit storage: InMemoryStorage) {
val found = storage.load(key)
assert(found.version == expectedVersion)
assert(expectedData == found.data) //todo: structural equals
}
def assertContent(key: String, expectedData: Array[Byte])(implicit storage: InMemoryStorage) {
val found = storage.load(key)
assert(expectedData == found.data) //todo: structural equals
}
}

View file

@ -0,0 +1,15 @@
package akka.cluster
object StorageTestUtils {
def assertContent(key: String, expectedData: Array[Byte], expectedVersion: Long)(implicit storage: InMemoryStorage) {
val found = storage.load(key)
assert(found.version == expectedVersion)
assert(expectedData == found.data) //todo: structural equals
}
def assertContent(key: String, expectedData: Array[Byte])(implicit storage: InMemoryStorage) {
val found = storage.load(key)
assert(expectedData == found.data) //todo: structural equals
}
}

View file

@ -0,0 +1,58 @@
package akka.cluster
import org.scalatest.matchers.MustMatchers
import akka.actor.Actor
import org.scalatest.{ BeforeAndAfterEach, BeforeAndAfterAll, WordSpec }
import org.I0Itec.zkclient.ZkServer
import zookeeper.AkkaZkClient
class ZooKeeperStorageSpec extends WordSpec with MustMatchers with BeforeAndAfterAll with BeforeAndAfterEach {
val dataPath = "_akka_cluster/data"
val logPath = "_akka_cluster/log"
var zkServer: ZkServer = _
var zkClient: AkkaZkClient = _
override def beforeAll() {
try {
zkServer = Cluster.startLocalCluster(dataPath, logPath)
Thread.sleep(5000)
Actor.cluster.start()
zkClient = Cluster.newZkClient()
} catch {
case e e.printStackTrace()
}
}
override def afterAll() {
zkClient.close()
Actor.cluster.shutdown()
ClusterDeployer.shutdown()
Cluster.shutdownLocalCluster()
Actor.registry.local.shutdownAll()
}
"unversioned load" must {
"throw MissingDataException if non existing key" in {
val store = new ZooKeeperStorage(zkClient)
//try {
// store.load("foo")
// fail()
//} catch {
// case e: MissingDataException
//}
}
/*
"return VersionedData if key existing" in {
val storage = new InMemoryStorage()
val key = "somekey"
val value = "somevalue".getBytes
storage.insert(key, value)
val result = storage.load(key)
//todo: strange that the implicit store is not found
assertContent(key, value, result.version)(storage)
} */
}
}

View file

@ -8,9 +8,12 @@ import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers
import org.scalatest.BeforeAndAfterAll
import org.apache.bookkeeper.client.{ BookKeeper, BKException }
import BKException._
import akka.cluster._
import akka.actor._
import Actor._
import akka.actor.Actor._
import akka.config.Config
object RoundRobin1ReplicaMultiJvmSpec {
@ -27,6 +30,9 @@ object RoundRobin1ReplicaMultiJvmSpec {
class RoundRobin1ReplicaMultiJvmNode1 extends WordSpec with MustMatchers with BeforeAndAfterAll {
import RoundRobin1ReplicaMultiJvmSpec._
private var bookKeeper: BookKeeper = _
private var localBookKeeper: LocalBookKeeper = _
"A cluster" must {
"create clustered actor, get a 'local' actor on 'home' node and a 'ref' to actor on remote node" in {
@ -49,10 +55,13 @@ class RoundRobin1ReplicaMultiJvmNode1 extends WordSpec with MustMatchers with Be
override def beforeAll() = {
Cluster.startLocalCluster()
LocalBookKeeperEnsemble.start()
}
override def afterAll() = {
Cluster.shutdownLocalCluster()
TransactionLog.shutdown()
LocalBookKeeperEnsemble.shutdown()
}
}
@ -81,7 +90,7 @@ class RoundRobin1ReplicaMultiJvmNode2 extends WordSpec with MustMatchers {
Cluster.barrier("send-message-from-node2-to-node1", NrOfNodes) {
hello must not equal (null)
val reply = (hello !! "Hello").as[String].getOrElse(fail("Should have recieved reply from node1"))
val reply = (hello ? "Hello").as[String].getOrElse(fail("Should have recieved reply from node1"))
reply must equal("World from node [node1]")
}

View file

@ -8,9 +8,12 @@ import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers
import org.scalatest.BeforeAndAfterAll
import org.apache.bookkeeper.client.{ BookKeeper, BKException }
import BKException._
import akka.cluster._
import akka.actor._
import Actor._
import akka.actor.Actor._
import akka.config.Config
object RoundRobin2ReplicasMultiJvmSpec {
@ -28,6 +31,9 @@ object RoundRobin2ReplicasMultiJvmSpec {
class RoundRobin2ReplicasMultiJvmNode1 extends WordSpec with MustMatchers with BeforeAndAfterAll {
import RoundRobin2ReplicasMultiJvmSpec._
private var bookKeeper: BookKeeper = _
private var localBookKeeper: LocalBookKeeper = _
"A cluster" must {
"create clustered actor, get a 'local' actor on 'home' node and a 'ref' to actor on remote node" in {
@ -52,10 +58,13 @@ class RoundRobin2ReplicasMultiJvmNode1 extends WordSpec with MustMatchers with B
override def beforeAll() = {
Cluster.startLocalCluster()
LocalBookKeeperEnsemble.start()
}
override def afterAll() = {
Cluster.shutdownLocalCluster()
TransactionLog.shutdown()
LocalBookKeeperEnsemble.shutdown()
}
}
@ -93,14 +102,14 @@ class RoundRobin2ReplicasMultiJvmNode2 extends WordSpec with MustMatchers {
else replies.put(reply, replies(reply) + 1)
}
count((hello !! "Hello").as[String].getOrElse(fail("Should have recieved reply from node1")))
count((hello !! "Hello").as[String].getOrElse(fail("Should have recieved reply from node3")))
count((hello !! "Hello").as[String].getOrElse(fail("Should have recieved reply from node1")))
count((hello !! "Hello").as[String].getOrElse(fail("Should have recieved reply from node3")))
count((hello !! "Hello").as[String].getOrElse(fail("Should have recieved reply from node1")))
count((hello !! "Hello").as[String].getOrElse(fail("Should have recieved reply from node3")))
count((hello !! "Hello").as[String].getOrElse(fail("Should have recieved reply from node1")))
count((hello !! "Hello").as[String].getOrElse(fail("Should have recieved reply from node3")))
count((hello ? "Hello").as[String].getOrElse(fail("Should have recieved reply from node1")))
count((hello ? "Hello").as[String].getOrElse(fail("Should have recieved reply from node3")))
count((hello ? "Hello").as[String].getOrElse(fail("Should have recieved reply from node1")))
count((hello ? "Hello").as[String].getOrElse(fail("Should have recieved reply from node3")))
count((hello ? "Hello").as[String].getOrElse(fail("Should have recieved reply from node1")))
count((hello ? "Hello").as[String].getOrElse(fail("Should have recieved reply from node3")))
count((hello ? "Hello").as[String].getOrElse(fail("Should have recieved reply from node1")))
count((hello ? "Hello").as[String].getOrElse(fail("Should have recieved reply from node3")))
replies("World from node [node1]") must equal(4)
replies("World from node [node3]") must equal(4)

View file

@ -18,8 +18,8 @@ in its mailbox.
.. sidebar:: **IMPORTANT**
None of these mailboxes work with blocking message send, e.g. the message
send operations that are relying on futures; ``!!``, ``!!!``,
``sendRequestReply`` and ``sendRequestReplyFuture``. If the node has crashed
send operations that are relying on futures; ``!!``, ``?``,
``sendRequestReply`` and ``ask``. If the node has crashed
and then restarted, the thread that was blocked waiting for the reply is gone
and there is no way we can deliver the message.

View file

@ -67,7 +67,7 @@ Here is the master actor:
A couple of things are worth explaining further.
First, we are passing in a ``java.util.concurrent.CountDownLatch`` to the ``Master`` actor. This latch is only used for plumbing (in this specific tutorial), to have a simple way of letting the outside world knowing when the master can deliver the result and shut down. In more idiomatic Akka code, as we will see in part two of this tutorial series, we would not use a latch but other abstractions and functions like ``Channel``, ``Future`` and ``!!!`` to achive the same thing in a non-blocking way. But for simplicity let's stick to a ``CountDownLatch`` for now.
First, we are passing in a ``java.util.concurrent.CountDownLatch`` to the ``Master`` actor. This latch is only used for plumbing (in this specific tutorial), to have a simple way of letting the outside world knowing when the master can deliver the result and shut down. In more idiomatic Akka code, as we will see in part two of this tutorial series, we would not use a latch but other abstractions and functions like ``Channel``, ``Future`` and ``?`` to achive the same thing in a non-blocking way. But for simplicity let's stick to a ``CountDownLatch`` for now.
Second, we are adding a couple of life-cycle callback methods; ``preStart`` and ``postStop``. In the ``preStart`` callback we are recording the time when the actor is started and in the ``postStop`` callback we are printing out the result (the approximation of Pi) and the time it took to calculate it. In this call we also invoke ``latch.countDown`` to tell the outside world that we are done.

View file

@ -435,7 +435,7 @@ Here is the master actor::
A couple of things are worth explaining further.
First, we are passing in a ``java.util.concurrent.CountDownLatch`` to the ``Master`` actor. This latch is only used for plumbing (in this specific tutorial), to have a simple way of letting the outside world knowing when the master can deliver the result and shut down. In more idiomatic Akka code, as we will see in part two of this tutorial series, we would not use a latch but other abstractions and functions like ``Channel``, ``Future`` and ``!!!`` to achieve the same thing in a non-blocking way. But for simplicity let's stick to a ``CountDownLatch`` for now.
First, we are passing in a ``java.util.concurrent.CountDownLatch`` to the ``Master`` actor. This latch is only used for plumbing (in this specific tutorial), to have a simple way of letting the outside world knowing when the master can deliver the result and shut down. In more idiomatic Akka code, as we will see in part two of this tutorial series, we would not use a latch but other abstractions and functions like ``Channel``, ``Future`` and ``?`` to achieve the same thing in a non-blocking way. But for simplicity let's stick to a ``CountDownLatch`` for now.
Second, we are adding a couple of life-cycle callback methods; ``preStart`` and ``postStop``. In the ``preStart`` callback we are recording the time when the actor is started and in the ``postStop`` callback we are printing out the result (the approximation of Pi) and the time it took to calculate it. In this call we also invoke ``latch.countDown()`` to tell the outside world that we are done.

View file

@ -335,7 +335,7 @@ Here is the master actor::
A couple of things are worth explaining further.
First, we are passing in a ``java.util.concurrent.CountDownLatch`` to the ``Master`` actor. This latch is only used for plumbing (in this specific tutorial), to have a simple way of letting the outside world knowing when the master can deliver the result and shut down. In more idiomatic Akka code, as we will see in part two of this tutorial series, we would not use a latch but other abstractions and functions like ``Channel``, ``Future`` and ``!!!`` to achieve the same thing in a non-blocking way. But for simplicity let's stick to a ``CountDownLatch`` for now.
First, we are passing in a ``java.util.concurrent.CountDownLatch`` to the ``Master`` actor. This latch is only used for plumbing (in this specific tutorial), to have a simple way of letting the outside world knowing when the master can deliver the result and shut down. In more idiomatic Akka code, as we will see in part two of this tutorial series, we would not use a latch but other abstractions and functions like ``Channel``, ``Future`` and ``?`` to achieve the same thing in a non-blocking way. But for simplicity let's stick to a ``CountDownLatch`` for now.
Second, we are adding a couple of life-cycle callback methods; ``preStart`` and ``postStop``. In the ``preStart`` callback we are recording the time when the actor is started and in the ``postStop`` callback we are printing out the result (the approximation of Pi) and the time it took to calculate it. In this call we also invoke ``latch.countDown`` to tell the outside world that we are done.

View file

@ -329,7 +329,7 @@ Here is the master actor::
A couple of things are worth explaining further.
First, we are passing in a ``java.util.concurrent.CountDownLatch`` to the ``Master`` actor. This latch is only used for plumbing (in this specific tutorial), to have a simple way of letting the outside world knowing when the master can deliver the result and shut down. In more idiomatic Akka code, as we will see in part two of this tutorial series, we would not use a latch but other abstractions and functions like ``Channel``, ``Future`` and ``!!!`` to achieve the same thing in a non-blocking way. But for simplicity let's stick to a ``CountDownLatch`` for now.
First, we are passing in a ``java.util.concurrent.CountDownLatch`` to the ``Master`` actor. This latch is only used for plumbing (in this specific tutorial), to have a simple way of letting the outside world knowing when the master can deliver the result and shut down. In more idiomatic Akka code, as we will see in part two of this tutorial series, we would not use a latch but other abstractions and functions like ``Channel``, ``Future`` and ``?`` to achieve the same thing in a non-blocking way. But for simplicity let's stick to a ``CountDownLatch`` for now.
Second, we are adding a couple of life-cycle callback methods; ``preStart`` and ``postStop``. In the ``preStart`` callback we are recording the time when the actor is started and in the ``postStop`` callback we are printing out the result (the approximation of Pi) and the time it took to calculate it. In this call we also invoke ``latch.countDown`` to tell the outside world that we are done.

View file

@ -13,7 +13,7 @@ The Dispatcher is an important piece that allows you to configure the right sema
Akka supports dispatchers for both event-driven lightweight threads, allowing creation of millions threads on a single workstation, and thread-based Actors, where each dispatcher is bound to a dedicated OS thread.
The event-based Actors currently consume ~600 bytes per Actor which means that you can create more than 6.5 million Actors on 4 G RAM.
The event-based Actors currently consume ~600 bytes per Actor which means that you can create more than 6.5 million Actors on 4 GB RAM.
Default dispatcher
------------------
@ -84,6 +84,7 @@ The 'Dispatcher' binds a set of Actors to a thread pool backed up by a 'Blocking
The event-driven dispatchers **must be shared** between multiple Typed Actors and/or Actors. One best practice is to let each top-level Actor, e.g. the Actors you define in the declarative supervisor config, to get their own dispatcher but reuse the dispatcher for each new Actor that the top-level Actor creates. But you can also share dispatcher between multiple top-level Actors. This is very use-case specific and needs to be tried out on a case by case basis. The important thing is that Akka tries to provide you with the freedom you need to design and implement your system in the most efficient way in regards to performance, throughput and latency.
It comes with many different predefined BlockingQueue configurations:
* Bounded LinkedBlockingQueue
* Unbounded LinkedBlockingQueue
* Bounded ArrayBlockingQueue

View file

@ -17,11 +17,11 @@ Use with Actors
There are generally two ways of getting a reply from an ``UntypedActor``: the first is by a sent message (``actorRef.sendOneWay(msg);``), which only works if the original sender was an ``UntypedActor``) and the second is through a ``Future``.
Using the ``ActorRef``\'s ``sendRequestReplyFuture`` method to send a message will return a Future. To wait for and retrieve the actual result the simplest method is:
Using the ``ActorRef``\'s ``ask`` method to send a message will return a Future. To wait for and retrieve the actual result the simplest method is:
.. code-block:: java
Future[Object] future = actorRef.sendRequestReplyFuture[Object](msg);
Future[Object] future = actorRef.ask[Object](msg);
Object result = future.get(); //Block until result is available, usually bad practice
This will cause the current thread to block and wait for the ``UntypedActor`` to 'complete' the ``Future`` with it's reply. Due to the dynamic nature of Akka's ``UntypedActor``\s this result can be anything. The safest way to deal with this is to specify the result to an ``Object`` as is shown in the above example. You can also use the expected result type instead of ``Any``, but if an unexpected type were to be returned you will get a ``ClassCastException``. For more elegant ways to deal with this and to use the result without blocking, refer to `Functional Futures`_.

View file

@ -115,6 +115,7 @@ Methods that return something (e.g. non-void methods) are turned into send-an
User user = service.getUser(username);
Generally it is preferred to use fire-forget messages as much as possible since they will never block, e.g. consume a resource by waiting. But sometimes they are neat to use since they:
# Simulates standard Java method dispatch, which is more intuitive for most Java developers
# Are a neat to model request-reply
# Are useful when you need to do things in a defined order

View file

@ -109,7 +109,7 @@ Send messages
Messages are sent to an Actor through one of the 'send' methods.
* 'sendOneWay' means “fire-and-forget”, e.g. send a message asynchronously and return immediately.
* 'sendRequestReply' means “send-and-reply-eventually”, e.g. send a message asynchronously and wait for a reply through a Future. Here you can specify a timeout. Using timeouts is very important. If no timeout is specified then the actors default timeout (set by the 'getContext().setTimeout(..)' method in the 'ActorRef') is used. This method throws an 'ActorTimeoutException' if the call timed out.
* 'sendRequestReplyFuture' sends a message asynchronously and returns a 'Future'.
* 'ask' sends a message asynchronously and returns a 'Future'.
In all these methods you have the option of passing along your 'ActorRef' context variable. Make it a practice of doing so because it will allow the receiver actors to be able to respond to your message, since the sender reference is sent along with the message.
@ -158,11 +158,11 @@ Here are some examples:
Send-And-Receive-Future
^^^^^^^^^^^^^^^^^^^^^^^
Using 'sendRequestReplyFuture' will send a message to the receiving Actor asynchronously and will immediately return a 'Future'.
Using 'ask' will send a message to the receiving Actor asynchronously and will immediately return a 'Future'.
.. code-block:: java
Future future = actorRef.sendRequestReplyFuture("Hello", getContext(), 1000);
Future future = actorRef.ask("Hello", getContext(), 1000);
The 'Future' interface looks like this:
@ -182,7 +182,7 @@ So the normal way of working with futures is something like this:
.. code-block:: java
Future future = actorRef.sendRequestReplyFuture("Hello", getContext(), 1000);
Future future = actorRef.ask("Hello", getContext(), 1000);
future.await();
if (future.isCompleted()) {
Option resultOption = future.result();
@ -305,7 +305,7 @@ On this 'Option' you can invoke 'boolean isDefined()' or 'boolean isEmpty()' to
Reply using the sender future
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
If a message was sent with the 'sendRequestReply' or 'sendRequestReplyFuture' methods, which both implements request-reply semantics using Future's, then you either have the option of replying using the 'reply' method as above. This method will then resolve the Future. But you can also get a reference to the Future directly and resolve it yourself or if you would like to store it away to resolve it later, or pass it on to some other Actor to resolve it.
If a message was sent with the 'sendRequestReply' or 'ask' methods, which both implements request-reply semantics using Future's, then you either have the option of replying using the 'reply' method as above. This method will then resolve the Future. But you can also get a reference to the Future directly and resolve it yourself or if you would like to store it away to resolve it later, or pass it on to some other Actor to resolve it.
The reference to the Future resides in the 'ActorRef' instance and can be retrieved using 'Option<Promise> getSenderFuture()'.

View file

@ -323,7 +323,7 @@ The Akka configuration can be made available as property placeholders by using a
<akka:property-placeholder location="akka.conf"/>
<akka:untyped-actor id="actor-1" implementation="com.biz.MyActor" timeout="${akka.actor.timeout}">
<akka:remote host="${akka.remote.server.hostname}" port="${akka.remote.server.port}"/>
<akka:remote host="${akka.cluster.server.hostname}" port="${akka.cluster.server.port}"/>
</akka:untyped-actor>
Camel configuration

View file

@ -524,7 +524,7 @@ Release 0.6 - January 5th 2010
- **ADD** - New and much improved docs (Jonas Bonér)
- **ADD** - Enhanced trapping of failures: 'trapExit = List(classOf[..], classOf[..])' (Jonas Bonér)
- **ADD** - Upgraded to Netty 3.2, Protobuf 2.2, ScalaTest 1.0, Jersey 1.1.3, Atmosphere 0.4.1, Cassandra 0.4.1, Configgy 1.4 (Jonas Bonér)
- **FIX** - Lowered actor memory footprint; now an actor consumes ~600 bytes, which mean that you can create 6.5 million on 4 G RAM (Jonas Bonér)
- **FIX** - Lowered actor memory footprint; now an actor consumes ~600 bytes, which mean that you can create 6.5 million on 4 GB RAM (Jonas Bonér)
- **FIX** - Remote actors are now defined by their UUID (not class name) (Jonas Bonér)
- **FIX** - Fixed dispatcher bugs (Jonas Bonér)
- **FIX** - Cleaned up Maven scripts and distribution in general (Jonas Bonér)

View file

@ -126,7 +126,7 @@ Messages are sent to an Actor through one of the “bang” methods.
* ! means “fire-and-forget”, e.g. send a message asynchronously and return immediately.
* !! means “send-and-reply-eventually”, e.g. send a message asynchronously and wait for a reply through aFuture. Here you can specify a timeout. Using timeouts is very important. If no timeout is specified then the actors default timeout (set by the this.timeout variable in the actor) is used. This method returns an ``Option[Any]`` which will be either ``Some(result)`` if returning successfully or None if the call timed out.
* !!! sends a message asynchronously and returns a ``Future``.
* ? sends a message asynchronously and returns a ``Future``.
You can check if an Actor can handle a specific message by invoking the ``isDefinedAt`` method:
@ -180,11 +180,11 @@ Here are some examples:
Send-And-Receive-Future
^^^^^^^^^^^^^^^^^^^^^^^
Using ``!!!`` will send a message to the receiving Actor asynchronously and will return a 'Future':
Using ``?`` will send a message to the receiving Actor asynchronously and will return a 'Future':
.. code-block:: scala
val future = actor !!! "Hello"
val future = actor ? "Hello"
See :ref:`futures-scala` for more information.
@ -329,7 +329,7 @@ The same pattern holds for using the ``senderFuture`` in the section below.
Reply using the sender future
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
If a message was sent with the ``!!`` or ``!!!`` methods, which both implements request-reply semantics using Future's, then you either have the option of replying using the ``reply`` method as above. This method will then resolve the Future. But you can also get a reference to the Future directly and resolve it yourself or if you would like to store it away to resolve it later, or pass it on to some other Actor to resolve it.
If a message was sent with the ``!!`` or ``?`` methods, which both implements request-reply semantics using Future's, then you either have the option of replying using the ``reply`` method as above. This method will then resolve the Future. But you can also get a reference to the Future directly and resolve it yourself or if you would like to store it away to resolve it later, or pass it on to some other Actor to resolve it.
The reference to the Future resides in the ``senderFuture: Option[Promise[_]]`` member field in the ``ActorRef`` class.
@ -427,7 +427,7 @@ PoisonPill
You can also send an actor the ``akka.actor.PoisonPill`` message, which will stop the actor when the message is processed.
If the sender is a ``Future`` (e.g. the message is sent with ``!!`` or ``!!!``), the ``Future`` will be completed with an ``akka.actor.ActorKilledException("PoisonPill")``.
If the sender is a ``Future`` (e.g. the message is sent with ``!!`` or ``?``), the ``Future`` will be completed with an ``akka.actor.ActorKilledException("PoisonPill")``.
HotSwap
-------
@ -457,7 +457,7 @@ To hotswap the Actor using ``become``:
.. code-block:: scala
def angry: Receive = {
case "foo" => self reply "I am already angry!!!"
case "foo" => self reply "I am already angry?"
case "bar" => become(happy)
}

View file

@ -13,7 +13,7 @@ The Dispatcher is an important piece that allows you to configure the right sema
Akka supports dispatchers for both event-driven lightweight threads, allowing creation of millions of threads on a single workstation, and thread-based Actors, where each dispatcher is bound to a dedicated OS thread.
The event-based Actors currently consume ~600 bytes per Actor which means that you can create more than 6.5 million Actors on 4 G RAM.
The event-based Actors currently consume ~600 bytes per Actor which means that you can create more than 6.5 million Actors on 4 GB RAM.
Default dispatcher
------------------

View file

@ -17,11 +17,11 @@ Use with Actors
There are generally two ways of getting a reply from an ``Actor``: the first is by a sent message (``actor ! msg``), which only works if the original sender was an ``Actor``) and the second is through a ``Future``.
Using an ``Actor``\'s ``!!!`` method to send a message will return a Future. To wait for and retrieve the actual result the simplest method is:
Using an ``Actor``\'s ``?`` method to send a message will return a Future. To wait for and retrieve the actual result the simplest method is:
.. code-block:: scala
val future = actor !!! msg
val future = actor ? msg
val result: Any = future.get()
This will cause the current thread to block and wait for the ``Actor`` to 'complete' the ``Future`` with it's reply. Due to the dynamic nature of Akka's ``Actor``\s this result will be untyped and will default to ``Nothing``. The safest way to deal with this is to cast the result to an ``Any`` as is shown in the above example. You can also use the expected result type instead of ``Any``, but if an unexpected type were to be returned you will get a ``ClassCastException``. For more elegant ways to deal with this and to use the result without blocking, refer to `Functional Futures`_.
@ -141,13 +141,13 @@ The example for comprehension above is an example of composing ``Future``\s. A c
.. code-block:: scala
val f1 = actor1 !!! msg1
val f2 = actor2 !!! msg2
val f1 = actor1 ? msg1
val f2 = actor2 ? msg2
val a: Int = f1.get()
val b: Int = f2.get()
val f3 = actor3 !!! (a + b)
val f3 = actor3 ? (a + b)
val result: String = f3.get()
@ -155,13 +155,13 @@ Here we wait for the results from the first 2 ``Actor``\s before sending that re
.. code-block:: scala
val f1 = actor1 !!! msg1
val f2 = actor2 !!! msg2
val f1 = actor1 ? msg1
val f2 = actor2 ? msg2
val f3 = for {
a: Int <- f1
b: Int <- f2
c: String <- actor3 !!! (a + b)
c: String <- actor3 ? (a + b)
} yield c
val result = f3.get()
@ -173,7 +173,7 @@ This is fine when dealing with a known amount of Actors, but can grow unwieldy i
.. code-block:: scala
// oddActor returns odd numbers sequentially from 1
val listOfFutures: List[Future[Int]] = List.fill(100)(oddActor !!! GetNext)
val listOfFutures: List[Future[Int]] = List.fill(100)(oddActor ? GetNext)
// now we have a Future[List[Int]]
val futureList = Future.sequence(listOfFutures)
@ -242,7 +242,7 @@ It is also possible to handle an ``Exception`` by returning a different result.
.. code-block:: scala
val future = actor !!! msg1 recover {
val future = actor ? msg1 recover {
case e: ArithmeticException => 0
}

View file

@ -72,7 +72,7 @@ The 'actorOf' factory method can be imported like this:
From now on we will assume that it is imported like this and can use it directly.
Akka Actors are extremely lightweight. Each Actor consume ~600 bytes, which means that you can create 6.5 million on 4 G RAM.
Akka Actors are extremely lightweight. Each Actor consume ~600 bytes, which means that you can create 6.5 million on 4 GB RAM.
Messages are sent using the '!' operator:

View file

@ -95,7 +95,7 @@ case class DurableDispatcher(
private[akka] override def dispatch(invocation: MessageInvocation): Unit = {
if (invocation.channel.isInstanceOf[ActorPromise])
throw new IllegalArgumentException("Durable mailboxes do not support Future-based messages from !! and !!!")
throw new IllegalArgumentException("Durable mailboxes do not support Future-based messages from ?")
super.dispatch(invocation)
}
@ -132,7 +132,7 @@ case class DurablePinnedDispatcher(
private[akka] override def dispatch(invocation: MessageInvocation): Unit = {
if (invocation.channel.isInstanceOf[ActorPromise])
throw new IllegalArgumentException("Actor has a durable mailbox that does not support !! or !!!")
throw new IllegalArgumentException("Actor has a durable mailbox that does not support ?")
super.dispatch(invocation)
}
}

View file

@ -46,6 +46,23 @@ enum CommandType {
SHUTDOWN = 2;
}
/**
* Defines the type of the ReplicationStorage
*/
enum ReplicationStorageType {
TRANSIENT = 1;
TRANSACTION_LOG = 2;
DATA_GRID = 3;
}
/**
* Defines the type of the ReplicationStrategy
*/
enum ReplicationStrategyType {
WRITE_THROUGH = 1;
WRITE_BEHIND = 2;
}
/**
* Defines a remote ActorRef that "remembers" and uses its original Actor instance
* on the original node.
@ -72,7 +89,9 @@ message SerializedActorRefProtocol {
optional LifeCycleProtocol lifeCycle = 8;
optional RemoteActorRefProtocol supervisor = 9;
optional bytes hotswapStack = 10;
repeated RemoteMessageProtocol messages = 11;
optional ReplicationStorageType replicationStorage = 11;
optional ReplicationStrategyType replicationStrategy = 12;
repeated RemoteMessageProtocol messages = 13;
}
/**

View file

@ -9,59 +9,58 @@ import akka.config.Config._
import akka.config.ConfigurationException
object RemoteClientSettings {
val SECURE_COOKIE: Option[String] = config.getString("akka.remote.secure-cookie", "") match {
val SECURE_COOKIE: Option[String] = config.getString("akka.cluster.secure-cookie", "") match {
case "" None
case cookie Some(cookie)
}
val RECONNECTION_TIME_WINDOW = Duration(config.getInt("akka.remote.client.reconnection-time-window", 600), TIME_UNIT).toMillis
val READ_TIMEOUT = Duration(config.getInt("akka.remote.client.read-timeout", 10), TIME_UNIT)
val RECONNECT_DELAY = Duration(config.getInt("akka.remote.client.reconnect-delay", 5), TIME_UNIT)
val REAP_FUTURES_DELAY = Duration(config.getInt("akka.remote.client.reap-futures-delay", 5), TIME_UNIT)
val MESSAGE_FRAME_SIZE = config.getInt("akka.remote.client.message-frame-size", 1048576)
val RECONNECTION_TIME_WINDOW = Duration(config.getInt("akka.cluster.client.reconnection-time-window", 600), TIME_UNIT).toMillis
val READ_TIMEOUT = Duration(config.getInt("akka.cluster.client.read-timeout", 10), TIME_UNIT)
val RECONNECT_DELAY = Duration(config.getInt("akka.cluster.client.reconnect-delay", 5), TIME_UNIT)
val REAP_FUTURES_DELAY = Duration(config.getInt("akka.cluster.client.reap-futures-delay", 5), TIME_UNIT)
val MESSAGE_FRAME_SIZE = config.getInt("akka.cluster.client.message-frame-size", 1048576)
}
object RemoteServerSettings {
val isRemotingEnabled = config.getList("akka.enabled-modules").exists(_ == "remote")
val MESSAGE_FRAME_SIZE = config.getInt("akka.remote.server.message-frame-size", 1048576)
val SECURE_COOKIE = config.getString("akka.remote.secure-cookie")
val isRemotingEnabled = config.getList("akka.enabled-modules").exists(_ == "cluster")
val MESSAGE_FRAME_SIZE = config.getInt("akka.cluster.server.message-frame-size", 1048576)
val SECURE_COOKIE = config.getString("akka.cluster.secure-cookie")
val REQUIRE_COOKIE = {
val requireCookie = config.getBool("akka.remote.server.require-cookie", false)
val requireCookie = config.getBool("akka.cluster.server.require-cookie", false)
if (isRemotingEnabled && requireCookie && SECURE_COOKIE.isEmpty) throw new ConfigurationException(
"Configuration option 'akka.remote.server.require-cookie' is turned on but no secure cookie is defined in 'akka.remote.secure-cookie'.")
"Configuration option 'akka.cluster.server.require-cookie' is turned on but no secure cookie is defined in 'akka.cluster.secure-cookie'.")
requireCookie
}
val UNTRUSTED_MODE = config.getBool("akka.remote.server.untrusted-mode", false)
val HOSTNAME = config.getString("akka.remote.server.hostname", "localhost")
val PORT = config.getInt("akka.remote.server.port", 2552)
val CONNECTION_TIMEOUT_MILLIS = Duration(config.getInt("akka.remote.server.connection-timeout", 1), TIME_UNIT)
val COMPRESSION_SCHEME = config.getString("akka.remote.compression-scheme", "zlib")
val UNTRUSTED_MODE = config.getBool("akka.cluster.server.untrusted-mode", false)
val PORT = config.getInt("akka.cluster.server.port", 2552)
val CONNECTION_TIMEOUT_MILLIS = Duration(config.getInt("akka.cluster.server.connection-timeout", 1), TIME_UNIT)
val COMPRESSION_SCHEME = config.getString("akka.cluster.compression-scheme", "zlib")
val ZLIB_COMPRESSION_LEVEL = {
val level = config.getInt("akka.remote.zlib-compression-level", 6)
val level = config.getInt("akka.cluster.zlib-compression-level", 6)
if (level < 1 && level > 9) throw new IllegalArgumentException(
"zlib compression level has to be within 1-9, with 1 being fastest and 9 being the most compressed")
level
}
val BACKLOG = config.getInt("akka.remote.server.backlog", 4096)
val BACKLOG = config.getInt("akka.cluster.server.backlog", 4096)
val EXECUTION_POOL_KEEPALIVE = Duration(config.getInt("akka.remote.server.execution-pool-keepalive", 60), TIME_UNIT)
val EXECUTION_POOL_KEEPALIVE = Duration(config.getInt("akka.cluster.server.execution-pool-keepalive", 60), TIME_UNIT)
val EXECUTION_POOL_SIZE = {
val sz = config.getInt("akka.remote.server.execution-pool-size", 16)
if (sz < 1) throw new IllegalArgumentException("akka.remote.server.execution-pool-size is less than 1")
val sz = config.getInt("akka.cluster.server.execution-pool-size", 16)
if (sz < 1) throw new IllegalArgumentException("akka.cluster.server.execution-pool-size is less than 1")
sz
}
val MAX_CHANNEL_MEMORY_SIZE = {
val sz = config.getInt("akka.remote.server.max-channel-memory-size", 0)
if (sz < 0) throw new IllegalArgumentException("akka.remote.server.max-channel-memory-size is less than 0")
val sz = config.getInt("akka.cluster.server.max-channel-memory-size", 0)
if (sz < 0) throw new IllegalArgumentException("akka.cluster.server.max-channel-memory-size is less than 0")
sz
}
val MAX_TOTAL_MEMORY_SIZE = {
val sz = config.getInt("akka.remote.server.max-total-memory-size", 0)
if (sz < 0) throw new IllegalArgumentException("akka.remote.server.max-total-memory-size is less than 0")
val sz = config.getInt("akka.cluster.server.max-total-memory-size", 0)
if (sz < 0) throw new IllegalArgumentException("akka.cluster.server.max-total-memory-size is less than 0")
sz
}
}

View file

@ -148,8 +148,8 @@ abstract class RemoteClient private[akka] (
val module: NettyRemoteClientModule,
val remoteAddress: InetSocketAddress) {
val useTransactionLog = config.getBool("akka.remote.client.buffering.retry-message-send-on-failure", true)
val transactionLogCapacity = config.getInt("akka.remote.client.buffering.capacity", -1)
val useTransactionLog = config.getBool("akka.cluster.client.buffering.retry-message-send-on-failure", true)
val transactionLogCapacity = config.getInt("akka.cluster.client.buffering.capacity", -1)
val name = this.getClass.getSimpleName + "@" +
remoteAddress.getAddress.getHostAddress + "::" +
@ -879,9 +879,13 @@ class RemoteServerHandler(
case _ None
}
private def handleRemoteMessageProtocol(request: RemoteMessageProtocol, channel: Channel) = {
private def handleRemoteMessageProtocol(request: RemoteMessageProtocol, channel: Channel) = try {
EventHandler.debug(this, "Received remote message [%s]".format(request))
dispatchToActor(request, channel)
} catch {
case e: Exception
server.notifyListeners(RemoteServerError(e, server))
EventHandler.error(e, this, e.getMessage)
}
private def dispatchToActor(request: RemoteMessageProtocol, channel: Channel) {

View file

@ -4,20 +4,21 @@
package akka.serialization
import akka.dispatch.MessageInvocation
import akka.remote.protocol.RemoteProtocol._
import akka.remote.protocol.RemoteProtocol
import akka.config.Supervision._
import akka.actor.{ uuidFrom, newUuid }
import akka.actor._
import DeploymentConfig._
import akka.dispatch.MessageInvocation
import akka.util.ReflectiveAccess
import akka.remote.{ RemoteClientSettings, MessageSerializer }
import akka.remote.protocol.RemoteProtocol
import RemoteProtocol._
import scala.collection.immutable.Stack
import com.google.protobuf.ByteString
import akka.util.ReflectiveAccess
import java.net.InetSocketAddress
import akka.remote.{ RemoteClientSettings, MessageSerializer }
import com.google.protobuf.ByteString
/**
* Module for local actor serialization.
@ -31,19 +32,30 @@ object ActorSerialization {
def fromBinary[T <: Actor](bytes: Array[Byte])(implicit format: Serializer): ActorRef =
fromBinaryToLocalActorRef(bytes, None, format)
def toBinary[T <: Actor](a: ActorRef, serializeMailBox: Boolean = true)(implicit format: Serializer): Array[Byte] =
toSerializedActorRefProtocol(a, format, serializeMailBox).toByteArray
def toBinary[T <: Actor](
a: ActorRef,
serializeMailBox: Boolean = true,
replicationScheme: ReplicationScheme = Transient)(implicit format: Serializer): Array[Byte] =
toSerializedActorRefProtocol(a, format, serializeMailBox, replicationScheme).toByteArray
// wrapper for implicits to be used by Java
def fromBinaryJ[T <: Actor](bytes: Array[Byte], format: Serializer): ActorRef =
fromBinary(bytes)(format)
// wrapper for implicits to be used by Java
def toBinaryJ[T <: Actor](a: ActorRef, format: Serializer, srlMailBox: Boolean = true): Array[Byte] =
toBinary(a, srlMailBox)(format)
def toBinaryJ[T <: Actor](
a: ActorRef,
format: Serializer,
srlMailBox: Boolean,
replicationScheme: ReplicationScheme): Array[Byte] =
toBinary(a, srlMailBox, replicationScheme)(format)
private[akka] def toSerializedActorRefProtocol[T <: Actor](
actorRef: ActorRef, format: Serializer, serializeMailBox: Boolean = true): SerializedActorRefProtocol = {
actorRef: ActorRef,
format: Serializer,
serializeMailBox: Boolean,
replicationScheme: ReplicationScheme): SerializedActorRefProtocol = {
val lifeCycleProtocol: Option[LifeCycleProtocol] = {
actorRef.lifeCycle match {
case Permanent Some(LifeCycleProtocol.newBuilder.setLifeCycle(LifeCycleType.PERMANENT).build)
@ -58,6 +70,24 @@ object ActorSerialization {
.setActorClassname(actorRef.actorInstance.get.getClass.getName)
.setTimeout(actorRef.timeout)
replicationScheme match {
case _: Transient | Transient
builder.setReplicationStorage(ReplicationStorageType.TRANSIENT)
case Replication(storage, strategy)
val storageType = storage match {
case _: TransactionLog | TransactionLog ReplicationStorageType.TRANSACTION_LOG
case _: DataGrid | DataGrid ReplicationStorageType.DATA_GRID
}
builder.setReplicationStorage(storageType)
val strategyType = strategy match {
case _: WriteBehind | WriteBehind ReplicationStrategyType.WRITE_BEHIND
case _: WriteThrough | WriteThrough ReplicationStrategyType.WRITE_THROUGH
}
builder.setReplicationStrategy(strategyType)
}
if (serializeMailBox == true) {
if (actorRef.mailbox eq null) throw new IllegalActorStateException("Can't serialize an actor that has not been started.")
val messages =
@ -115,6 +145,29 @@ object ActorSerialization {
if (protocol.hasSupervisor) Some(RemoteActorSerialization.fromProtobufToRemoteActorRef(protocol.getSupervisor, loader))
else None
import ReplicationStorageType._
import ReplicationStrategyType._
val replicationScheme =
if (protocol.hasReplicationStorage) {
protocol.getReplicationStorage match {
case TRANSIENT Transient
case store
val storage = store match {
case TRANSACTION_LOG TransactionLog
case DATA_GRID DataGrid
}
val strategy = if (protocol.hasReplicationStrategy) {
protocol.getReplicationStrategy match {
case WRITE_THROUGH WriteThrough
case WRITE_BEHIND WriteBehind
}
} else throw new IllegalActorStateException(
"Expected replication strategy for replication storage [" + storage + "]")
Replication(storage, strategy)
}
} else Transient
val hotswap =
try {
format
@ -124,7 +177,7 @@ object ActorSerialization {
case e: Exception Stack[PartialFunction[Any, Unit]]()
}
val classLoader = loader.getOrElse(getClass.getClassLoader)
val classLoader = loader.getOrElse(this.getClass.getClassLoader)
val factory = () {
val actorClass = classLoader.loadClass(protocol.getActorClassname)
@ -143,7 +196,8 @@ object ActorSerialization {
lifeCycle,
supervisor,
hotswap,
factory)
factory,
replicationScheme)
val messages = protocol.getMessagesList.toArray.toList.asInstanceOf[List[RemoteMessageProtocol]]
messages.foreach(message ar ! MessageSerializer.deserialize(message.getMessage))

View file

@ -3,8 +3,8 @@
*/
package sample.osgi
import akka.actor.{ Actor, ActorRegistry }
import Actor._
import akka.actor.Actor
import akka.actor.Actor._
import org.osgi.framework.{ BundleActivator, BundleContext }
@ -13,7 +13,7 @@ class Activator extends BundleActivator {
def start(context: BundleContext) {
println("Starting the OSGi example ...")
val echo = actorOf[EchoActor].start()
val answer = (echo !! "OSGi example")
val answer = (echo ? "OSGi example").as[String]
println(answer getOrElse "No answer!")
}

View file

@ -39,7 +39,7 @@ class Slf4jEventHandler extends Actor with Logging {
def receive = {
case Error(cause, instance, message)
log.error("\n\t[{}]\n\t[{}]\n\t[{}]",
Array[Any](instance.getClass.getName, message, stackTraceFor(cause)))
Array[AnyRef](instance.getClass.getName, message.asInstanceOf[AnyRef], stackTraceFor(cause)))
case Warning(instance, message)
log.warn("\n\t[{}]\n\t[{}]", instance.getClass.getName, message)

View file

@ -25,8 +25,8 @@ akka {
actor {
timeout = 2000 # Default timeout for Future based invocations
# - Actor: !! && !!!
# - UntypedActor: sendRequestReply && sendRequestReplyFuture
# - Actor: !! && ?
# - UntypedActor: sendRequestReply && ask
# - TypedActor: methods with non-void return type
serialize-messages = off # Does a deep clone of (non-primitive) messages to ensure immutability
throughput = 5 # Default throughput for all ExecutorBasedEventDrivenDispatcher, set to 1 for complete fairness

View file

@ -15,7 +15,7 @@ http://akka.io/akka-2.0-SNAPSHOT.xsd">
<akka:untyped-actor id="actor-1"
implementation="akka.spring.foo.PingActor"
timeout="${akka.actor.timeout}">
<akka:remote host="${akka.remote.server.hostname}" port="${akka.remote.server.port}"/>
<akka:remote host="${akka.cluster.server.hostname}" port="${akka.cluster.server.port}"/>
</akka:untyped-actor>

View file

@ -120,7 +120,7 @@ class Agent[T](initialValue: T) {
* within the given timeout
*/
def alter(f: T T)(timeout: Long): Future[T] = {
def dispatch = updater.!!!(Update(f), timeout)
def dispatch = updater.?(Update(f))(timeout = timeout).asInstanceOf[Future[T]]
if (Stm.activeTransaction) {
val result = new DefaultPromise[T](timeout)
get //Join xa
@ -168,7 +168,7 @@ class Agent[T](initialValue: T) {
send((value: T) {
suspend
val threadBased = Actor.actorOf(new ThreadBasedAgentUpdater(this)).start()
result completeWith threadBased.!!!(Update(f), timeout)
result completeWith threadBased.?(Update(f))(timeout = timeout).asInstanceOf[Future[T]]
value
})
result
@ -178,7 +178,7 @@ class Agent[T](initialValue: T) {
* A future to the current value that will be completed after any currently
* queued updates.
*/
def future(): Future[T] = (updater !!! Get).asInstanceOf[Future[T]]
def future(): Future[T] = (updater ? Get).asInstanceOf[Future[T]]
/**
* Gets this agent's value after all currently queued updates have completed.

View file

@ -19,8 +19,8 @@ public class UntypedCoordinatedExample {
Thread.sleep(3000);
Future future1 = counter1.sendRequestReplyFuture("GetCount");
Future future2 = counter2.sendRequestReplyFuture("GetCount");
Future future1 = counter1.ask("GetCount");
Future future2 = counter2.ask("GetCount");
future1.await();
if (future1.isCompleted()) {

View file

@ -18,8 +18,8 @@ public class UntypedTransactorExample {
Thread.sleep(3000);
Future future1 = counter1.sendRequestReplyFuture("GetCount");
Future future2 = counter2.sendRequestReplyFuture("GetCount");
Future future1 = counter1.ask("GetCount");
Future future2 = counter2.ask("GetCount");
future1.await();
if (future1.isCompleted()) {

View file

@ -49,7 +49,7 @@ public class UntypedCoordinatedIncrementTest {
incrementLatch.await(timeout, TimeUnit.SECONDS);
} catch (InterruptedException exception) {}
for (ActorRef counter : counters) {
Future future = counter.sendRequestReplyFuture("GetCount");
Future future = counter.ask("GetCount");
future.await();
if (future.isCompleted()) {
Option resultOption = future.result();
@ -72,7 +72,7 @@ public class UntypedCoordinatedIncrementTest {
incrementLatch.await(timeout, TimeUnit.SECONDS);
} catch (InterruptedException exception) {}
for (ActorRef counter : counters) {
Future future = counter.sendRequestReplyFuture("GetCount");
Future future = counter.ask("GetCount");
future.await();
if (future.isCompleted()) {
Option resultOption = future.result();

View file

@ -48,7 +48,7 @@ public class UntypedTransactorTest {
incrementLatch.await(timeout, TimeUnit.SECONDS);
} catch (InterruptedException exception) {}
for (ActorRef counter : counters) {
Future future = counter.sendRequestReplyFuture("GetCount");
Future future = counter.ask("GetCount");
future.await();
if (future.isCompleted()) {
Option resultOption = future.result();
@ -71,7 +71,7 @@ public class UntypedTransactorTest {
incrementLatch.await(timeout, TimeUnit.SECONDS);
} catch (InterruptedException exception) {}
for (ActorRef counter : counters) {
Future future = counter.sendRequestReplyFuture("GetCount");
Future future = counter.ask("GetCount");
future.await();
if (future.isCompleted()) {
Option resultOption = future.result();

View file

@ -64,7 +64,7 @@ class CoordinatedIncrementSpec extends WordSpec with MustMatchers {
counters(0) ! coordinated(Increment(counters.tail))
coordinated.await
for (counter counters) {
(counter !! GetCount).get must be === 1
(counter ? GetCount).as[Int].get must be === 1
}
counters foreach (_.stop())
failer.stop()
@ -76,7 +76,7 @@ class CoordinatedIncrementSpec extends WordSpec with MustMatchers {
counters(0) ! Coordinated(Increment(counters.tail :+ failer))
coordinated.await
for (counter counters) {
(counter !! GetCount).get must be === 0
(counter ? GetCount).as[Int].get must be === 0
}
counters foreach (_.stop())
failer.stop()

View file

@ -109,9 +109,9 @@ class FickleFriendsSpec extends WordSpec with MustMatchers {
val latch = new CountDownLatch(1)
coordinator ! FriendlyIncrement(counters, latch)
latch.await // this could take a while
(coordinator !! GetCount).get must be === 1
(coordinator ? GetCount).as[Int].get must be === 1
for (counter counters) {
(counter !! GetCount).get must be === 1
(counter ? GetCount).as[Int].get must be === 1
}
counters foreach (_.stop())
coordinator.stop()

View file

@ -92,7 +92,7 @@ class TransactorSpec extends WordSpec with MustMatchers {
counters(0) ! Increment(counters.tail, incrementLatch)
incrementLatch.await(timeout.length, timeout.unit)
for (counter counters) {
(counter !! GetCount).get must be === 1
(counter ? GetCount).as[Int].get must be === 1
}
counters foreach (_.stop())
failer.stop()
@ -104,7 +104,7 @@ class TransactorSpec extends WordSpec with MustMatchers {
counters(0) ! Increment(counters.tail :+ failer, failLatch)
failLatch.await(timeout.length, timeout.unit)
for (counter counters) {
(counter !! GetCount).get must be === 0
(counter ? GetCount).as[Int].get must be === 0
}
counters foreach (_.stop())
failer.stop()

View file

@ -135,6 +135,8 @@ class CallingThreadDispatcher(val warnings: Boolean = true) extends MessageDispa
override def mailboxSize(actor: ActorRef) = getMailbox(actor).queue.size
def mailboxIsEmpty(actorRef: ActorRef): Boolean = getMailbox(actorRef).queue.isEmpty
private[akka] override def dispatch(handle: MessageInvocation) {
val mbox = getMailbox(handle.receiver)
val queue = mbox.queue
@ -214,6 +216,7 @@ class CallingThreadDispatcher(val warnings: Boolean = true) extends MessageDispa
class NestingQueue {
private var q = new LinkedList[MessageInvocation]()
def size = q.size
def isEmpty = q.isEmpty
def push(handle: MessageInvocation) { q.offer(handle) }
def peek = q.peek
def pop = q.poll

View file

@ -15,11 +15,11 @@ import com.eaio.uuid.UUID
* overrides the dispatcher to CallingThreadDispatcher and sets the receiveTimeout to None. Otherwise,
* it acts just like a normal ActorRef. You may retrieve a reference to the underlying actor to test internal logic.
*
*
* @author Roland Kuhn
* @since 1.1
*/
class TestActorRef[T <: Actor](factory: () T, address: String) extends LocalActorRef(factory, address) {
class TestActorRef[T <: Actor](factory: () T, address: String)
extends LocalActorRef(factory, address, DeploymentConfig.Transient) {
dispatcher = CallingThreadDispatcher.global
receiveTimeout = None

View file

@ -114,7 +114,7 @@ class TestActorRefSpec extends WordSpec with MustMatchers with BeforeAndAfterEac
def receive = { case _ self reply nested }
}).start()
a must not be (null)
val nested = (a !! "any").get.asInstanceOf[ActorRef]
val nested = (a ? "any").as[ActorRef].get
nested must not be (null)
a must not be theSameInstanceAs(nested)
}
@ -125,7 +125,7 @@ class TestActorRefSpec extends WordSpec with MustMatchers with BeforeAndAfterEac
def receive = { case _ self reply nested }
}).start()
a must not be (null)
val nested = (a !! "any").get.asInstanceOf[ActorRef]
val nested = (a ? "any").as[ActorRef].get
nested must not be (null)
a must not be theSameInstanceAs(nested)
}
@ -160,7 +160,7 @@ class TestActorRefSpec extends WordSpec with MustMatchers with BeforeAndAfterEac
"stop when sent a poison pill" in {
val a = TestActorRef[WorkerActor].start()
intercept[ActorKilledException] {
a !! PoisonPill
(a ? PoisonPill).get
}
a must not be ('running)
a must be('shutdown)

View file

@ -184,7 +184,7 @@ public class Pi {
// send calculate message
long timeout = 60000;
Future<Object> replyFuture = master.sendRequestReplyFuture(new Calculate(), timeout, null);
Future<Object> replyFuture = master.ask(new Calculate(), timeout, null);
Option<Object> result = replyFuture.await().resultOrException();
if (result.isDefined()) {
double pi = (Double) result.get();

View file

@ -104,7 +104,7 @@ object Pi extends App {
val start = now
//send calculate message
master.!!![Double](Calculate, timeout = 60000).
master.?(Calculate)(timeout = Actor.Timeout(60000)).
await.resultOrException match {//wait for the result, with a 60 seconds timeout
case Some(pi) =>
EventHandler.info(this, "\n\tPi estimate: \t\t%s\n\tCalculation time: \t%s millis".format(pi, (now - start)))

View file

@ -10,7 +10,7 @@
akka {
version = "2.0-SNAPSHOT" # Akka version, checked against the runtime version of Akka.
enabled-modules = [] # Comma separated list of the enabled modules. Options: ["remote", "camel", "http"]
enabled-modules = [] # Comma separated list of the enabled modules. Options: ["cluster", "camel", "http"]
time-unit = "seconds" # Time unit for all timeout properties throughout the config
@ -27,63 +27,76 @@ akka {
boot = []
actor {
timeout = 5 # Default timeout for Future based invocations
# - Actor: !! && !!!
# - UntypedActor: sendRequestReply && sendRequestReplyFuture
# - TypedActor: methods with non-void return type
serialize-messages = off # Does a deep clone of (non-primitive) messages to ensure immutability
throughput = 5 # Default throughput for all Dispatcher, set to 1 for complete fairness
throughput-deadline-time = -1 # Default throughput deadline for all Dispatcher, set to 0 or negative for no deadline
dispatcher-shutdown-timeout = 1 # Using the akka.time-unit, how long dispatchers by default will wait for new actors until they shut down
timeout = 5 # Default timeout for Future based invocations
# - Actor: !! && ?
# - UntypedActor: sendRequestReply && ask
# - TypedActor: methods with non-void return type
serialize-messages = off # Does a deep clone of (non-primitive) messages to ensure immutability
throughput = 5 # Default throughput for all Dispatcher, set to 1 for complete fairness
throughput-deadline-time = -1 # Default throughput deadline for all Dispatcher, set to 0 or negative for no deadline
dispatcher-shutdown-timeout = 1 # Using the akka.time-unit, how long dispatchers by default will wait for new actors until they shut down
deployment {
# -------------------------------
# -- all configuration options --
# -------------------------------
service-ping { # stateless actor with replication factor 3 and round-robin load-balancer
service-ping { # stateless actor with replication factor 3 and round-robin load-balancer
router = "least-cpu" # routing (load-balance) scheme to use
# available: "direct", "round-robin", "random", "least-cpu", "least-ram", "least-messages"
# or: fully qualified class name of the router class
# default is "direct";
format = "akka.serialization.Format$Default$"
clustered { # makes the actor available in the cluster registry
# default (if omitted) is local non-clustered actor
home = "node:node1" # defines the hostname, IP-address or node name of the "home" node for clustered actor
# available: "host:<hostname>", "ip:<ip address>" and "node:<node name>"
# default is "host:localhost"
replicas = 3 # number of actor replicas in the cluster
# available: positivoe integer (0-N) or the string "auto" for auto-scaling
# if "auto" is used then 'home' has no meaning
# default is '0', meaning no replicas;
stateless = on # is the actor stateless or stateful
# if turned 'on': actor is defined as stateless and can be load-balanced accordingly
# if turned 'off' (or omitted): actor is defined as stateful which means replicatable through transaction log
# default is 'off'
format = "akka.serialization.Format$Default$" # serializer for messages and actor instance
router = "least-cpu" # routing (load-balance) scheme to use
# available: "direct", "round-robin", "random",
# "least-cpu", "least-ram", "least-messages"
# or: fully qualified class name of the router class
# default is "direct";
clustered { # makes the actor available in the cluster registry
# default (if omitted) is local non-clustered actor
home = "node:node1" # hostname, IP-address or node name of the "home" node for clustered actor
# available: "host:<hostname>", "ip:<ip address>" and "node:<node name>"
# default is "host:localhost"
replicas = 3 # number of actor replicas in the cluster
# available: positivoe integer (0-N) or the string "auto" for auto-scaling
# if "auto" is used then 'home' has no meaning
# default is '0', meaning no replicas;
replication { # use replication or not?
# FIXME should we have this config option here? If so, implement it all through.
serialize-mailbox = on # should the actor mailbox be part of the serialized snapshot?
storage = "transaction-log" # storage model for replication
# available: "transaction-log" and "data-grid"
# default is "transaction-log"
strategy = "write-through" # guaranteees for replication
# available: "write-through" and "write-behind"
# default is "write-through"
}
}
}
}
default-dispatcher {
type = "GlobalDispatcher" # Must be one of the following, all "Global*" are non-configurable
# - Dispatcher
# - BalancingDispatcher
# - GlobalDispatcher
keep-alive-time = 60 # Keep alive time for threads
core-pool-size-factor = 1.0 # No of core threads ... ceil(available processors * factor)
max-pool-size-factor = 4.0 # Max no of threads ... ceil(available processors * factor)
executor-bounds = -1 # Makes the Executor bounded, -1 is unbounded
allow-core-timeout = on # Allow core threads to time out
rejection-policy = "caller-runs" # abort, caller-runs, discard-oldest, discard
throughput = 5 # Throughput for Dispatcher, set to 1 for complete fairness
throughput-deadline-time = -1 # Throughput deadline for Dispatcher, set to 0 or negative for no deadline
mailbox-capacity = -1 # If negative (or zero) then an unbounded mailbox is used (default)
# If positive then a bounded mailbox is used and the capacity is set using the property
# NOTE: setting a mailbox to 'blocking' can be a bit dangerous, could lead to deadlock, use with care
# The following are only used for Dispatcher and only if mailbox-capacity > 0
mailbox-push-timeout-time = 10 # Specifies the timeout to add a new message to a mailbox that is full - negative number means infinite timeout
# (in unit defined by the time-unit property)
type = "GlobalDispatcher" # Must be one of the following, all "Global*" are non-configurable
# - Dispatcher
# - BalancingDispatcher
# - GlobalDispatcher
keep-alive-time = 60 # Keep alive time for threads
core-pool-size-factor = 1.0 # No of core threads ... ceil(available processors * factor)
max-pool-size-factor = 4.0 # Max no of threads ... ceil(available processors * factor)
executor-bounds = -1 # Makes the Executor bounded, -1 is unbounded
allow-core-timeout = on # Allow core threads to time out
rejection-policy = "caller-runs" # abort, caller-runs, discard-oldest, discard
throughput = 5 # Throughput for Dispatcher, set to 1 for complete fairness
throughput-deadline-time = -1 # Throughput deadline for Dispatcher, set to 0 or negative for no deadline
mailbox-capacity = -1 # If negative (or zero) then an unbounded mailbox is used (default)
# If positive then a bounded mailbox is used and the capacity is set using the property
# NOTE: setting a mailbox to 'blocking' can be a bit dangerous, could lead to deadlock, use with care
# The following are only used for Dispatcher and only if mailbox-capacity > 0
mailbox-push-timeout-time = 10 # Specifies the timeout to add a new message to a mailbox that is full - negative number means infinite timeout
# (in unit defined by the time-unit property)
}
mailbox {
@ -128,39 +141,36 @@ akka {
cluster {
name = "test-cluster"
zookeeper-server-addresses = "localhost:2181"
zookeeper-server-addresses = "localhost:2181" # comma-separated list of '<hostname>:<port>' elements
remote-server-port = 2552
max-time-to-wait-until-connected = 30
session-timeout = 60
connection-timeout = 60
use-compression = off
remote-daemon-ack-timeout = 30 # Timeout for ACK of cluster operations, lik checking actor out etc.
exclude-ref-node-in-replica-set = on # Should a replica be instantiated on the same node as the
# cluster reference to the actor
# Default: on
replication {
digest-type = "MAC" # Options: CRC32 (cheap & unsafe), MAC (expensive & secure using password)
password = "secret" # FIXME: store open in file?
ensemble-size = 3
quorum-size = 2
}
}
remote {
# secure-cookie = "050E0A0D0D06010A00000900040D060F0C09060B" # generate your own with '$AKKA_HOME/scripts/generate_config_with_secure_cookie.sh' or using 'Crypt.generateSecureCookie'
secure-cookie = ""
compression-scheme = "zlib" # Options: "zlib" (lzf to come), leave out for no compression
zlib-compression-level = 6 # Options: 0-9 (1 being fastest and 9 being the most compressed), default is 6
remote-daemon-ack-timeout = 30 # Timeout for ACK of cluster operations, lik checking actor out etc.
exclude-ref-node-in-replica-set = on # Should a replica be instantiated on the same node as the
# cluster reference to the actor
# Default: on
compression-scheme = "zlib" # Options: "zlib" (lzf to come), leave out for no compression
zlib-compression-level = 6 # Options: 0-9 (1 being fastest and 9 being the most compressed), default is 6
# FIXME rename to transport
layer = "akka.remote.netty.NettyRemoteSupport"
secure-cookie = "" # generate your own with '$AKKA_HOME/scripts/generate_config_with_secure_cookie.sh'
# or using 'Crypt.generateSecureCookie'
replication {
digest-type = "MAC" # Options: CRC32 (cheap & unsafe), MAC (expensive & secure using password)
password = "secret" # FIXME: store open in file?
ensemble-size = 3
quorum-size = 2
snapshot-frequency = 1000 # The number of messages that should be logged between every actor snapshot
timeout = 30 # Timeout for asyncronous (write-behind) operations
}
server {
# FIXME remove hostname/port
hostname = "localhost" # The hostname or IP that clients should connect to
port = 2552 # The port clients should connect to. Default is 2552 (AKKA)
port = 2552 # The default remote server port clients should connect to. Default is 2552 (AKKA)
message-frame-size = 1048576 # Increase this if you want to be able to send messages with large payloads
connection-timeout = 1
require-cookie = off # Should the remote server require that it peers share the same secure-cookie (defined in the 'remote' section)?
@ -182,7 +192,7 @@ akka {
read-timeout = 10
message-frame-size = 1048576
reap-futures-delay = 5
reconnection-time-window = 600 # Maximum time window that a client should try to reconnect for
reconnection-time-window = 600 # Maximum time window that a client should try to reconnect for
}
}
@ -204,15 +214,14 @@ akka {
hostname = "localhost"
port = 9998
# If you are using akka.http.AkkaMistServlet
mist-dispatcher {
#type = "GlobalDispatcher" # Uncomment if you want to use a different dispatcher than the default one for Comet
mist-dispatcher { # If you are using akka.http.AkkaMistServlet
#type = "GlobalDispatcher" # Uncomment if you want to use a different dispatcher than the default one for Comet
}
connection-close = true # toggles the addition of the "Connection" response header with a "close" value
root-actor-id = "_httproot" # the id of the actor to use as the root endpoint
root-actor-builtin = true # toggles the use of the built-in root endpoint base class
timeout = 1000 # the default timeout for all async requests (in ms)
expired-header-name = "Async-Timeout" # the name of the response header to use when an async request expires
expired-header-value = "expired" # the value of the response header to use when an async request expires
connection-close = true # toggles the addition of the "Connection" response header with a "close" value
root-actor-id = "_httproot" # the id of the actor to use as the root endpoint
root-actor-builtin = true # toggles the use of the built-in root endpoint base class
timeout = 1000 # the default timeout for all async requests (in ms)
expired-header-name = "Async-Timeout" # the name of the response header to use when an async request expires
expired-header-value = "expired" # the value of the response header to use when an async request expires
}
}

View file

@ -0,0 +1,106 @@
<?xml version="1.0"?>
<!DOCTYPE Configure PUBLIC "-//Jetty//Configure//EN" "http://www.eclipse.org/jetty/configure.dtd">
<!-- =============================================================== -->
<!-- Configure the Jetty Server -->
<!-- -->
<!-- Documentation of this file format can be found at: -->
<!-- http://wiki.eclipse.org/Jetty/Reference/jetty.xml_syntax -->
<!-- -->
<!-- Additional configuration files are available in $JETTY_HOME/etc -->
<!-- and can be mixed in. For example: -->
<!-- java -jar start.jar etc/jetty.xml etc/jetty-ssl.xml -->
<!-- -->
<!-- See start.ini file for the default configuration files -->
<!-- =============================================================== -->
<Configure id="Server" class="org.eclipse.jetty.server.Server">
<!-- =========================================================== -->
<!-- Server Thread Pool -->
<!-- =========================================================== -->
<Set name="ThreadPool">
<New class="org.eclipse.jetty.util.thread.ExecutorThreadPool">
</New>
</Set>
<!-- =========================================================== -->
<!-- Set connectors -->
<!-- =========================================================== -->
<Call name="addConnector">
<Arg>
<New class="org.eclipse.jetty.server.nio.SelectChannelConnector">
<Set name="host"><SystemProperty name="jetty.host" /></Set>
<Set name="port"><SystemProperty name="jetty.port" default="8080"/></Set>
<Set name="maxIdleTime">300000</Set>
<Set name="Acceptors">2</Set>
<Set name="statsOn">false</Set>
<Set name="confidentialPort">8443</Set>
<Set name="lowResourcesConnections">20000</Set>
<Set name="lowResourcesMaxIdleTime">5000</Set>
</New>
</Arg>
</Call>
<!-- Uncomment this and enter your SSL config/credentials to enable https
<Call name="addConnector">
<Arg>
<New class="org.eclipse.jetty.server.ssl.SslSelectChannelConnector">
<Set name="Port">8443</Set>
<Set name="maxIdleTime">30000</Set>
<Set name="Acceptors">2</Set>
<Set name="AcceptQueueSize">100</Set>
<Set name="Keystore"><SystemProperty name="jetty.home" default="." />/etc/keystore</Set>
<Set name="Password">PASSWORD</Set>
<Set name="KeyPassword">KEYPASSWORD</Set>
<Set name="truststore"><SystemProperty name="jetty.home" default="." />/etc/keystore</Set>
<Set name="trustPassword">TRUSTPASSWORD</Set>
</New>
</Arg>
</Call>
-->
<!-- =========================================================== -->
<!-- Set handler Collection Structure -->
<!-- =========================================================== -->
<Set name="handler">
<New id="Handlers" class="org.eclipse.jetty.server.handler.HandlerCollection">
<Set name="handlers">
<Array type="org.eclipse.jetty.server.Handler">
<!--Item>
<New id="AkkaRestHandler" class="org.eclipse.jetty.servlet.ServletContextHandler">
<Set name="contextPath">/</Set>
<Call name="addServlet">
<Arg>akka.http.AkkaRestServlet</Arg>
<Arg>/*</Arg>
</Call>
</New>
</Item-->
<Item>
<New id="AkkaMistHandler" class="org.eclipse.jetty.servlet.ServletContextHandler">
<Set name="contextPath">/</Set>
<Call name="addServlet">
<Arg>akka.http.AkkaMistServlet</Arg>
<Arg>/*</Arg>
</Call>
</New>
</Item>
<Item>
<New id="DefaultHandler" class="org.eclipse.jetty.server.handler.DefaultHandler"/>
</Item>
</Array>
</Set>
</New>
</Set>
<!-- =========================================================== -->
<!-- extra options -->
<!-- =========================================================== -->
<Set name="stopAtShutdown">true</Set>
<Set name="sendServerVersion">true</Set>
<Set name="sendDateHeader">true</Set>
<Set name="gracefulShutdown">1000</Set>
</Configure>

View file

@ -73,6 +73,7 @@ class AkkaParentProject(info: ProjectInfo) extends ParentProject(info) with Exec
lazy val jmsModuleConfig = ModuleConfiguration("javax.jms", JBossRepo)
lazy val jsr311ModuleConfig = ModuleConfiguration("javax.ws.rs", "jsr311-api", sbt.DefaultMavenRepository)
lazy val zookeeperModuleConfig = ModuleConfiguration("org.apache.hadoop.zookeeper", AkkaRepo)
lazy val protobufModuleConfig = ModuleConfiguration("com.google.protobuf", AkkaRepo)
lazy val zkclientModuleConfig = ModuleConfiguration("zkclient", AkkaRepo)
lazy val camelModuleConfig = ModuleConfiguration("org.apache.camel", "camel-core", AkkaRepo)
@ -126,7 +127,7 @@ class AkkaParentProject(info: ProjectInfo) extends ParentProject(info) with Exec
lazy val multiverse = "org.multiverse" % "multiverse-alpha" % MULTIVERSE_VERSION % "compile" //ApacheV2
lazy val netty = "org.jboss.netty" % "netty" % "3.2.4.Final" % "compile" //ApacheV2
lazy val osgi_core = "org.osgi" % "org.osgi.core" % "4.2.0" //ApacheV2
lazy val protobuf = "com.google.protobuf" % "protobuf-java" % "2.3.0" % "compile" //New BSD
lazy val protobuf = "com.google.protobuf" % "protobuf-java" % "2.4.1" % "compile" //New BSD
lazy val redis = "net.debasishg" % "redisclient_2.9.0" % "2.3.1" //ApacheV2
lazy val sjson = "net.debasishg" %% "sjson" % "0.11" % "compile" //ApacheV2
lazy val sjson_test = "net.debasishg" %% "sjson" % "0.11" % "test" //ApacheV2
@ -135,7 +136,7 @@ class AkkaParentProject(info: ProjectInfo) extends ParentProject(info) with Exec
lazy val spring_context = "org.springframework" % "spring-context" % SPRING_VERSION % "compile" //ApacheV2
lazy val stax_api = "javax.xml.stream" % "stax-api" % "1.0-2" % "compile" //ApacheV2
lazy val logback = "ch.qos.logback" % "logback-classic" % "0.9.28" % "runtime" //MIT
lazy val logback = "ch.qos.logback" % "logback-classic" % "0.9.28" % "runtime" //MIT
lazy val log4j = "log4j" % "log4j" % "1.2.15" //ApacheV2
lazy val zookeeper = "org.apache.hadoop.zookeeper" % "zookeeper" % ZOOKEEPER_VERSION //ApacheV2
lazy val zookeeper_lock = "org.apache.hadoop.zookeeper" % "zookeeper-recipes-lock" % ZOOKEEPER_VERSION //ApacheV2
@ -143,14 +144,14 @@ class AkkaParentProject(info: ProjectInfo) extends ParentProject(info) with Exec
// Test
lazy val multiverse_test = "org.multiverse" % "multiverse-alpha" % MULTIVERSE_VERSION % "test" //ApacheV2
lazy val commons_coll = "commons-collections" % "commons-collections" % "3.2.1" % "test" //ApacheV2
lazy val testJetty = "org.eclipse.jetty" % "jetty-server" % JETTY_VERSION % "test" //Eclipse license
lazy val testJettyWebApp = "org.eclipse.jetty" % "jetty-webapp" % JETTY_VERSION % "test" //Eclipse license
lazy val junit = "junit" % "junit" % "4.5" % "test" //Common Public License 1.0
lazy val mockito = "org.mockito" % "mockito-all" % "1.8.1" % "test" //MIT
lazy val scalatest = "org.scalatest" %% "scalatest" % SCALATEST_VERSION % "test" //ApacheV2
lazy val testLogback = "ch.qos.logback" % "logback-classic" % LOGBACK_VERSION % "test" // EPL 1.0 / LGPL 2.1
lazy val camel_spring = "org.apache.camel" % "camel-spring" % CAMEL_VERSION % "test" //ApacheV2
lazy val commons_coll = "commons-collections" % "commons-collections" % "3.2.1" % "test" //ApacheV2
lazy val testJetty = "org.eclipse.jetty" % "jetty-server" % JETTY_VERSION % "test" //Eclipse license
lazy val testJettyWebApp = "org.eclipse.jetty" % "jetty-webapp" % JETTY_VERSION % "test" //Eclipse license
lazy val junit = "junit" % "junit" % "4.5" % "test" //Common Public License 1.0
lazy val mockito = "org.mockito" % "mockito-all" % "1.8.1" % "test" //MIT
lazy val scalatest = "org.scalatest" %% "scalatest" % SCALATEST_VERSION % "test" //ApacheV2
lazy val testLogback = "ch.qos.logback" % "logback-classic" % LOGBACK_VERSION % "test" // EPL 1.0 / LGPL 2.1
lazy val camel_spring = "org.apache.camel" % "camel-spring" % CAMEL_VERSION % "test" //ApacheV2
}
@ -168,10 +169,10 @@ class AkkaParentProject(info: ProjectInfo) extends ParentProject(info) with Exec
lazy val akka_cluster = project("akka-cluster", "akka-cluster", new AkkaClusterProject(_), akka_remote)
lazy val akka_durable_mailboxes = project("akka-durable-mailboxes", "akka-durable-mailboxes", new AkkaDurableMailboxesParentProject(_), akka_remote)
lazy val akka_camel = project("akka-camel", "akka-camel", new AkkaCamelProject(_), akka_actor, akka_slf4j)
lazy val akka_camel_typed = project("akka-camel-typed", "akka-camel-typed", new AkkaCamelTypedProject(_), akka_actor, akka_slf4j, akka_camel)
//lazy val akka_camel = project("akka-camel", "akka-camel", new AkkaCamelProject(_), akka_actor, akka_slf4j)
//lazy val akka_camel_typed = project("akka-camel-typed", "akka-camel-typed", new AkkaCamelTypedProject(_), akka_actor, akka_slf4j, akka_camel)
//lazy val akka_spring = project("akka-spring", "akka-spring", new AkkaSpringProject(_), akka_remote, akka_actor, akka_camel)
lazy val akka_kernel = project("akka-kernel", "akka-kernel", new AkkaKernelProject(_), akka_stm, akka_remote, akka_http, akka_slf4j, akka_camel)
//lazy val akka_kernel = project("akka-kernel", "akka-kernel", new AkkaKernelProject(_), akka_stm, akka_remote, akka_http, akka_slf4j, akka_camel)
lazy val akka_sbt_plugin = project("akka-sbt-plugin", "akka-sbt-plugin", new AkkaSbtPluginProject(_))
lazy val akka_tutorials = project("akka-tutorials", "akka-tutorials", new AkkaTutorialsParentProject(_), akka_actor)
@ -616,16 +617,16 @@ class AkkaParentProject(info: ProjectInfo) extends ParentProject(info) with Exec
lazy val akka_sample_ants = project("akka-sample-ants", "akka-sample-ants",
new AkkaSampleAntsProject(_), akka_stm)
// lazy val akka_sample_chat = project("akka-sample-chat", "akka-sample-chat",
// new AkkaSampleChatProject(_), akka_remote)
// lazy val akka_sample_chat = project("akka-sample-chat", "akka-sample-chat",
// new AkkaSampleChatProject(_), akka_remote)
lazy val akka_sample_fsm = project("akka-sample-fsm", "akka-sample-fsm",
new AkkaSampleFSMProject(_), akka_actor)
lazy val akka_sample_hello = project("akka-sample-hello", "akka-sample-hello",
new AkkaSampleHelloProject(_), akka_kernel)
// lazy val akka_sample_hello = project("akka-sample-hello", "akka-sample-hello",
// new AkkaSampleHelloProject(_), akka_kernel)
lazy val akka_sample_osgi = project("akka-sample-osgi", "akka-sample-osgi",
new AkkaSampleOsgiProject(_), akka_actor)
// lazy val akka_sample_remote = project("akka-sample-remote", "akka-sample-remote",
// new AkkaSampleRemoteProject(_), akka_remote)
// lazy val akka_sample_remote = project("akka-sample-remote", "akka-sample-remote",
// new AkkaSampleRemoteProject(_), akka_remote)
lazy val publishRelease = {
val releaseConfiguration = new DefaultPublishConfiguration(localReleaseRepository, "release", false)
@ -787,8 +788,8 @@ class AkkaParentProject(info: ProjectInfo) extends ParentProject(info) with Exec
lazy val akkaCoreDist = project("core", "akka-dist-core", new AkkaCoreDistProject(_),
akkaActorsDist, akka_remote, akka_http, akka_slf4j, akka_testkit, akka_actor_tests)
lazy val akkaMicrokernelDist = project("microkernel", "akka-dist-microkernel", new AkkaMicrokernelDistProject(_),
akkaCoreDist, akka_kernel, akka_samples)
// lazy val akkaMicrokernelDist = project("microkernel", "akka-dist-microkernel", new AkkaMicrokernelDistProject(_),
// akkaCoreDist, akka_kernel, akka_samples)
def doNothing = task { None }
override def publishLocalAction = doNothing
@ -830,44 +831,44 @@ class AkkaParentProject(info: ProjectInfo) extends ParentProject(info) with Exec
override def distScriptSources = akkaParent.info.projectPath / "scripts" / "microkernel" * "*"
override def distClasspath = akka_kernel.runClasspath
// override def distClasspath = akka_kernel.runClasspath
override def projectDependencies = akka_kernel.topologicalSort
// override def projectDependencies = akka_kernel.topologicalSort
override def distAction = super.distAction dependsOn (distSamples)
// override def distAction = super.distAction dependsOn (distSamples)
val distSamplesPath = distDocPath / "samples"
// val distSamplesPath = distDocPath / "samples"
lazy val distSamples = task {
val demo = akka_samples.akka_sample_hello.jarPath
val samples = Set(//akka_samples.akka_sample_camel
akka_samples.akka_sample_hello)
//akka_samples.akka_sample_security)
// lazy val distSamples = task {
// val demo = akka_samples.akka_sample_hello.jarPath
// val samples = Set(//akka_samples.akka_sample_camel
// akka_samples.akka_sample_hello)
// //akka_samples.akka_sample_security)
def copySamples[P <: DefaultProject](samples: Set[P]) = {
samples.map { sample =>
val sampleOutputPath = distSamplesPath / sample.name
val binPath = sampleOutputPath / "bin"
val configPath = sampleOutputPath / "config"
val deployPath = sampleOutputPath / "deploy"
val libPath = sampleOutputPath / "lib"
val srcPath = sampleOutputPath / "src"
val confs = sample.info.projectPath / "config" ** "*.*"
val scripts = akkaParent.info.projectPath / "scripts" / "samples" * "*"
val libs = sample.managedClasspath(Configurations.Runtime)
val deployed = sample.jarPath
val sources = sample.packageSourcePaths
copyFiles(confs, configPath) orElse
copyScripts(scripts, binPath) orElse
copyFiles(libs, libPath) orElse
copyFiles(deployed, deployPath) orElse
copyPaths(sources, srcPath)
}.foldLeft(None: Option[String])(_ orElse _)
}
// def copySamples[P <: DefaultProject](samples: Set[P]) = {
// samples.map { sample =>
// val sampleOutputPath = distSamplesPath / sample.name
// val binPath = sampleOutputPath / "bin"
// val configPath = sampleOutputPath / "config"
// val deployPath = sampleOutputPath / "deploy"
// val libPath = sampleOutputPath / "lib"
// val srcPath = sampleOutputPath / "src"
// val confs = sample.info.projectPath / "config" ** "*.*"
// val scripts = akkaParent.info.projectPath / "scripts" / "samples" * "*"
// val libs = sample.managedClasspath(Configurations.Runtime)
// val deployed = sample.jarPath
// val sources = sample.packageSourcePaths
// copyFiles(confs, configPath) orElse
// copyScripts(scripts, binPath) orElse
// copyFiles(libs, libPath) orElse
// copyFiles(deployed, deployPath) orElse
// copyPaths(sources, srcPath)
// }.foldLeft(None: Option[String])(_ orElse _)
// }
copyFiles(demo, distDeployPath) orElse
copySamples(samples)
} dependsOn (distBase)
// copyFiles(demo, distDeployPath) orElse
// copySamples(samples)
// } dependsOn (distBase)
}
}
}

View file

@ -1,6 +1,6 @@
@echo off
set AKKA_HOME=%~dp0..
set JAVA_OPTS=-Xms1024M -Xmx1024M -Xss1M -XX:MaxPermSize=256M -XX:+UseParallelGC
set AKKA_CLASSPATH=%AKKA_HOME%\lib\scala-library.jar;%AKKA_HOME%\lib\akka\*;%AKKA_HOME%\config
set AKKA_CLASSPATH=%AKKA_HOME%\lib\scala-library.jar;%AKKA_HOME%\config;%AKKA_HOME%\lib\akka\*
java %JAVA_OPTS% -cp "%AKKA_CLASSPATH%" -Dakka.home="%AKKA_HOME%" akka.kernel.Main