Merged wip-2.0 branch with latest master

This commit is contained in:
Jonas Bonér 2011-05-16 12:32:00 +02:00
commit 2655d44ee9
207 changed files with 6049 additions and 4773 deletions

3
.gitignore vendored
View file

@ -13,7 +13,6 @@ TAGS
akka.tmproj akka.tmproj
reports reports
dist dist
build
target target
deploy/*.jar deploy/*.jar
data data
@ -52,4 +51,4 @@ akka-tutorials/akka-tutorial-first/project/plugins/project/
akka-docs/exts/ akka-docs/exts/
_akka_cluster/ _akka_cluster/
Makefile Makefile
akka.sublime-project akka.sublime-project

View file

@ -4,13 +4,13 @@ import org.junit.Test;
import static org.junit.Assert.*; import static org.junit.Assert.*;
import java.util.concurrent.Callable; import java.util.concurrent.Callable;
import java.util.LinkedList; import java.util.LinkedList;
import java.lang.Iterable;
import akka.japi.Function; import akka.japi.Function;
import akka.japi.Function2;
import akka.japi.Procedure; import akka.japi.Procedure;
import scala.Some; import scala.Some;
import scala.Right; import scala.Right;
import static akka.dispatch.Futures.future; import static akka.dispatch.Futures.*;
import static akka.dispatch.Futures.traverse;
import static akka.dispatch.Futures.sequence;
public class JavaFutureTests { public class JavaFutureTests {
@ -44,9 +44,75 @@ public class JavaFutureTests {
})); }));
} }
Future<LinkedList<String>> futureList = sequence(listFutures); Future<Iterable<String>> futureList = sequence(listFutures);
assertEquals(futureList.get(), listExpected); assertEquals(futureList.get(), listExpected);
} }
// TODO: Improve this test, perhaps with an Actor
@Test public void foldForJavaApiMustWork() {
LinkedList<Future<String>> listFutures = new LinkedList<Future<String>>();
StringBuilder expected = new StringBuilder();
for (int i = 0; i < 10; i++) {
expected.append("test");
listFutures.add(future(new Callable<String>() {
public String call() {
return "test";
}
}));
}
Future<String> result = fold("", 15000,listFutures, new Function2<String,String,String>(){
public String apply(String r, String t) {
return r + t;
}
});
assertEquals(result.get(), expected.toString());
}
@Test public void reduceForJavaApiMustWork() {
LinkedList<Future<String>> listFutures = new LinkedList<Future<String>>();
StringBuilder expected = new StringBuilder();
for (int i = 0; i < 10; i++) {
expected.append("test");
listFutures.add(future(new Callable<String>() {
public String call() {
return "test";
}
}));
}
Future<String> result = reduce(listFutures, 15000, new Function2<String,String,String>(){
public String apply(String r, String t) {
return r + t;
}
});
assertEquals(result.get(), expected.toString());
}
@Test public void traverseForJavaApiMustWork() {
LinkedList<String> listStrings = new LinkedList<String>();
LinkedList<String> expectedStrings = new LinkedList<String>();
for (int i = 0; i < 10; i++) {
expectedStrings.add("TEST");
listStrings.add("test");
}
Future<Iterable<String>> result = traverse(listStrings, new Function<String,Future<String>>(){
public Future<String> apply(final String r) {
return future(new Callable<String>() {
public String call() {
return r.toUpperCase();
}
});
}
});
assertEquals(result.get(), expectedStrings);
}
} }

View file

@ -78,7 +78,7 @@ object Chameneos {
var sumMeetings = 0 var sumMeetings = 0
var numFaded = 0 var numFaded = 0
override def preStart = { override def preStart() = {
for (i <- 0 until numChameneos) actorOf(new Chameneo(self, colours(i % 3), i)) for (i <- 0 until numChameneos) actorOf(new Chameneo(self, colours(i % 3), i))
} }

View file

@ -46,7 +46,7 @@ class RestartStrategySpec extends JUnitSuite {
secondRestartLatch.open secondRestartLatch.open
} }
override def postStop = { override def postStop() = {
stopLatch.open stopLatch.open
} }
}) })
@ -131,7 +131,7 @@ class RestartStrategySpec extends JUnitSuite {
thirdRestartLatch.open thirdRestartLatch.open
} }
override def postStop = { override def postStop() = {
if (restartLatch.isOpen) { if (restartLatch.isOpen) {
secondRestartLatch.open secondRestartLatch.open
} }
@ -189,7 +189,7 @@ class RestartStrategySpec extends JUnitSuite {
secondRestartLatch.open secondRestartLatch.open
} }
override def postStop = { override def postStop() = {
stopLatch.open stopLatch.open
} }
}) })
@ -243,7 +243,7 @@ class RestartStrategySpec extends JUnitSuite {
restartLatch.open restartLatch.open
} }
override def postStop = { override def postStop() = {
stopLatch.open stopLatch.open
} }
}) })

View file

@ -385,7 +385,7 @@ class SupervisorSpec extends WordSpec with MustMatchers with BeforeAndAfterEach
inits.get must be (3) inits.get must be (3)
supervisor.shutdown supervisor.shutdown()
} }
} }
} }

View file

@ -65,7 +65,7 @@ object Ticket669Spec {
self.reply_?("failure1") self.reply_?("failure1")
} }
override def postStop { override def postStop() {
self.reply_?("failure2") self.reply_?("failure2")
} }
} }

View file

@ -18,7 +18,7 @@ class ConfigSpec extends WordSpec with MustMatchers {
getList("akka.boot") must equal(Nil) getList("akka.boot") must equal(Nil)
getString("akka.time-unit") must equal(Some("seconds")) getString("akka.time-unit") must equal(Some("seconds"))
getString("akka.version") must equal(Some("1.1-SNAPSHOT")) getString("akka.version") must equal(Some("1.2-SNAPSHOT"))
getString("akka.actor.default-dispatcher.type") must equal(Some("GlobalExecutorBasedEventDriven")) getString("akka.actor.default-dispatcher.type") must equal(Some("GlobalExecutorBasedEventDriven"))
getInt("akka.actor.default-dispatcher.keep-alive-time") must equal(Some(60)) getInt("akka.actor.default-dispatcher.keep-alive-time") must equal(Some(60))

View file

@ -0,0 +1,165 @@
/**
* Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.dataflow
import org.scalatest.Spec
import org.scalatest.Assertions
import org.scalatest.matchers.ShouldMatchers
import org.scalatest.BeforeAndAfterAll
import org.scalatest.junit.JUnitRunner
import org.junit.runner.RunWith
import akka.dispatch.DefaultCompletableFuture
import java.util.concurrent.{TimeUnit, CountDownLatch}
import annotation.tailrec
import java.util.concurrent.atomic.{AtomicLong, AtomicReference, AtomicInteger}
import akka.actor.ActorRegistry
@RunWith(classOf[JUnitRunner])
class DataFlowTest extends Spec with ShouldMatchers with BeforeAndAfterAll {
describe("DataflowVariable") {
it("should be able to set the value of one variable from other variables") {
import DataFlow._
val latch = new CountDownLatch(1)
val result = new AtomicInteger(0)
val x, y, z = new DataFlowVariable[Int]
thread {
z << x() + y()
result.set(z())
latch.countDown()
}
thread { x << 40 }
thread { y << 2 }
latch.await(10,TimeUnit.SECONDS) should equal (true)
result.get should equal (42)
List(x,y,z).foreach(_.shutdown())
}
it("should be able to sum a sequence of ints") {
import DataFlow._
def ints(n: Int, max: Int): List[Int] =
if (n == max) Nil
else n :: ints(n + 1, max)
def sum(s: Int, stream: List[Int]): List[Int] = stream match {
case Nil => s :: Nil
case h :: t => s :: sum(h + s, t)
}
val latch = new CountDownLatch(1)
val result = new AtomicReference[List[Int]](Nil)
val x = new DataFlowVariable[List[Int]]
val y = new DataFlowVariable[List[Int]]
val z = new DataFlowVariable[List[Int]]
thread { x << ints(0, 1000) }
thread { y << sum(0, x()) }
thread { z << y()
result.set(z())
latch.countDown()
}
latch.await(10,TimeUnit.SECONDS) should equal (true)
result.get should equal (sum(0,ints(0,1000)))
List(x,y,z).foreach(_.shutdown())
}
/*
it("should be able to join streams") {
import DataFlow._
Actor.registry.shutdownAll()
def ints(n: Int, max: Int, stream: DataFlowStream[Int]): Unit = if (n != max) {
stream <<< n
ints(n + 1, max, stream)
}
def sum(s: Int, in: DataFlowStream[Int], out: DataFlowStream[Int]): Unit = {
out <<< s
sum(in() + s, in, out)
}
val producer = new DataFlowStream[Int]
val consumer = new DataFlowStream[Int]
val latch = new CountDownLatch(1)
val result = new AtomicInteger(0)
val t1 = thread { ints(0, 1000, producer) }
val t2 = thread {
Thread.sleep(1000)
result.set(producer.map(x => x * x).foldLeft(0)(_ + _))
latch.countDown()
}
latch.await(3,TimeUnit.SECONDS) should equal (true)
result.get should equal (332833500)
}
it("should be able to sum streams recursively") {
import DataFlow._
def ints(n: Int, max: Int, stream: DataFlowStream[Int]): Unit = if (n != max) {
stream <<< n
ints(n + 1, max, stream)
}
def sum(s: Int, in: DataFlowStream[Int], out: DataFlowStream[Int]): Unit = {
out <<< s
sum(in() + s, in, out)
}
val result = new AtomicLong(0)
val producer = new DataFlowStream[Int]
val consumer = new DataFlowStream[Int]
val latch = new CountDownLatch(1)
@tailrec def recurseSum(stream: DataFlowStream[Int]): Unit = {
val x = stream()
if(result.addAndGet(x) == 166666500)
latch.countDown()
recurseSum(stream)
}
thread { ints(0, 1000, producer) }
thread { sum(0, producer, consumer) }
thread { recurseSum(consumer) }
latch.await(15,TimeUnit.SECONDS) should equal (true)
}
*/
/* Test not ready for prime time, causes some sort of deadlock */
/* it("should be able to conditionally set variables") {
import DataFlow._
Actor.registry.shutdownAll()
val latch = new CountDownLatch(1)
val x, y, z, v = new DataFlowVariable[Int]
val main = thread {
x << 1
z << Math.max(x(),y())
latch.countDown()
}
val setY = thread {
// Thread.sleep(2000)
y << 2
}
val setV = thread {
v << y
}
List(x,y,z,v) foreach (_.shutdown())
latch.await(2,TimeUnit.SECONDS) should equal (true)
}*/
}
}

View file

@ -14,6 +14,7 @@ import java.util.concurrent.atomic.AtomicLong
import java.util.concurrent. {ConcurrentHashMap, CountDownLatch, TimeUnit} import java.util.concurrent. {ConcurrentHashMap, CountDownLatch, TimeUnit}
import akka.actor.dispatch.ActorModelSpec.MessageDispatcherInterceptor import akka.actor.dispatch.ActorModelSpec.MessageDispatcherInterceptor
import akka.util.{Duration, Switch} import akka.util.{Duration, Switch}
import org.multiverse.api.latches.StandardLatch
object ActorModelSpec { object ActorModelSpec {
@ -110,13 +111,13 @@ object ActorModelSpec {
super.dispatch(invocation) super.dispatch(invocation)
} }
private[akka] abstract override def start { private[akka] abstract override def start() {
super.start super.start()
starts.incrementAndGet() starts.incrementAndGet()
} }
private[akka] abstract override def shutdown { private[akka] abstract override def shutdown() {
super.shutdown super.shutdown()
stops.incrementAndGet() stops.incrementAndGet()
} }
} }
@ -216,6 +217,21 @@ abstract class ActorModelSpec extends JUnitSuite {
msgsProcessed = 0, msgsProcessed = 0,
restarts = 0 restarts = 0
) )
val futures = for(i <- 1 to 10) yield Future { i }
await(dispatcher.stops.get == 2)(withinMs = dispatcher.timeoutMs * 5)
assertDispatcher(dispatcher)(starts = 2, stops = 2)
val a2 = newTestActor
a2.start
val futures2 = for(i <- 1 to 10) yield Future { i }
await(dispatcher.starts.get == 3)(withinMs = dispatcher.timeoutMs * 5)
assertDispatcher(dispatcher)(starts = 3, stops = 2)
a2.stop
await(dispatcher.stops.get == 3)(withinMs = dispatcher.timeoutMs * 5)
assertDispatcher(dispatcher)(starts = 3, stops = 3)
} }
@Test def dispatcherShouldProcessMessagesOneAtATime { @Test def dispatcherShouldProcessMessagesOneAtATime {

View file

@ -62,9 +62,9 @@ class FutureSpec extends JUnitSuite {
val future1 = actor1 !!! "Hello" flatMap ((s: String) => actor2 !!! s) val future1 = actor1 !!! "Hello" flatMap ((s: String) => actor2 !!! s)
val future2 = actor1 !!! "Hello" flatMap (actor2 !!! (_: String)) val future2 = actor1 !!! "Hello" flatMap (actor2 !!! (_: String))
val future3 = actor1 !!! "Hello" flatMap (actor2 !!! (_: Int)) val future3 = actor1 !!! "Hello" flatMap (actor2 !!! (_: Int))
assert(Some(Right("WORLD")) === future1.await.value) assert((future1.get: Any) === "WORLD")
assert(Some(Right("WORLD")) === future2.await.value) assert((future2.get: Any) === "WORLD")
intercept[ClassCastException] { future3.await.resultOrException } intercept[ClassCastException] { future3.get }
actor1.stop() actor1.stop()
actor2.stop() actor2.stop()
} }
@ -74,8 +74,8 @@ class FutureSpec extends JUnitSuite {
val actor2 = actorOf(new Actor { def receive = { case s: String => self reply s.toUpperCase } } ).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 future1 = actor1 !!! "Hello" collect { case (s: String) => s } flatMap (actor2 !!! _)
val future2 = actor1 !!! "Hello" collect { case (n: Int) => n } flatMap (actor2 !!! _) val future2 = actor1 !!! "Hello" collect { case (n: Int) => n } flatMap (actor2 !!! _)
assert(Some(Right("WORLD")) === future1.await.value) assert((future1.get: Any) === "WORLD")
intercept[MatchError] { future2.await.resultOrException } intercept[MatchError] { future2.get }
actor1.stop() actor1.stop()
actor2.stop() actor2.stop()
} }
@ -102,8 +102,8 @@ class FutureSpec extends JUnitSuite {
c: String <- actor !!! 7 c: String <- actor !!! 7
} yield b + "-" + c } yield b + "-" + c
assert(Some(Right("10-14")) === future1.await.value) assert(future1.get === "10-14")
intercept[ClassCastException] { future2.await.resultOrException } intercept[ClassCastException] { future2.get }
actor.stop() actor.stop()
} }
@ -118,19 +118,64 @@ class FutureSpec extends JUnitSuite {
}).start() }).start()
val future1 = for { val future1 = for {
a <- actor !!! Req("Hello") collect { case Res(x: Int) => x } Res(a: Int) <- actor !!! Req("Hello")
b <- actor !!! Req(a) collect { case Res(x: String) => x } Res(b: String) <- actor !!! Req(a)
c <- actor !!! Req(7) collect { case Res(x: String) => x } Res(c: String) <- actor !!! Req(7)
} yield b + "-" + c } yield b + "-" + c
val future2 = for { val future2 = for {
a <- actor !!! Req("Hello") collect { case Res(x: Int) => x } Res(a: Int) <- actor !!! Req("Hello")
b <- actor !!! Req(a) collect { case Res(x: Int) => x } Res(b: Int) <- actor !!! Req(a)
c <- actor !!! Req(7) collect { case Res(x: String) => x } Res(c: Int) <- actor !!! Req(7)
} yield b + "-" + c } yield b + "-" + c
assert(Some(Right("10-14")) === future1.await.value) assert(future1.get === "10-14")
intercept[MatchError] { future2.await.resultOrException } intercept[MatchError] { future2.get }
actor.stop()
}
@Test def shouldMapMatchedExceptionsToResult {
val future1 = Future(5)
val future2 = future1 map (_ / 0)
val future3 = future2 map (_.toString)
val future4 = future1 failure {
case e: ArithmeticException => 0
} map (_.toString)
val future5 = future2 failure {
case e: ArithmeticException => 0
} map (_.toString)
val future6 = future2 failure {
case e: MatchError => 0
} map (_.toString)
val future7 = future3 failure { case e: ArithmeticException => "You got ERROR" }
val actor = actorOf[TestActor].start()
val future8 = actor !!! "Failure"
val future9 = actor !!! "Failure" failure {
case e: RuntimeException => "FAIL!"
}
val future10 = actor !!! "Hello" failure {
case e: RuntimeException => "FAIL!"
}
val future11 = actor !!! "Failure" failure { case _ => "Oops!" }
assert(future1.get === 5)
intercept[ArithmeticException] { future2.get }
intercept[ArithmeticException] { future3.get }
assert(future4.get === "5")
assert(future5.get === "0")
intercept[ArithmeticException] { future6.get }
assert(future7.get === "You got ERROR")
intercept[RuntimeException] { future8.get }
assert(future9.get === "FAIL!")
assert(future10.get === "World")
assert(future11.get === "Oops!")
actor.stop() actor.stop()
} }
@ -140,8 +185,9 @@ class FutureSpec extends JUnitSuite {
def receive = { case (add: Int, wait: Int) => Thread.sleep(wait); self reply_? add } def receive = { case (add: Int, wait: Int) => Thread.sleep(wait); self reply_? add }
}).start() }).start()
} }
def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) => actor.!!![Int]((idx, idx * 200 )) } val timeout = 10000
assert(Futures.fold(0)(futures)(_ + _).awaitBlocking.result.get === 45) def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) => actor.!!![Int]((idx, idx * 200 ), timeout) }
assert(Futures.fold(0, timeout)(futures)(_ + _).await.result.get === 45)
} }
@Test def shouldFoldResultsByComposing { @Test def shouldFoldResultsByComposing {
@ -150,8 +196,8 @@ class FutureSpec extends JUnitSuite {
def receive = { case (add: Int, wait: Int) => Thread.sleep(wait); self reply_? add } def receive = { case (add: Int, wait: Int) => Thread.sleep(wait); self reply_? add }
}).start() }).start()
} }
def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) => actor.!!![Int]((idx, idx * 200 )) } def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) => actor.!!![Int]((idx, idx * 200 ), 10000) }
assert(futures.foldLeft(Future(0))((fr, fa) => for (r <- fr; a <- fa) yield (r + a)).awaitBlocking.result.get === 45) assert(futures.foldLeft(Future(0))((fr, fa) => for (r <- fr; a <- fa) yield (r + a)).get === 45)
} }
@Test def shouldFoldResultsWithException { @Test def shouldFoldResultsWithException {
@ -165,12 +211,13 @@ class FutureSpec extends JUnitSuite {
} }
}).start() }).start()
} }
def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) => actor.!!![Int]((idx, idx * 100 )) } val timeout = 10000
assert(Futures.fold(0)(futures)(_ + _).awaitBlocking.exception.get.getMessage === "shouldFoldResultsWithException: expected") def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) => actor.!!![Int]((idx, idx * 100 ), timeout) }
assert(Futures.fold(0, timeout)(futures)(_ + _).await.exception.get.getMessage === "shouldFoldResultsWithException: expected")
} }
@Test def shouldFoldReturnZeroOnEmptyInput { @Test def shouldFoldReturnZeroOnEmptyInput {
assert(Futures.fold(0)(List[Future[Int]]())(_ + _).awaitBlocking.result.get === 0) assert(Futures.fold(0)(List[Future[Int]]())(_ + _).get === 0)
} }
@Test def shouldReduceResults { @Test def shouldReduceResults {
@ -179,8 +226,9 @@ class FutureSpec extends JUnitSuite {
def receive = { case (add: Int, wait: Int) => Thread.sleep(wait); self reply_? add } def receive = { case (add: Int, wait: Int) => Thread.sleep(wait); self reply_? add }
}).start() }).start()
} }
def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) => actor.!!![Int]((idx, idx * 200 )) } val timeout = 10000
assert(Futures.reduce(futures)(_ + _).awaitBlocking.result.get === 45) def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) => actor.!!![Int]((idx, idx * 200 ), timeout) }
assert(Futures.reduce(futures, timeout)(_ + _).get === 45)
} }
@Test def shouldReduceResultsWithException { @Test def shouldReduceResultsWithException {
@ -194,34 +242,15 @@ class FutureSpec extends JUnitSuite {
} }
}).start() }).start()
} }
def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) => actor.!!![Int]((idx, idx * 100 )) } val timeout = 10000
assert(Futures.reduce(futures)(_ + _).awaitBlocking.exception.get.getMessage === "shouldFoldResultsWithException: expected") def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) => actor.!!![Int]((idx, idx * 100 ), timeout) }
assert(Futures.reduce(futures, timeout)(_ + _).await.exception.get.getMessage === "shouldFoldResultsWithException: expected")
} }
@Test(expected = classOf[UnsupportedOperationException]) def shouldReduceThrowIAEOnEmptyInput { @Test(expected = classOf[UnsupportedOperationException]) def shouldReduceThrowIAEOnEmptyInput {
Futures.reduce(List[Future[Int]]())(_ + _).await.resultOrException Futures.reduce(List[Future[Int]]())(_ + _).await.resultOrException
} }
@Test def resultWithinShouldNotThrowExceptions {
val latch = new StandardLatch
val actors = (1 to 10).toList map { _ =>
actorOf(new Actor {
def receive = { case (add: Int, wait: Boolean, latch: StandardLatch) => if (wait) latch.await; self reply_? add }
}).start()
}
def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) => actor.!!![Int]((idx, idx >= 5, latch)) }
val result = for(f <- futures) yield f.valueWithin(2, TimeUnit.SECONDS)
latch.open
val done = result collect { case Some(Right(x)) => x }
val undone = result collect { case None => None }
val errors = result collect { case Some(Left(t)) => t }
assert(done.size === 5)
assert(undone.size === 5)
assert(errors.size === 0)
}
@Test def receiveShouldExecuteOnComplete { @Test def receiveShouldExecuteOnComplete {
val latch = new StandardLatch val latch = new StandardLatch
val actor = actorOf[TestActor].start() val actor = actorOf[TestActor].start()
@ -269,41 +298,303 @@ class FutureSpec extends JUnitSuite {
assert(f3.resultOrException === Some("SUCCESS")) assert(f3.resultOrException === Some("SUCCESS"))
// make sure all futures are completed in dispatcher // make sure all futures are completed in dispatcher
assert(Dispatchers.defaultGlobalDispatcher.futureQueueSize === 0) assert(Dispatchers.defaultGlobalDispatcher.pendingFutures === 0)
} }
@Test def shouldBlockUntilResult { @Test def shouldBlockUntilResult {
val latch = new StandardLatch val latch = new StandardLatch
val f = Future({ latch.await; 5}) val f = Future({ latch.await; 5})
val f2 = Future({ f() + 5 }) val f2 = Future({ f.get + 5 })
assert(f2.resultOrException === None) assert(f2.resultOrException === None)
latch.open latch.open
assert(f2() === 10) assert(f2.get === 10)
val f3 = Future({ Thread.sleep(100); 5}, 10) val f3 = Future({ Thread.sleep(100); 5}, 10)
intercept[FutureTimeoutException] { intercept[FutureTimeoutException] {
f3() f3.get
} }
} }
@Test def lesslessIsMore { @Test def futureComposingWithContinuations {
import akka.actor.Actor.spawn import Future.flow
val dataflowVar, dataflowVar2 = new DefaultCompletableFuture[Int](Long.MaxValue)
val begin, end = new StandardLatch val actor = actorOf[TestActor].start
spawn {
begin.await val x = Future("Hello")
dataflowVar2 << dataflowVar val y = x flatMap (actor !!! _)
end.open
val r = flow(x() + " " + y[String]() + "!")
assert(r.get === "Hello World!")
actor.stop
}
@Test def futureComposingWithContinuationsFailureDivideZero {
import Future.flow
val x = Future("Hello")
val y = x map (_.length)
val r = flow(x() + " " + y.map(_ / 0).map(_.toString)(), 100)
intercept[java.lang.ArithmeticException](r.get)
}
@Test def futureComposingWithContinuationsFailureCastInt {
import Future.flow
val actor = actorOf[TestActor].start
val x = Future(3)
val y = actor !!! "Hello"
val r = flow(x() + y[Int](), 100)
intercept[ClassCastException](r.get)
}
@Test def futureComposingWithContinuationsFailureCastNothing {
import Future.flow
val actor = actorOf[TestActor].start
val x = Future("Hello")
val y = actor !!! "Hello"
val r = flow(x() + y())
intercept[ClassCastException](r.get)
}
@Test def futureCompletingWithContinuations {
import Future.flow
val x, y, z = Promise[Int]()
val ly, lz = new StandardLatch
val result = flow {
y completeWith x
ly.open // not within continuation
z << x
lz.open // within continuation, will wait for 'z' to complete
z() + y()
} }
spawn { assert(ly.tryAwaitUninterruptible(100, TimeUnit.MILLISECONDS))
dataflowVar << 5 assert(!lz.tryAwaitUninterruptible(100, TimeUnit.MILLISECONDS))
flow { x << 5 }
assert(y.get === 5)
assert(z.get === 5)
assert(lz.isOpen)
assert(result.get === 10)
val a, b, c = Promise[Int]()
val result2 = flow {
val n = (a << c).result.get + 10
b << (c() - 2)
a() + n * b()
} }
begin.open
end.await c completeWith Future(5)
assert(dataflowVar2() === 5)
assert(dataflowVar.get === 5) assert(a.get === 5)
assert(b.get === 3)
assert(result2.get === 50)
Thread.sleep(100)
// make sure all futures are completed in dispatcher
assert(Dispatchers.defaultGlobalDispatcher.pendingFutures === 0)
}
@Test def shouldNotAddOrRunCallbacksAfterFailureToBeCompletedBeforeExpiry {
val latch = new StandardLatch
val f = Promise[Int](0)
Thread.sleep(25)
f.onComplete( _ => latch.open ) //Shouldn't throw any exception here
assert(f.isExpired) //Should be expired
f.complete(Right(1)) //Shouldn't complete the Future since it is expired
assert(f.value.isEmpty) //Shouldn't be completed
assert(!latch.isOpen) //Shouldn't run the listener
}
@Test def futureDataFlowShouldEmulateBlocking1 {
import Future.flow
val one, two = Promise[Int](1000 * 60)
val simpleResult = flow {
one() + two()
}
assert(List(one, two, simpleResult).forall(_.isCompleted == false))
flow { one << 1 }
assert(one.isCompleted)
assert(List(two, simpleResult).forall(_.isCompleted == false))
flow { two << 9 }
assert(List(one, two).forall(_.isCompleted == true))
assert(simpleResult.get === 10)
}
@Test def futureDataFlowShouldEmulateBlocking2 {
import Future.flow
val x1, x2, y1, y2 = Promise[Int](1000 * 60)
val lx, ly, lz = new StandardLatch
val result = flow {
lx.open()
x1 << y1
ly.open()
x2 << y2
lz.open()
x1() + x2()
}
assert(lx.isOpen)
assert(!ly.isOpen)
assert(!lz.isOpen)
assert(List(x1,x2,y1,y2).forall(_.isCompleted == false))
flow { y1 << 1 } // When this is set, it should cascade down the line
assert(ly.tryAwaitUninterruptible(2000, TimeUnit.MILLISECONDS))
assert(x1.get === 1)
assert(!lz.isOpen)
flow { y2 << 9 } // When this is set, it should cascade down the line
assert(lz.tryAwaitUninterruptible(2000, TimeUnit.MILLISECONDS))
assert(x2.get === 9)
assert(List(x1,x2,y1,y2).forall(_.isCompleted == true))
assert(result.get === 10)
}
@Test def dataFlowAPIshouldbeSlick {
import Future.flow
val i1, i2, s1, s2 = new StandardLatch
val callService1 = Future { i1.open; s1.awaitUninterruptible; 1 }
val callService2 = Future { i2.open; s2.awaitUninterruptible; 9 }
val result = flow { callService1() + callService2() }
assert(!s1.isOpen)
assert(!s2.isOpen)
assert(!result.isCompleted)
assert(i1.tryAwaitUninterruptible(2000, TimeUnit.MILLISECONDS))
assert(i2.tryAwaitUninterruptible(2000, TimeUnit.MILLISECONDS))
s1.open
s2.open
assert(result.get === 10)
}
@Test def futureCompletingWithContinuationsFailure {
import Future.flow
val x, y, z = Promise[Int]()
val ly, lz = new StandardLatch
val result = flow {
y << x
ly.open
val oops = 1 / 0
z << x
lz.open
z() + y() + oops
}
assert(!ly.tryAwaitUninterruptible(100, TimeUnit.MILLISECONDS))
assert(!lz.tryAwaitUninterruptible(100, TimeUnit.MILLISECONDS))
flow { x << 5 }
assert(y.get === 5)
intercept[java.lang.ArithmeticException](result.get)
assert(z.value === None)
assert(!lz.isOpen)
}
@Test def futureContinuationsShouldNotBlock {
import Future.flow
val latch = new StandardLatch
val future = Future {
latch.await
"Hello"
}
val result = flow {
Some(future()).filter(_ == "Hello")
}
assert(!result.isCompleted)
latch.open
assert(result.get === Some("Hello"))
}
@Test def futureFlowShouldBeTypeSafe {
import Future.flow
def checkType[A: Manifest, B](in: Future[A], refmanifest: Manifest[B]): Boolean = manifest[A] == refmanifest
val rString = flow {
val x = Future(5)
x().toString
}
val rInt = flow {
val x = rString.apply
val y = Future(5)
x.length + y()
}
assert(checkType(rString, manifest[String]))
assert(checkType(rInt, manifest[Int]))
assert(!checkType(rInt, manifest[String]))
assert(!checkType(rInt, manifest[Nothing]))
assert(!checkType(rInt, manifest[Any]))
rString.await
rInt.await
}
@Test def futureFlowSimpleAssign {
import Future.flow
val x, y, z = Promise[Int]()
flow {
z << x() + y()
}
flow { x << 40 }
flow { y << 2 }
assert(z.get === 42)
}
@Test def ticket812FutureDispatchCleanup {
val dispatcher = implicitly[MessageDispatcher]
assert(dispatcher.pendingFutures === 0)
val future = Future({Thread.sleep(100);"Done"}, 10)
intercept[FutureTimeoutException] { future.await }
assert(dispatcher.pendingFutures === 1)
Thread.sleep(100)
assert(dispatcher.pendingFutures === 0)
} }
} }

View file

@ -23,8 +23,8 @@ abstract class MailboxSpec extends
def factory: MailboxType => MessageQueue def factory: MailboxType => MessageQueue
name should { name should {
"create a !blockDequeue && unbounded mailbox" in { "create an unbounded mailbox" in {
val config = UnboundedMailbox(false) val config = UnboundedMailbox()
val q = factory(config) val q = factory(config)
ensureInitialMailboxState(config, q) ensureInitialMailboxState(config, q)
@ -37,8 +37,8 @@ abstract class MailboxSpec extends
f.await.resultOrException must be === Some(null) f.await.resultOrException must be === Some(null)
} }
"create a !blockDequeue and bounded mailbox with 10 capacity and with push timeout" in { "create a bounded mailbox with 10 capacity and with push timeout" in {
val config = BoundedMailbox(false, 10, Duration(10,TimeUnit.MILLISECONDS)) val config = BoundedMailbox(10, Duration(10,TimeUnit.MILLISECONDS))
val q = factory(config) val q = factory(config)
ensureInitialMailboxState(config, q) ensureInitialMailboxState(config, q)
@ -59,30 +59,16 @@ abstract class MailboxSpec extends
} }
"dequeue what was enqueued properly for unbounded mailboxes" in { "dequeue what was enqueued properly for unbounded mailboxes" in {
testEnqueueDequeue(UnboundedMailbox(false)) testEnqueueDequeue(UnboundedMailbox())
} }
"dequeue what was enqueued properly for bounded mailboxes" in { "dequeue what was enqueued properly for bounded mailboxes" in {
testEnqueueDequeue(BoundedMailbox(false, 10000, Duration(-1, TimeUnit.MILLISECONDS))) testEnqueueDequeue(BoundedMailbox(10000, Duration(-1, TimeUnit.MILLISECONDS)))
} }
"dequeue what was enqueued properly for bounded mailboxes with pushTimeout" in { "dequeue what was enqueued properly for bounded mailboxes with pushTimeout" in {
testEnqueueDequeue(BoundedMailbox(false, 10000, Duration(100, TimeUnit.MILLISECONDS))) testEnqueueDequeue(BoundedMailbox(10000, Duration(100, TimeUnit.MILLISECONDS)))
} }
/** FIXME Adapt test so it works with the last dequeue
"dequeue what was enqueued properly for unbounded mailboxes with blockDeque" in {
testEnqueueDequeue(UnboundedMailbox(true))
}
"dequeue what was enqueued properly for bounded mailboxes with blockDeque" in {
testEnqueueDequeue(BoundedMailbox(true, 1000, Duration(-1, TimeUnit.MILLISECONDS)))
}
"dequeue what was enqueued properly for bounded mailboxes with blockDeque and pushTimeout" in {
testEnqueueDequeue(BoundedMailbox(true, 1000, Duration(100, TimeUnit.MILLISECONDS)))
}*/
} }
//CANDIDATE FOR TESTKIT //CANDIDATE FOR TESTKIT
@ -111,8 +97,8 @@ abstract class MailboxSpec extends
q match { q match {
case aQueue: BlockingQueue[_] => case aQueue: BlockingQueue[_] =>
config match { config match {
case BoundedMailbox(_,capacity,_) => aQueue.remainingCapacity must be === capacity case BoundedMailbox(capacity,_) => aQueue.remainingCapacity must be === capacity
case UnboundedMailbox(_) => aQueue.remainingCapacity must be === Int.MaxValue case UnboundedMailbox() => aQueue.remainingCapacity must be === Int.MaxValue
} }
case _ => case _ =>
} }
@ -165,10 +151,8 @@ abstract class MailboxSpec extends
class DefaultMailboxSpec extends MailboxSpec { class DefaultMailboxSpec extends MailboxSpec {
lazy val name = "The default mailbox implementation" lazy val name = "The default mailbox implementation"
def factory = { def factory = {
case UnboundedMailbox(blockDequeue) => case UnboundedMailbox() => new DefaultUnboundedMessageQueue()
new DefaultUnboundedMessageQueue(blockDequeue) case BoundedMailbox(capacity, pushTimeOut) => new DefaultBoundedMessageQueue(capacity, pushTimeOut)
case BoundedMailbox(blocking, capacity, pushTimeOut) =>
new DefaultBoundedMessageQueue(capacity, pushTimeOut, blocking)
} }
} }
@ -176,9 +160,7 @@ class PriorityMailboxSpec extends MailboxSpec {
val comparator = PriorityGenerator(_.##) val comparator = PriorityGenerator(_.##)
lazy val name = "The priority mailbox implementation" lazy val name = "The priority mailbox implementation"
def factory = { def factory = {
case UnboundedMailbox(blockDequeue) => case UnboundedMailbox() => new UnboundedPriorityMessageQueue(comparator)
new UnboundedPriorityMessageQueue(blockDequeue, comparator) case BoundedMailbox(capacity, pushTimeOut) => new BoundedPriorityMessageQueue(capacity, pushTimeOut, comparator)
case BoundedMailbox(blocking, capacity, pushTimeOut) =>
new BoundedPriorityMessageQueue(capacity, pushTimeOut, blocking, comparator)
} }
} }

View file

@ -10,11 +10,11 @@ class PriorityDispatcherSpec extends WordSpec with MustMatchers {
"A PriorityExecutorBasedEventDrivenDispatcher" must { "A PriorityExecutorBasedEventDrivenDispatcher" must {
"Order it's messages according to the specified comparator using an unbounded mailbox" in { "Order it's messages according to the specified comparator using an unbounded mailbox" in {
testOrdering(UnboundedMailbox(false)) testOrdering(UnboundedMailbox())
} }
"Order it's messages according to the specified comparator using a bounded mailbox" in { "Order it's messages according to the specified comparator using a bounded mailbox" in {
testOrdering(BoundedMailbox(false,1000)) testOrdering(BoundedMailbox(1000))
} }
} }

View file

@ -16,14 +16,11 @@ import java.net.{InetAddress, UnknownHostException}
* *
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
class AkkaException(message: String = "") extends RuntimeException(message) with Serializable { class AkkaException(message: String = "", cause: Throwable = null) extends RuntimeException(message, cause) with Serializable {
val uuid = "%s_%s".format(AkkaException.hostname, newUuid) val uuid = "%s_%s".format(AkkaException.hostname, newUuid)
override lazy val toString = { override lazy val toString =
val name = getClass.getName "%s: %s\n[%s]\n%s".format(getClass.getName, message, uuid, stackTraceToString)
val trace = stackTraceToString
"%s: %s\n[%s]\n%s".format(name, message, uuid, trace)
}
def stackTraceToString = { def stackTraceToString = {
val trace = getStackTrace val trace = getStackTrace

View file

@ -17,6 +17,7 @@ import akka.event.EventHandler
import scala.reflect.BeanProperty import scala.reflect.BeanProperty
import com.eaio.uuid.UUID import com.eaio.uuid.UUID
import java.lang.reflect.InvocationTargetException
/** /**
* Life-cycle messages for the Actors * Life-cycle messages for the Actors
@ -28,18 +29,18 @@ sealed trait LifeCycleMessage extends Serializable
*/ */
sealed trait AutoReceivedMessage { self: LifeCycleMessage => } sealed trait AutoReceivedMessage { self: LifeCycleMessage => }
case class HotSwap(code: ActorRef => Actor.Receive, discardOld: Boolean = true) case class HotSwap(code: ActorRef => Actor.Receive, discardOld: Boolean = true) extends AutoReceivedMessage with LifeCycleMessage {
extends AutoReceivedMessage with LifeCycleMessage {
/** /**
* Java API * Java API
*/ */
def this(code: akka.japi.Function[ActorRef,Procedure[Any]], discardOld: Boolean) = def this(code: akka.japi.Function[ActorRef,Procedure[Any]], discardOld: Boolean) = {
this( (self: ActorRef) => { this( (self: ActorRef) => {
val behavior = code(self) val behavior = code(self)
val result: Actor.Receive = { case msg => behavior(msg) } val result: Actor.Receive = { case msg => behavior(msg) }
result result
}, discardOld) }, discardOld)
}
/** /**
* Java API with default non-stacking behavior * Java API with default non-stacking behavior
@ -72,12 +73,12 @@ case class MaximumNumberOfRestartsWithinTimeRangeReached(
@BeanProperty val lastExceptionCausingRestart: Throwable) extends LifeCycleMessage @BeanProperty val lastExceptionCausingRestart: Throwable) extends LifeCycleMessage
// Exceptions for Actors // Exceptions for Actors
class ActorStartException private[akka](message: String) extends AkkaException(message) class ActorStartException private[akka](message: String, cause: Throwable = null) extends AkkaException(message, cause)
class IllegalActorStateException private[akka](message: String) extends AkkaException(message) class IllegalActorStateException private[akka](message: String, cause: Throwable = null) extends AkkaException(message, cause)
class ActorKilledException private[akka](message: String) extends AkkaException(message) class ActorKilledException private[akka](message: String, cause: Throwable = null) extends AkkaException(message, cause)
class ActorInitializationException private[akka](message: String) extends AkkaException(message) class ActorInitializationException private[akka](message: String, cause: Throwable = null) extends AkkaException(message, cause)
class ActorTimeoutException private[akka](message: String) extends AkkaException(message) class ActorTimeoutException private[akka](message: String, cause: Throwable = null) extends AkkaException(message, cause)
class InvalidMessageException private[akka](message: String) extends AkkaException(message) class InvalidMessageException private[akka](message: String, cause: Throwable = null) extends AkkaException(message, cause)
/** /**
* This message is thrown by default when an Actors behavior doesn't match a message * This message is thrown by default when an Actors behavior doesn't match a message
@ -130,7 +131,9 @@ object Actor extends ListenerManagement {
*/ */
type Receive = PartialFunction[Any, Unit] type Receive = PartialFunction[Any, Unit]
private[actor] val actorRefInCreation = new scala.util.DynamicVariable[Option[ActorRef]](None) private[actor] val actorRefInCreation = new ThreadLocal[Option[ActorRef]]{
override def initialValue = None
}
/** /**
* Creates an ActorRef out of the Actor with type T. * Creates an ActorRef out of the Actor with type T.
@ -238,7 +241,15 @@ object Actor extends ListenerManagement {
implicit val format: Format[T] = { implicit val format: Format[T] = {
if (formatClassName == "N/A") formatErrorDueTo("no class name defined in configuration") if (formatClassName == "N/A") formatErrorDueTo("no class name defined in configuration")
val f = ReflectiveAccess.getObjectFor(formatClassName).getOrElse(formatErrorDueTo("it could not be loaded")) val f = ReflectiveAccess.getObjectFor(formatClassName) match {
case Right(actor) => actor
case Left(exception) =>
val cause = exception match {
case i: InvocationTargetException => i.getTargetException
case _ => exception
}
formatErrorDueTo(" " + cause.toString)
}
if (f.isInstanceOf[Format[T]]) f.asInstanceOf[Format[T]] if (f.isInstanceOf[Format[T]]) f.asInstanceOf[Format[T]]
else formatErrorDueTo("class must be of type [akka.serialization.Format[T]]") else formatErrorDueTo("class must be of type [akka.serialization.Format[T]]")
} }
@ -296,7 +307,6 @@ object Actor extends ListenerManagement {
* times if for example the Actor is supervised and needs to be restarted. * times if for example the Actor is supervised and needs to be restarted.
* Uses generated address. * Uses generated address.
* <p/> * <p/>
* This function should <b>NOT</b> be used for remote actors.
* <pre> * <pre>
* import Actor._ * import Actor._
* val actor = actorOf(new MyActor) * val actor = actorOf(new MyActor)
@ -340,7 +350,6 @@ object Actor extends ListenerManagement {
* times if for example the Actor is supervised and needs to be restarted. * times if for example the Actor is supervised and needs to be restarted.
* Uses generated address. * Uses generated address.
* <p/> * <p/>
* This function should <b>NOT</b> be used for remote actors.
* JAVA API * JAVA API
*/ */
def actorOf[T <: Actor](creator: Creator[T]): ActorRef = def actorOf[T <: Actor](creator: Creator[T]): ActorRef =
@ -387,13 +396,21 @@ object Actor extends ListenerManagement {
private[akka] def newLocalActorRef(clazz: Class[_ <: Actor], address: String): ActorRef = { private[akka] def newLocalActorRef(clazz: Class[_ <: Actor], address: String): ActorRef = {
new LocalActorRef(() => { new LocalActorRef(() => {
import ReflectiveAccess.{ createInstance, noParams, noArgs } import ReflectiveAccess.{ createInstance, noParams, noArgs }
createInstance[Actor](clazz.asInstanceOf[Class[_]], noParams, noArgs).getOrElse( createInstance[Actor](clazz.asInstanceOf[Class[_]], noParams, noArgs) match {
throw new ActorInitializationException( case Right(actor) => actor
"Could not instantiate Actor" + case Left(exception) =>
"\nMake sure Actor is NOT defined inside a class/trait," + val cause = exception match {
"\nif so put it outside the class/trait, f.e. in a companion object," + case i: InvocationTargetException => i.getTargetException
"\nOR try to change: 'actorOf[MyActor]' to 'actorOf(new MyActor)'.")) case _ => exception
}, address) }
throw new ActorInitializationException(
"Could not instantiate Actor of " + clazz +
"\nMake sure Actor is NOT defined inside a class/trait," +
"\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)
} }
/** /**
@ -401,6 +418,19 @@ object Actor extends ListenerManagement {
* to convert an Option[Any] to an Option[T]. * to convert an Option[Any] to an Option[T].
*/ */
implicit def toAnyOptionAsTypedOption(anyOption: Option[Any]) = new AnyOptionAsTypedOption(anyOption) 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
})
} }
/** /**
@ -463,7 +493,7 @@ trait Actor {
* the 'forward' function. * the 'forward' function.
*/ */
@transient implicit val someSelf: Some[ActorRef] = { @transient implicit val someSelf: Some[ActorRef] = {
val optRef = Actor.actorRefInCreation.value val optRef = Actor.actorRefInCreation.get
if (optRef.isEmpty) throw new ActorInitializationException( if (optRef.isEmpty) throw new ActorInitializationException(
"ActorRef for instance of actor [" + getClass.getName + "] is not in scope." + "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 actor explicitly using 'new MyActor'." +
@ -471,7 +501,7 @@ trait Actor {
"\n\tEither use:" + "\n\tEither use:" +
"\n\t\t'val actor = Actor.actorOf[MyActor]', or" + "\n\t\t'val actor = Actor.actorOf[MyActor]', or" +
"\n\t\t'val actor = Actor.actorOf(new MyActor(..))'") "\n\t\t'val actor = Actor.actorOf(new MyActor(..))'")
Actor.actorRefInCreation.value = None Actor.actorRefInCreation.set(None)
optRef.asInstanceOf[Some[ActorRef]] optRef.asInstanceOf[Some[ActorRef]]
} }
@ -540,14 +570,14 @@ trait Actor {
* <p/> * <p/>
* Is called when an Actor is started by invoking 'actor.start()'. * Is called when an Actor is started by invoking 'actor.start()'.
*/ */
def preStart {} def preStart() {}
/** /**
* User overridable callback. * User overridable callback.
* <p/> * <p/>
* Is called when 'actor.stop()' is invoked. * Is called when 'actor.stop()' is invoked.
*/ */
def postStop {} def postStop() {}
/** /**
* User overridable callback. * User overridable callback.

View file

@ -97,7 +97,7 @@ trait ActorRef extends ActorRefShared with java.lang.Comparable[ActorRef] { scal
* Defines the default timeout for '!!' and '!!!' invocations, * Defines the default timeout for '!!' and '!!!' invocations,
* e.g. the timeout for the future returned by the call to '!!' and '!!!'. * e.g. the timeout for the future returned by the call to '!!' and '!!!'.
*/ */
@deprecated("Will be replaced by implicit-scoped timeout on all methods that needs it, will default to timeout specified in config") @deprecated("Will be replaced by implicit-scoped timeout on all methods that needs it, will default to timeout specified in config", "1.1")
@BeanProperty @BeanProperty
@volatile @volatile
var timeout: Long = Actor.TIMEOUT var timeout: Long = Actor.TIMEOUT
@ -232,7 +232,7 @@ trait ActorRef extends ActorRefShared with java.lang.Comparable[ActorRef] { scal
/** /**
* Is the actor able to handle the message passed in as arguments? * Is the actor able to handle the message passed in as arguments?
*/ */
@deprecated("Will be removed without replacement, it's just not reliable in the face of `become` and `unbecome`") @deprecated("Will be removed without replacement, it's just not reliable in the face of `become` and `unbecome`", "1.1")
def isDefinedAt(message: Any): Boolean = actor.isDefinedAt(message) def isDefinedAt(message: Any): Boolean = actor.isDefinedAt(message)
/** /**
@ -512,7 +512,7 @@ class LocalActorRef private[akka] (private[this] val actorFactory: () => Actor,
@volatile @volatile
private var maxNrOfRetriesCount: Int = 0 private var maxNrOfRetriesCount: Int = 0
@volatile @volatile
private var restartsWithinTimeRangeTimestamp: Long = 0L private var restartTimeWindowStartNanos: Long = 0L
@volatile @volatile
private var _mailbox: AnyRef = _ private var _mailbox: AnyRef = _
@volatile @volatile
@ -724,30 +724,32 @@ class LocalActorRef private[akka] (private[this] val actorFactory: () => Actor,
} }
private def requestRestartPermission(maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Boolean = { private def requestRestartPermission(maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Boolean = {
val denied = if (maxNrOfRetries.isEmpty && withinTimeRange.isEmpty) { //Immortal val denied = if (maxNrOfRetries.isEmpty && withinTimeRange.isEmpty) { //Immortal
false false
} else if (withinTimeRange.isEmpty) { // restrict number of restarts } else if (withinTimeRange.isEmpty) { // restrict number of restarts
maxNrOfRetriesCount += 1 //Increment number of retries val retries = maxNrOfRetriesCount + 1
maxNrOfRetriesCount > maxNrOfRetries.get maxNrOfRetriesCount = retries //Increment number of retries
retries > maxNrOfRetries.get
} else { // cannot restart more than N within M timerange } else { // cannot restart more than N within M timerange
maxNrOfRetriesCount += 1 //Increment number of retries val retries = maxNrOfRetriesCount + 1
val windowStart = restartsWithinTimeRangeTimestamp
val now = System.currentTimeMillis val windowStart = restartTimeWindowStartNanos
val retries = maxNrOfRetriesCount val now = System.nanoTime
//We are within the time window if it isn't the first restart, or if the window hasn't closed //We are within the time window if it isn't the first restart, or if the window hasn't closed
val insideWindow = if (windowStart == 0) false val insideWindow = if (windowStart == 0) false
else (now - windowStart) <= withinTimeRange.get else (now - windowStart) <= TimeUnit.MILLISECONDS.toNanos(withinTimeRange.get)
//The actor is dead if it dies X times within the window of restart
val unrestartable = insideWindow && retries > maxNrOfRetries.getOrElse(1)
if (windowStart == 0 || !insideWindow) //(Re-)set the start of the window if (windowStart == 0 || !insideWindow) //(Re-)set the start of the window
restartsWithinTimeRangeTimestamp = now restartTimeWindowStartNanos = now
if (windowStart != 0 && !insideWindow) //Reset number of restarts if window has expired //Reset number of restarts if window has expired, otherwise, increment it
maxNrOfRetriesCount = 1 maxNrOfRetriesCount = if (windowStart != 0 && !insideWindow) 1 else retries //Increment number of retries
unrestartable val restartCountLimit = if (maxNrOfRetries.isDefined) maxNrOfRetries.get else 1
//The actor is dead if it dies X times within the window of restart
insideWindow && retries > restartCountLimit
} }
denied == false //If we weren't denied, we have a go denied == false //If we weren't denied, we have a go
@ -839,12 +841,12 @@ class LocalActorRef private[akka] (private[this] val actorFactory: () => Actor,
private[this] def newActor: Actor = { private[this] def newActor: Actor = {
try { try {
Actor.actorRefInCreation.value = Some(this) Actor.actorRefInCreation.set(Some(this))
val a = actorFactory() val a = actorFactory()
if (a eq null) throw new ActorInitializationException("Actor instance passed to ActorRef can not be 'null'") if (a eq null) throw new ActorInitializationException("Actor instance passed to ActorRef can not be 'null'")
a a
} finally { } finally {
Actor.actorRefInCreation.value = None Actor.actorRefInCreation.set(None)
} }
} }
@ -1009,7 +1011,7 @@ private[akka] case class RemoteActorRef private[akka] (
} }
// ==== NOT SUPPORTED ==== // ==== NOT SUPPORTED ====
@deprecated("Will be removed without replacement, doesn't make any sense to have in the face of `become` and `unbecome`") @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 def actorClass: Class[_ <: Actor] = unsupported
def dispatcher_=(md: MessageDispatcher): Unit = unsupported def dispatcher_=(md: MessageDispatcher): Unit = unsupported
def dispatcher: MessageDispatcher = unsupported def dispatcher: MessageDispatcher = unsupported

View file

@ -297,7 +297,7 @@ class Index[K <: AnyRef,V <: AnyRef : Manifest] {
/** /**
* Associates the value of type V with the key of type K * Associates the value of type V with the key of type K
* @returns true if the value didn't exist for the key previously, and false otherwise * @return true if the value didn't exist for the key previously, and false otherwise
*/ */
def put(key: K, value: V): Boolean = { def put(key: K, value: V): Boolean = {
//Tailrecursive spin-locking put //Tailrecursive spin-locking put
@ -339,7 +339,7 @@ class Index[K <: AnyRef,V <: AnyRef : Manifest] {
} }
/** /**
* @returns a _new_ array of all existing values for the given key at the time of the call * @return a _new_ array of all existing values for the given key at the time of the call
*/ */
def values(key: K): Array[V] = { def values(key: K): Array[V] = {
val set: JSet[V] = container get key val set: JSet[V] = container get key
@ -348,7 +348,7 @@ class Index[K <: AnyRef,V <: AnyRef : Manifest] {
} }
/** /**
* @returns Some(value) for the first matching value where the supplied function returns true for the given key, * @return Some(value) for the first matching value where the supplied function returns true for the given key,
* if no matches it returns None * if no matches it returns None
*/ */
def findValue(key: K)(f: (V) => Boolean): Option[V] = { def findValue(key: K)(f: (V) => Boolean): Option[V] = {
@ -370,7 +370,7 @@ class Index[K <: AnyRef,V <: AnyRef : Manifest] {
/** /**
* Disassociates the value of type V from the key of type K * Disassociates the value of type V from the key of type K
* @returns true if the value was disassociated from the key and false if it wasn't previously associated with the key * @return true if the value was disassociated from the key and false if it wasn't previously associated with the key
*/ */
def remove(key: K, value: V): Boolean = { def remove(key: K, value: V): Boolean = {
val set = container get key val set = container get key
@ -388,7 +388,7 @@ class Index[K <: AnyRef,V <: AnyRef : Manifest] {
} }
/** /**
* @returns true if the underlying containers is empty, may report false negatives when the last remove is underway * @return true if the underlying containers is empty, may report false negatives when the last remove is underway
*/ */
def isEmpty: Boolean = container.isEmpty def isEmpty: Boolean = container.isEmpty

View file

@ -21,7 +21,6 @@ trait BootableActorLoaderService extends Bootable {
protected def createApplicationClassLoader : Option[ClassLoader] = Some({ protected def createApplicationClassLoader : Option[ClassLoader] = Some({
if (HOME.isDefined) { if (HOME.isDefined) {
val CONFIG = HOME.get + "/config"
val DEPLOY = HOME.get + "/deploy" val DEPLOY = HOME.get + "/deploy"
val DEPLOY_DIR = new File(DEPLOY) val DEPLOY_DIR = new File(DEPLOY)
if (!DEPLOY_DIR.exists) { if (!DEPLOY_DIR.exists) {

View file

@ -105,13 +105,17 @@ object Scheduler {
} }
} }
def shutdown: Unit = synchronized { def shutdown() {
service.shutdown synchronized {
service.shutdown()
}
} }
def restart: Unit = synchronized { def restart() {
shutdown synchronized {
service = Executors.newSingleThreadScheduledExecutor(SchedulerThreadFactory) shutdown()
service = Executors.newSingleThreadScheduledExecutor(SchedulerThreadFactory)
}
} }
} }

View file

@ -13,7 +13,7 @@ import java.util.concurrent.{CopyOnWriteArrayList, ConcurrentHashMap}
import java.net.InetSocketAddress import java.net.InetSocketAddress
import akka.config.Supervision._ import akka.config.Supervision._
class SupervisorException private[akka](message: String) extends AkkaException(message) class SupervisorException private[akka](message: String, cause: Throwable = null) extends AkkaException(message, cause)
/** /**
* Factory object for creating supervisors declarative. It creates instances of the 'Supervisor' class. * Factory object for creating supervisors declarative. It creates instances of the 'Supervisor' class.

View file

@ -88,14 +88,14 @@ abstract class UntypedActor extends Actor {
* <p/> * <p/>
* Is called when an Actor is started by invoking 'actor.start()'. * Is called when an Actor is started by invoking 'actor.start()'.
*/ */
override def preStart {} override def preStart() {}
/** /**
* User overridable callback. * User overridable callback.
* <p/> * <p/>
* Is called when 'actor.stop()' is invoked. * Is called when 'actor.stop()' is invoked.
*/ */
override def postStop {} override def postStop() {}
/** /**
* User overridable callback. * User overridable callback.

View file

@ -4,8 +4,6 @@
package akka package akka
import actor.{ScalaActorRef, ActorRef}
package object actor { package object actor {
implicit def actorRef2Scala(ref: ActorRef): ScalaActorRef = implicit def actorRef2Scala(ref: ActorRef): ScalaActorRef =
ref.asInstanceOf[ScalaActorRef] ref.asInstanceOf[ScalaActorRef]

View file

@ -6,8 +6,8 @@ package akka.config
import akka.AkkaException import akka.AkkaException
class ConfigurationException(message: String) extends AkkaException(message) class ConfigurationException(message: String, cause: Throwable = null) extends AkkaException(message, cause)
class ModuleNotAvailableException(message: String) extends AkkaException(message) class ModuleNotAvailableException(message: String, cause: Throwable = null) extends AkkaException(message, cause)
/** /**
* Loads up the configuration (from the akka.conf file). * Loads up the configuration (from the akka.conf file).
@ -15,7 +15,7 @@ class ModuleNotAvailableException(message: String) extends AkkaException(message
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
object Config { object Config {
val VERSION = "1.1-SNAPSHOT" val VERSION = "1.2-SNAPSHOT"
val HOME = { val HOME = {
val envHome = System.getenv("AKKA_HOME") match { val envHome = System.getenv("AKKA_HOME") match {
@ -31,7 +31,7 @@ object Config {
envHome orElse systemHome envHome orElse systemHome
} }
val config: Configuration = try { val config: Configuration = {
val confName = { val confName = {
val envConf = System.getenv("AKKA_MODE") match { val envConf = System.getenv("AKKA_MODE") match {
case null | "" => None case null | "" => None
@ -46,42 +46,46 @@ object Config {
(envConf orElse systemConf).map("akka." + _ + ".conf").getOrElse("akka.conf") (envConf orElse systemConf).map("akka." + _ + ".conf").getOrElse("akka.conf")
} }
val newInstance = val (newInstance, source) =
if (System.getProperty("akka.config", "") != "") { if (System.getProperty("akka.config", "") != "") {
val configFile = System.getProperty("akka.config", "") val configFile = System.getProperty("akka.config", "")
println("Loading config from -Dakka.config=" + configFile) (() => Configuration.fromFile(configFile), "Loading config from -Dakka.config=" + configFile)
Configuration.fromFile(configFile)
} else if (getClass.getClassLoader.getResource(confName) ne null) { } else if (getClass.getClassLoader.getResource(confName) ne null) {
println("Loading config [" + confName + "] from the application classpath.") (() => Configuration.fromResource(confName, getClass.getClassLoader), "Loading config [" + confName + "] from the application classpath.")
Configuration.fromResource(confName, getClass.getClassLoader)
} else if (HOME.isDefined) { } else if (HOME.isDefined) {
val configFile = HOME.get + "/config/" + confName val configFile = HOME.get + "/config/" + confName
println("AKKA_HOME is defined as [" + HOME.get + "], loading config from [" + configFile + "].") (() => Configuration.fromFile(configFile), "AKKA_HOME is defined as [" + HOME.get + "], loading config from [" + configFile + "].")
Configuration.fromFile(configFile)
} else { } else {
println( (() => Configuration.fromString("akka {}"), // default empty config
"\nCan't load '" + confName + "'." + "\nCan't load '" + confName + "'." +
"\nOne of the three ways of locating the '" + confName + "' file needs to be defined:" + "\nOne of the three ways of locating the '" + confName + "' file needs to be defined:" +
"\n\t1. Define the '-Dakka.config=...' system property option." + "\n\t1. Define the '-Dakka.config=...' system property option." +
"\n\t2. Put the '" + confName + "' file on the classpath." + "\n\t2. Put the '" + confName + "' file on the classpath." +
"\n\t3. Define 'AKKA_HOME' environment variable pointing to the root of the Akka distribution." + "\n\t3. Define 'AKKA_HOME' environment variable pointing to the root of the Akka distribution." +
"\nI have no way of finding the '" + confName + "' configuration file." + "\nI have no way of finding the '" + confName + "' configuration file." +
"\nUsing default values everywhere.") "\nUsing default values everywhere.")
Configuration.fromString("akka {}") // default empty config
} }
val configVersion = newInstance.getString("akka.version", VERSION) try {
if (configVersion != VERSION) val i = newInstance()
throw new ConfigurationException(
"Akka JAR version [" + VERSION + "] is different than the provided config version [" + configVersion + "]")
newInstance val configVersion = i.getString("akka.version", VERSION)
} catch { if (configVersion != VERSION)
case e => throw new ConfigurationException(
System.err.println("Couldn't parse config, fatal error.") "Akka JAR version [" + VERSION + "] is different than the provided config version [" + configVersion + "]")
e.printStackTrace(System.err)
System.exit(-1) if(Configuration.outputConfigSources)
throw e System.out.println(source)
i
} catch {
case e =>
System.err.println("Couldn't parse config, fatal error.")
System.err.println("Config source: " + source)
e.printStackTrace(System.err)
System.exit(-1)
throw e
}
} }
val CONFIG_VERSION = config.getString("akka.version", VERSION) val CONFIG_VERSION = config.getString("akka.version", VERSION)

View file

@ -14,6 +14,8 @@ object Configuration {
val DefaultPath = new File(".").getCanonicalPath val DefaultPath = new File(".").getCanonicalPath
val DefaultImporter = new FilesystemImporter(DefaultPath) val DefaultImporter = new FilesystemImporter(DefaultPath)
val outputConfigSources = System.getProperty("akka.output.config.source") ne null
def load(data: String, importer: Importer = DefaultImporter): Configuration = { def load(data: String, importer: Importer = DefaultImporter): Configuration = {
val parser = new ConfigParser(importer = importer) val parser = new ConfigParser(importer = importer)
new Configuration(parser parse data) new Configuration(parser parse data)
@ -59,6 +61,13 @@ class Configuration(val map: Map[String, Any]) {
private val trueValues = Set("true", "on") private val trueValues = Set("true", "on")
private val falseValues = Set("false", "off") private val falseValues = Set("false", "off")
private def outputIfDesiredAndReturnInput[T](key: String, t: T): T = {
if (Configuration.outputConfigSources)
println("Akka config is using default value for: " + key)
t
}
def contains(key: String): Boolean = map contains key def contains(key: String): Boolean = map contains key
def keys: Iterable[String] = map.keys def keys: Iterable[String] = map.keys
@ -71,7 +80,8 @@ class Configuration(val map: Map[String, Any]) {
} }
} }
def getAny(key: String, defaultValue: Any): Any = getAny(key).getOrElse(defaultValue) def getAny(key: String, defaultValue: Any): Any =
getAny(key).getOrElse(outputIfDesiredAndReturnInput(key, defaultValue))
def getListAny(key: String): Seq[Any] = { def getListAny(key: String): Seq[Any] = {
try { try {
@ -83,7 +93,8 @@ class Configuration(val map: Map[String, Any]) {
def getString(key: String): Option[String] = map.get(key).map(_.toString) def getString(key: String): Option[String] = map.get(key).map(_.toString)
def getString(key: String, defaultValue: String): String = getString(key).getOrElse(defaultValue) def getString(key: String, defaultValue: String): String =
getString(key).getOrElse(outputIfDesiredAndReturnInput(key, defaultValue))
def getList(key: String): Seq[String] = { def getList(key: String): Seq[String] = {
try { try {
@ -101,7 +112,8 @@ class Configuration(val map: Map[String, Any]) {
} }
} }
def getInt(key: String, defaultValue: Int): Int = getInt(key).getOrElse(defaultValue) def getInt(key: String, defaultValue: Int): Int =
getInt(key).getOrElse(outputIfDesiredAndReturnInput(key, defaultValue))
def getLong(key: String): Option[Long] = { def getLong(key: String): Option[Long] = {
try { try {
@ -111,7 +123,8 @@ class Configuration(val map: Map[String, Any]) {
} }
} }
def getLong(key: String, defaultValue: Long): Long = getLong(key).getOrElse(defaultValue) def getLong(key: String, defaultValue: Long): Long =
getLong(key).getOrElse(outputIfDesiredAndReturnInput(key, defaultValue))
def getFloat(key: String): Option[Float] = { def getFloat(key: String): Option[Float] = {
try { try {
@ -121,7 +134,8 @@ class Configuration(val map: Map[String, Any]) {
} }
} }
def getFloat(key: String, defaultValue: Float): Float = getFloat(key).getOrElse(defaultValue) def getFloat(key: String, defaultValue: Float): Float =
getFloat(key).getOrElse(outputIfDesiredAndReturnInput(key, defaultValue))
def getDouble(key: String): Option[Double] = { def getDouble(key: String): Option[Double] = {
try { try {
@ -131,7 +145,8 @@ class Configuration(val map: Map[String, Any]) {
} }
} }
def getDouble(key: String, defaultValue: Double): Double = getDouble(key).getOrElse(defaultValue) def getDouble(key: String, defaultValue: Double): Double =
getDouble(key).getOrElse(outputIfDesiredAndReturnInput(key, defaultValue))
def getBoolean(key: String): Option[Boolean] = { def getBoolean(key: String): Option[Boolean] = {
getString(key) flatMap { s => getString(key) flatMap { s =>
@ -141,11 +156,13 @@ class Configuration(val map: Map[String, Any]) {
} }
} }
def getBoolean(key: String, defaultValue: Boolean): Boolean = getBool(key).getOrElse(defaultValue) def getBoolean(key: String, defaultValue: Boolean): Boolean =
getBool(key).getOrElse(outputIfDesiredAndReturnInput(key, defaultValue))
def getBool(key: String): Option[Boolean] = getBoolean(key) def getBool(key: String): Option[Boolean] = getBoolean(key)
def getBool(key: String, defaultValue: Boolean): Boolean = getBoolean(key, defaultValue) def getBool(key: String, defaultValue: Boolean): Boolean =
getBoolean(key, defaultValue)
def apply(key: String): String = getString(key) match { def apply(key: String): String = getString(key) match {
case None => throw new ConfigurationException("undefined config: " + key) case None => throw new ConfigurationException("undefined config: " + key)

View file

@ -46,6 +46,12 @@ object Supervision {
if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (withinTimeRange < 0) None else Some(withinTimeRange)) if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (withinTimeRange < 0) None else Some(withinTimeRange))
} }
/**
* Restart all actors linked to the same supervisor when one fails,
* trapExit = which Throwables should be intercepted
* maxNrOfRetries = the number of times an actor is allowed to be restarted
* withinTimeRange = millisecond time window for maxNrOfRetries, negative means no window
*/
case class AllForOneStrategy(override val trapExit: List[Class[_ <: Throwable]], case class AllForOneStrategy(override val trapExit: List[Class[_ <: Throwable]],
maxNrOfRetries: Option[Int] = None, maxNrOfRetries: Option[Int] = None,
withinTimeRange: Option[Int] = None) extends FaultHandlingStrategy(trapExit) { withinTimeRange: Option[Int] = None) extends FaultHandlingStrategy(trapExit) {
@ -68,6 +74,12 @@ object Supervision {
if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (withinTimeRange < 0) None else Some(withinTimeRange)) if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (withinTimeRange < 0) None else Some(withinTimeRange))
} }
/**
* Restart an actor when it fails
* trapExit = which Throwables should be intercepted
* maxNrOfRetries = the number of times an actor is allowed to be restarted
* withinTimeRange = millisecond time window for maxNrOfRetries, negative means no window
*/
case class OneForOneStrategy(override val trapExit: List[Class[_ <: Throwable]], case class OneForOneStrategy(override val trapExit: List[Class[_ <: Throwable]],
maxNrOfRetries: Option[Int] = None, maxNrOfRetries: Option[Int] = None,
withinTimeRange: Option[Int] = None) extends FaultHandlingStrategy(trapExit) { withinTimeRange: Option[Int] = None) extends FaultHandlingStrategy(trapExit) {

View file

@ -0,0 +1,165 @@
/**
* Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.dataflow
import java.util.concurrent.atomic.AtomicReference
import java.util.concurrent.{ConcurrentLinkedQueue, LinkedBlockingQueue}
import akka.event.EventHandler
import akka.actor.{Actor, ActorRef}
import akka.actor.Actor._
import akka.dispatch.CompletableFuture
import akka.AkkaException
import akka.japi.{ Function, Effect }
/**
* Implements Oz-style dataflow (single assignment) variables.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object DataFlow {
object Start
object Exit
class DataFlowVariableException(message: String, cause: Throwable = null) extends AkkaException(message, cause)
/**
* Executes the supplied thunk in another thread.
*/
def thread(body: => Unit): Unit = spawn(body)
/**
* JavaAPI.
* Executes the supplied Effect in another thread.
*/
def thread(body: Effect): Unit = spawn(body.apply)
/**
* Executes the supplied function in another thread.
*/
def thread[A <: AnyRef, R <: AnyRef](body: A => R) =
actorOf(new ReactiveEventBasedThread(body)).start()
/**
* JavaAPI.
* Executes the supplied Function in another thread.
*/
def thread[A <: AnyRef, R <: AnyRef](body: Function[A,R]) =
actorOf(new ReactiveEventBasedThread(body.apply)).start()
private class ReactiveEventBasedThread[A <: AnyRef, T <: AnyRef](body: A => T)
extends Actor {
def receive = {
case Exit => self.stop()
case message => self.reply(body(message.asInstanceOf[A]))
}
}
private object DataFlowVariable {
private sealed abstract class DataFlowVariableMessage
private case class Set[T <: Any](value: T) extends DataFlowVariableMessage
private object Get extends DataFlowVariableMessage
}
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
@deprecated("Superceeded by Future and CompletableFuture as of 1.1", "1.1")
sealed class DataFlowVariable[T <: Any](timeoutMs: Long) {
import DataFlowVariable._
def this() = this(1000 * 60)
private val value = new AtomicReference[Option[T]](None)
private val blockedReaders = new ConcurrentLinkedQueue[ActorRef]
private class In[T <: Any](dataFlow: DataFlowVariable[T]) extends Actor {
self.timeout = timeoutMs
def receive = {
case s@Set(v) =>
if (dataFlow.value.compareAndSet(None, Some(v.asInstanceOf[T]))) {
while(dataFlow.blockedReaders.peek ne null)
dataFlow.blockedReaders.poll ! s
} else throw new DataFlowVariableException(
"Attempt to change data flow variable (from [" + dataFlow.value.get + "] to [" + v + "])")
case Exit => self.stop()
}
}
private class Out[T <: Any](dataFlow: DataFlowVariable[T]) extends Actor {
self.timeout = timeoutMs
private var readerFuture: Option[CompletableFuture[Any]] = None
def receive = {
case Get => dataFlow.value.get match {
case Some(value) => self reply value
case None => readerFuture = self.senderFuture
}
case Set(v:T) => readerFuture.map(_ completeWithResult v)
case Exit => self.stop()
}
}
private[this] val in = actorOf(new In(this)).start()
/**
* Sets the value of this variable (if unset) with the value of the supplied variable.
*/
def <<(ref: DataFlowVariable[T]) {
if (this.value.get.isEmpty) in ! Set(ref())
else throw new DataFlowVariableException(
"Attempt to change data flow variable (from [" + this.value.get + "] to [" + ref() + "])")
}
/**
* JavaAPI.
* Sets the value of this variable (if unset) with the value of the supplied variable.
*/
def set(ref: DataFlowVariable[T]) { this << ref }
/**
* Sets the value of this variable (if unset).
*/
def <<(value: T) {
if (this.value.get.isEmpty) in ! Set(value)
else throw new DataFlowVariableException(
"Attempt to change data flow variable (from [" + this.value.get + "] to [" + value + "])")
}
/**
* JavaAPI.
* Sets the value of this variable (if unset) with the value of the supplied variable.
*/
def set(value: T) { this << value }
/**
* Retrieves the value of variable, throws a DataFlowVariableException if it times out.
*/
def get(): T = this()
/**
* Retrieves the value of variable, throws a DataFlowVariableException if it times out.
*/
def apply(): T = {
value.get getOrElse {
val out = actorOf(new Out(this)).start()
val result = try {
blockedReaders offer out
(out !! Get).as[T]
} catch {
case e: Exception =>
EventHandler.error(e, this, e.getMessage)
out ! Exit
throw e
}
result.getOrElse(throw new DataFlowVariableException(
"Timed out (after " + timeoutMs + " milliseconds) while waiting for result"))
}
}
def shutdown() { in ! Exit }
}
}

View file

@ -187,14 +187,14 @@ object Dispatchers {
case "GlobalExecutorBasedEventDriven" => GlobalExecutorBasedEventDrivenDispatcherConfigurator case "GlobalExecutorBasedEventDriven" => GlobalExecutorBasedEventDrivenDispatcherConfigurator
case fqn => case fqn =>
ReflectiveAccess.getClassFor[MessageDispatcherConfigurator](fqn) match { ReflectiveAccess.getClassFor[MessageDispatcherConfigurator](fqn) match {
case Some(clazz) => case r: Right[_, Class[MessageDispatcherConfigurator]] =>
val instance = ReflectiveAccess.createInstance[MessageDispatcherConfigurator](clazz, Array[Class[_]](), Array[AnyRef]()) ReflectiveAccess.createInstance[MessageDispatcherConfigurator](r.b, Array[Class[_]](), Array[AnyRef]()) match {
if (instance.isEmpty) case r: Right[Exception, MessageDispatcherConfigurator] => r.b
throw new IllegalArgumentException("Cannot instantiate MessageDispatcherConfigurator type [%s], make sure it has a default no-args constructor" format fqn) case l: Left[Exception, MessageDispatcherConfigurator] =>
else throw new IllegalArgumentException("Cannot instantiate MessageDispatcherConfigurator type [%s], make sure it has a default no-args constructor" format fqn, l.a)
instance.get }
case None => case l: Left[Exception, _] =>
throw new IllegalArgumentException("Unknown MessageDispatcherConfigurator type [%s]" format fqn) throw new IllegalArgumentException("Unknown MessageDispatcherConfigurator type [%s]" format fqn, l.a)
} }
} map { } map {
_ configure cfg _ configure cfg

View file

@ -5,12 +5,10 @@
package akka.dispatch package akka.dispatch
import akka.event.EventHandler import akka.event.EventHandler
import akka.actor.{ActorRef, IllegalActorStateException} import akka.actor.{ActorRef}
import akka.util.{ReflectiveAccess, Switch}
import java.util.Queue
import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.atomic.AtomicReference
import java.util.concurrent.{ TimeUnit, ExecutorService, RejectedExecutionException, ConcurrentLinkedQueue, LinkedBlockingQueue} import java.util.concurrent.{ TimeUnit, ExecutorService, RejectedExecutionException, ConcurrentLinkedQueue}
/** /**
* Default settings are: * Default settings are:
@ -99,7 +97,7 @@ class ExecutorBasedEventDrivenDispatcher(
registerForExecution(mbox) registerForExecution(mbox)
} }
private[akka] def executeFuture(invocation: FutureInvocation): Unit = if (active.isOn) { private[akka] def executeFuture(invocation: FutureInvocation[_]): Unit = if (active.isOn) {
try executorService.get() execute invocation try executorService.get() execute invocation
catch { catch {
case e: RejectedExecutionException => case e: RejectedExecutionException =>
@ -117,20 +115,14 @@ class ExecutorBasedEventDrivenDispatcher(
def createMailbox(actorRef: ActorRef): AnyRef = mailboxType match { def createMailbox(actorRef: ActorRef): AnyRef = mailboxType match {
case b: UnboundedMailbox => case b: UnboundedMailbox =>
if (b.blocking) { new ConcurrentLinkedQueue[MessageInvocation] with MessageQueue with ExecutableMailbox {
new DefaultUnboundedMessageQueue(true) with ExecutableMailbox { @inline final def dispatcher = ExecutorBasedEventDrivenDispatcher.this
final def dispatcher = ExecutorBasedEventDrivenDispatcher.this @inline final def enqueue(m: MessageInvocation) = this.add(m)
} @inline final def dequeue(): MessageInvocation = this.poll()
} else { //If we have an unbounded, non-blocking mailbox, we can go lockless
new ConcurrentLinkedQueue[MessageInvocation] with MessageQueue with ExecutableMailbox {
final def dispatcher = ExecutorBasedEventDrivenDispatcher.this
final def enqueue(m: MessageInvocation) = this.add(m)
final def dequeue(): MessageInvocation = this.poll()
}
} }
case b: BoundedMailbox => case b: BoundedMailbox =>
new DefaultBoundedMessageQueue(b.capacity, b.pushTimeOut, b.blocking) with ExecutableMailbox { new DefaultBoundedMessageQueue(b.capacity, b.pushTimeOut) with ExecutableMailbox {
final def dispatcher = ExecutorBasedEventDrivenDispatcher.this @inline final def dispatcher = ExecutorBasedEventDrivenDispatcher.this
} }
} }
@ -294,13 +286,13 @@ trait PriorityMailbox { self: ExecutorBasedEventDrivenDispatcher =>
override def createMailbox(actorRef: ActorRef): AnyRef = self.mailboxType match { override def createMailbox(actorRef: ActorRef): AnyRef = self.mailboxType match {
case b: UnboundedMailbox => case b: UnboundedMailbox =>
new UnboundedPriorityMessageQueue(b.blocking, comparator) with ExecutableMailbox { new UnboundedPriorityMessageQueue(comparator) with ExecutableMailbox {
final def dispatcher = self @inline final def dispatcher = self
} }
case b: BoundedMailbox => case b: BoundedMailbox =>
new BoundedPriorityMessageQueue(b.capacity, b.pushTimeOut, b.blocking, comparator) with ExecutableMailbox { new BoundedPriorityMessageQueue(b.capacity, b.pushTimeOut, comparator) with ExecutableMailbox {
final def dispatcher = self @inline final def dispatcher = self
} }
} }
} }

View file

@ -5,11 +5,8 @@
package akka.dispatch package akka.dispatch
import akka.actor.{ActorRef, Actor, IllegalActorStateException} import akka.actor.{ActorRef, Actor, IllegalActorStateException}
import akka.util.{ReflectiveAccess, Switch}
import java.util.Queue import util.DynamicVariable
import java.util.concurrent.atomic.{AtomicReference, AtomicInteger}
import java.util.concurrent.{ TimeUnit, ExecutorService, RejectedExecutionException, ConcurrentLinkedQueue, LinkedBlockingQueue}
/** /**
* An executor based event driven dispatcher which will try to redistribute work from busy actors to idle actors. It is assumed * An executor based event driven dispatcher which will try to redistribute work from busy actors to idle actors. It is assumed
@ -55,6 +52,7 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher(
@volatile private var actorType: Option[Class[_]] = None @volatile private var actorType: Option[Class[_]] = None
@volatile private var members = Vector[ActorRef]() @volatile private var members = Vector[ActorRef]()
private val donationInProgress = new DynamicVariable(false)
private[akka] override def register(actorRef: ActorRef) = { private[akka] override def register(actorRef: ActorRef) = {
//Verify actor type conformity //Verify actor type conformity
@ -78,18 +76,22 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher(
override private[akka] def dispatch(invocation: MessageInvocation) = { override private[akka] def dispatch(invocation: MessageInvocation) = {
val mbox = getMailbox(invocation.receiver) val mbox = getMailbox(invocation.receiver)
/*if (!mbox.isEmpty && attemptDonationOf(invocation, mbox)) { if (donationInProgress.value == false && (!mbox.isEmpty || mbox.dispatcherLock.locked) && attemptDonationOf(invocation, mbox)) {
//We were busy and we got to donate the message to some other lucky guy, we're done here //We were busy and we got to donate the message to some other lucky guy, we're done here
} else {*/ } else {
mbox enqueue invocation mbox enqueue invocation
registerForExecution(mbox) registerForExecution(mbox)
//} }
} }
override private[akka] def reRegisterForExecution(mbox: MessageQueue with ExecutableMailbox): Unit = { override private[akka] def reRegisterForExecution(mbox: MessageQueue with ExecutableMailbox): Unit = {
while(donateFrom(mbox)) {} //When we reregister, first donate messages to another actor try {
donationInProgress.value = true
while(donateFrom(mbox)) {} //When we reregister, first donate messages to another actor
} finally { donationInProgress.value = false }
if (!mbox.isEmpty) //If we still have messages left to process, reschedule for execution if (!mbox.isEmpty) //If we still have messages left to process, reschedule for execution
super.reRegisterForExecution(mbox) super.reRegisterForExecution(mbox)
} }
/** /**
@ -110,13 +112,14 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher(
/** /**
* Returns true if the donation succeeded or false otherwise * Returns true if the donation succeeded or false otherwise
*/ */
/*protected def attemptDonationOf(message: MessageInvocation, donorMbox: MessageQueue with ExecutableMailbox): Boolean = { protected def attemptDonationOf(message: MessageInvocation, donorMbox: MessageQueue with ExecutableMailbox): Boolean = try {
donationInProgress.value = true
val actors = members // copy to prevent concurrent modifications having any impact val actors = members // copy to prevent concurrent modifications having any impact
doFindDonorRecipient(donorMbox, actors, System.identityHashCode(message) % actors.size) match { doFindDonorRecipient(donorMbox, actors, System.identityHashCode(message) % actors.size) match {
case null => false case null => false
case recipient => donate(message, recipient) case recipient => donate(message, recipient)
} }
}*/ } finally { donationInProgress.value = false }
/** /**
* Rewrites the message and adds that message to the recipients mailbox * Rewrites the message and adds that message to the recipients mailbox

View file

@ -7,21 +7,24 @@ package akka.dispatch
import akka.AkkaException import akka.AkkaException
import akka.event.EventHandler import akka.event.EventHandler
import akka.actor.{Actor, Channel} import akka.actor.{Actor, Channel}
import akka.routing.Dispatcher import akka.util.Duration
import akka.japi.{ Procedure, Function => JFunc } import akka.japi.{ Procedure, Function => JFunc }
import scala.util.continuations._
import java.util.concurrent.locks.ReentrantLock import java.util.concurrent.locks.ReentrantLock
import java.util.concurrent. {ConcurrentLinkedQueue, TimeUnit, Callable} import java.util.concurrent. {ConcurrentLinkedQueue, TimeUnit, Callable}
import java.util.concurrent.TimeUnit.{NANOSECONDS => NANOS, MILLISECONDS => MILLIS} import java.util.concurrent.TimeUnit.{NANOSECONDS => NANOS, MILLISECONDS => MILLIS}
import java.util.concurrent.atomic. {AtomicBoolean, AtomicInteger} import java.util.concurrent.atomic. {AtomicBoolean}
import java.lang.{Iterable => JIterable} import java.lang.{Iterable => JIterable}
import java.util.{LinkedList => JLinkedList} import java.util.{LinkedList => JLinkedList}
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.collection.generic.CanBuildFrom import scala.collection.generic.CanBuildFrom
import scala.collection.mutable.Builder import scala.collection.mutable.Builder
import scala.collection.mutable.Stack
class FutureTimeoutException(message: String) extends AkkaException(message) class FutureTimeoutException(message: String, cause: Throwable = null) extends AkkaException(message, cause)
object Futures { object Futures {
@ -159,10 +162,10 @@ object Futures {
/** /**
* Java API. * Java API.
* Simple version of Futures.traverse. Transforms a java.lang.Iterable[Future[A]] into a Future[java.util.LinkedList[A]]. * Simple version of Futures.traverse. Transforms a java.lang.Iterable[Future[A]] into a Future[java.lang.Iterable[A]].
* Useful for reducing many Futures into a single Future. * Useful for reducing many Futures into a single Future.
*/ */
def sequence[A](in: JIterable[Future[A]], timeout: Long): Future[JLinkedList[A]] = def sequence[A](in: JIterable[Future[A]], timeout: Long): Future[JIterable[A]] =
scala.collection.JavaConversions.iterableAsScalaIterable(in).foldLeft(Future(new JLinkedList[A]()))((fr, fa) => scala.collection.JavaConversions.iterableAsScalaIterable(in).foldLeft(Future(new JLinkedList[A]()))((fr, fa) =>
for (r <- fr; a <- fa) yield { for (r <- fr; a <- fa) yield {
r add a r add a
@ -171,18 +174,18 @@ object Futures {
/** /**
* Java API. * Java API.
* Simple version of Futures.traverse. Transforms a java.lang.Iterable[Future[A]] into a Future[java.util.LinkedList[A]]. * Simple version of Futures.traverse. Transforms a java.lang.Iterable[Future[A]] into a Future[java.lang.Iterable[A]].
* Useful for reducing many Futures into a single Future. * Useful for reducing many Futures into a single Future.
*/ */
def sequence[A](in: JIterable[Future[A]]): Future[JLinkedList[A]] = sequence(in, Actor.TIMEOUT) def sequence[A](in: JIterable[Future[A]]): Future[JIterable[A]] = sequence(in, Actor.TIMEOUT)
/** /**
* Java API. * Java API.
* Transforms a java.lang.Iterable[A] into a Future[java.util.LinkedList[B]] using the provided Function A => Future[B]. * Transforms a java.lang.Iterable[A] into a Future[java.lang.Iterable[B]] using the provided Function A => Future[B].
* This is useful for performing a parallel map. For example, to apply a function to all items of a list * This is useful for performing a parallel map. For example, to apply a function to all items of a list
* in parallel. * in parallel.
*/ */
def traverse[A, B](in: JIterable[A], timeout: Long, fn: JFunc[A,Future[B]]): Future[JLinkedList[B]] = def traverse[A, B](in: JIterable[A], timeout: Long, fn: JFunc[A,Future[B]]): Future[JIterable[B]] =
scala.collection.JavaConversions.iterableAsScalaIterable(in).foldLeft(Future(new JLinkedList[B]())){(fr, a) => scala.collection.JavaConversions.iterableAsScalaIterable(in).foldLeft(Future(new JLinkedList[B]())){(fr, a) =>
val fb = fn(a) val fb = fn(a)
for (r <- fr; b <- fb) yield { for (r <- fr; b <- fb) yield {
@ -193,15 +196,17 @@ object Futures {
/** /**
* Java API. * Java API.
* Transforms a java.lang.Iterable[A] into a Future[java.util.LinkedList[B]] using the provided Function A => Future[B]. * Transforms a java.lang.Iterable[A] into a Future[java.lang.Iterable[B]] using the provided Function A => Future[B].
* This is useful for performing a parallel map. For example, to apply a function to all items of a list * This is useful for performing a parallel map. For example, to apply a function to all items of a list
* in parallel. * in parallel.
*/ *
def traverse[A, B, M[_] <: Traversable[_]](in: M[A], timeout: Long = Actor.TIMEOUT)(fn: A => Future[B])(implicit cbf: CanBuildFrom[M[A], B, M[B]]): Future[M[B]] = def traverse[A, B, M[_] <: Traversable[_]](in: M[A], timeout: Long = Actor.TIMEOUT)(fn: A => Future[B])(implicit cbf: CanBuildFrom[M[A], B, M[B]]): Future[M[B]] =
in.foldLeft(new DefaultCompletableFuture[Builder[B, M[B]]](timeout).completeWithResult(cbf(in)): Future[Builder[B, M[B]]]) { (fr, a) => in.foldLeft(new DefaultCompletableFuture[Builder[B, M[B]]](timeout).completeWithResult(cbf(in)): Future[Builder[B, M[B]]]) { (fr, a) =>
val fb = fn(a.asInstanceOf[A]) val fb = fn(a.asInstanceOf[A])
for (r <- fr; b <-fb) yield (r += b) for (r <- fr; b <-fb) yield (r += b)
}.map(_.result) }.map(_.result)
*/
def traverse[A, B](in: JIterable[A], fn: JFunc[A,Future[B]]): Future[JIterable[B]] = traverse(in, Actor.TIMEOUT, fn)
} }
object Future { object Future {
@ -210,18 +215,15 @@ object Future {
* This method constructs and returns a Future that will eventually hold the result of the execution of the supplied body * This method constructs and returns a Future that will eventually hold the result of the execution of the supplied body
* The execution is performed by the specified Dispatcher. * The execution is performed by the specified Dispatcher.
*/ */
def apply[T](body: => T, timeout: Long = Actor.TIMEOUT)(implicit dispatcher: MessageDispatcher): Future[T] = { def apply[T](body: => T, timeout: Long = Actor.TIMEOUT)(implicit dispatcher: MessageDispatcher): Future[T] =
val f = new DefaultCompletableFuture[T](timeout) dispatcher.dispatchFuture(() => body, timeout)
dispatcher.dispatchFuture(FutureInvocation(f.asInstanceOf[CompletableFuture[Any]], () => body))
f
}
/** /**
* Construct a completable channel * Construct a completable channel
*/ */
def channel(timeout: Long = Actor.TIMEOUT) = new Channel[Any] { def channel(timeout: Long = Actor.TIMEOUT) = new Channel[Any] {
val future = empty[Any](timeout) val future = empty[Any](timeout)
def !(msg: Any) = future << msg def !(msg: Any) = future completeWithResult msg
} }
/** /**
@ -252,22 +254,60 @@ object Future {
val fb = fn(a.asInstanceOf[A]) val fb = fn(a.asInstanceOf[A])
for (r <- fr; b <-fb) yield (r += b) for (r <- fr; b <-fb) yield (r += b)
}.map(_.result) }.map(_.result)
/**
* Captures a block that will be transformed into 'Continuation Passing Style' using Scala's Delimited
* Continuations plugin.
*
* Within the block, the result of a Future may be accessed by calling Future.apply. At that point
* execution is suspended with the rest of the block being stored in a continuation until the result
* of the Future is available. If an Exception is thrown while processing, it will be contained
* within the resulting Future.
*
* This allows working with Futures in an imperative style without blocking for each result.
*
* Completing a Future using 'CompletableFuture << Future' will also suspend execution until the
* value of the other Future is available.
*
* The Delimited Continuations compiler plugin must be enabled in order to use this method.
*/
def flow[A](body: => A @cps[Future[Any]], timeout: Long = Actor.TIMEOUT): Future[A] = {
val future = Promise[A](timeout)
(reset(future.asInstanceOf[CompletableFuture[Any]].completeWithResult(body)): Future[Any]) onComplete { f =>
val opte = f.exception
if (opte.isDefined) future completeWithException (opte.get)
}
future
}
private[akka] val callbacksPendingExecution = new ThreadLocal[Option[Stack[() => Unit]]]() {
override def initialValue = None
}
} }
sealed trait Future[+T] { sealed trait Future[+T] {
/** /**
* Returns the result of this future after waiting for it to complete, * For use only within a Future.flow block or another compatible Delimited Continuations reset block.
* this method will throw any throwable that this Future was completed with *
* and will throw a java.util.concurrent.TimeoutException if there is no result * Returns the result of this Future without blocking, by suspending execution and storing it as a
* within the Futures timeout * 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
* an untyped Future.
*/ */
def apply(): T = this.await.resultOrException.get def apply[A >: T](): A @cps[Future[Any]] = shift(this flatMap (_: A => Future[Any]))
/** /**
* Java API for apply() * Blocks awaiting completion of this Future, then returns the resulting value,
* or throws the completed exception
*
* Scala & Java API
*
* throws FutureTimeoutException if this Future times out when waiting for completion
*/ */
def get: T = apply() def get: T = this.await.resultOrException.get
/** /**
* Blocks the current thread until the Future has been completed or the * Blocks the current thread until the Future has been completed or the
@ -276,11 +316,20 @@ sealed trait Future[+T] {
*/ */
def await : Future[T] def await : Future[T]
/**
* Blocks the current thread until the Future has been completed or the
* timeout has expired. The timeout will be the least value of 'atMost' and the timeout
* supplied at the constructuion of this Future.
* In the case of the timeout expiring a FutureTimeoutException will be thrown.
*/
def await(atMost: Duration) : Future[T]
/** /**
* Blocks the current thread until the Future has been completed. Use * Blocks the current thread until the Future has been completed. Use
* caution with this method as it ignores the timeout and will block * caution with this method as it ignores the timeout and will block
* indefinitely if the Future is never completed. * indefinitely if the Future is never completed.
*/ */
@deprecated("Will be removed after 1.1, it's dangerous and can cause deadlocks, agony and insanity.", "1.1")
def awaitBlocking : Future[T] def awaitBlocking : Future[T]
/** /**
@ -318,24 +367,6 @@ sealed trait Future[+T] {
else None else None
} }
/**
* Waits for the completion of this Future, then returns the completed value.
* If the Future's timeout expires while waiting a FutureTimeoutException
* will be thrown.
*
* Equivalent to calling future.await.value.
*/
def awaitValue: Option[Either[Throwable, T]]
/**
* Returns the result of the Future if one is available within the specified
* time, if the time left on the future is less than the specified time, the
* time left on the future will be used instead of the specified time.
* returns None if no result, Some(Right(t)) if a result, or
* Some(Left(error)) if there was an exception
*/
def valueWithin(time: Long, unit: TimeUnit): Option[Either[Throwable, T]]
/** /**
* Returns the contained exception of this Future if it exists. * Returns the contained exception of this Future if it exists.
*/ */
@ -387,21 +418,18 @@ sealed trait Future[+T] {
final def collect[A](pf: PartialFunction[Any, A]): Future[A] = { final def collect[A](pf: PartialFunction[Any, A]): Future[A] = {
val fa = new DefaultCompletableFuture[A](timeoutInNanos, NANOS) val fa = new DefaultCompletableFuture[A](timeoutInNanos, NANOS)
onComplete { ft => onComplete { ft =>
val optv = ft.value val v = ft.value.get
if (optv.isDefined) { fa complete {
val v = optv.get if (v.isLeft) v.asInstanceOf[Either[Throwable, A]]
fa complete { else {
if (v.isLeft) v.asInstanceOf[Either[Throwable, A]] try {
else { val r = v.right.get
try { if (pf isDefinedAt r) Right(pf(r))
val r = v.right.get else Left(new MatchError(r))
if (pf isDefinedAt r) Right(pf(r)) } catch {
else Left(new MatchError(r)) case e: Exception =>
} catch { EventHandler.error(e, this, e.getMessage)
case e: Exception => Left(e)
EventHandler.error(e, this, e.getMessage)
Left(e)
}
} }
} }
} }
@ -409,6 +437,36 @@ sealed trait Future[+T] {
fa fa
} }
/**
* Creates a new Future that will handle any matching Throwable that this
* Future might contain. If there is no match, or if this Future contains
* a valid result then the new Future will contain the same.
* Example:
* <pre>
* Future(6 / 0) failure { case e: ArithmeticException => 0 } // result: 0
* Future(6 / 0) failure { case e: NotFoundException => 0 } // result: exception
* Future(6 / 2) failure { case e: ArithmeticException => 0 } // result: 3
* </pre>
*/
final def failure[A >: T](pf: PartialFunction[Throwable, A]): Future[A] = {
val fa = new DefaultCompletableFuture[A](timeoutInNanos, NANOS)
onComplete { ft =>
val opte = ft.exception
fa complete {
if (opte.isDefined) {
val e = opte.get
try {
if (pf isDefinedAt e) Right(pf(e))
else Left(e)
} catch {
case x: Exception => Left(x)
}
} else ft.value.get
}
}
fa
}
/** /**
* Creates a new Future by applying a function to the successful result of * Creates a new Future by applying a function to the successful result of
* this Future. If this Future is completed with an exception then the new * this Future. If this Future is completed with an exception then the new
@ -486,7 +544,7 @@ sealed trait Future[+T] {
f(optr.get) f(optr.get)
} }
final def filter(p: T => Boolean): Future[T] = { final def filter(p: Any => Boolean): Future[Any] = {
val f = new DefaultCompletableFuture[T](timeoutInNanos, NANOS) val f = new DefaultCompletableFuture[T](timeoutInNanos, NANOS)
onComplete { ft => onComplete { ft =>
val optv = ft.value val optv = ft.value
@ -531,7 +589,15 @@ sealed trait Future[+T] {
final def foreach[A >: T](proc: Procedure[A]): Unit = foreach(proc(_)) final def foreach[A >: T](proc: Procedure[A]): Unit = foreach(proc(_))
final def filter[A >: T](p: JFunc[A,Boolean]): Future[T] = filter(p(_)) final def filter(p: JFunc[Any,Boolean]): Future[Any] = filter(p(_))
}
object Promise {
def apply[A](timeout: Long): CompletableFuture[A] = new DefaultCompletableFuture[A](timeout)
def apply[A](): CompletableFuture[A] = apply(Actor.TIMEOUT)
} }
@ -567,15 +633,22 @@ trait CompletableFuture[T] extends Future[T] {
this this
} }
/** final def << (value: T): Future[T] @cps[Future[Any]] = shift { cont: (Future[T] => Future[Any]) => cont(complete(Right(value))) }
* Alias for complete(Right(value)).
*/ final def << (other: Future[T]): Future[T] @cps[Future[Any]] = shift { cont: (Future[T] => Future[Any]) =>
final def << (value: T): Future[T] = complete(Right(value)) val fr = new DefaultCompletableFuture[Any](Actor.TIMEOUT)
this completeWith other onComplete { f =>
try {
fr completeWith cont(f)
} catch {
case e: Exception =>
EventHandler.error(e, this, e.getMessage)
fr completeWithException e
}
}
fr
}
/**
* Alias for completeWith(other).
*/
final def << (other : Future[T]): Future[T] = completeWith(other)
} }
/** /**
@ -594,45 +667,34 @@ class DefaultCompletableFuture[T](timeout: Long, timeunit: TimeUnit) extends Com
private var _value: Option[Either[Throwable, T]] = None private var _value: Option[Either[Throwable, T]] = None
private var _listeners: List[Future[T] => Unit] = Nil private var _listeners: List[Future[T] => Unit] = Nil
/**
* Must be called inside _lock.lock<->_lock.unlock
*/
@tailrec @tailrec
private def awaitUnsafe(wait: Long): Boolean = { private def awaitUnsafe(waitTimeNanos: Long): Boolean = {
if (_value.isEmpty && wait > 0) { if (_value.isEmpty && waitTimeNanos > 0) {
val start = currentTimeInNanos val start = currentTimeInNanos
val remaining = try { val remainingNanos = try {
_signal.awaitNanos(wait) _signal.awaitNanos(waitTimeNanos)
} catch { } catch {
case e: InterruptedException => case e: InterruptedException =>
wait - (currentTimeInNanos - start) waitTimeNanos - (currentTimeInNanos - start)
} }
awaitUnsafe(remaining) awaitUnsafe(remainingNanos)
} else { } else {
_value.isDefined _value.isDefined
} }
} }
def awaitValue: Option[Either[Throwable, T]] = { def await(atMost: Duration) = {
_lock.lock _lock.lock
try { if (try { awaitUnsafe(atMost.toNanos min timeLeft()) } finally { _lock.unlock }) this
awaitUnsafe(timeoutInNanos - (currentTimeInNanos - _startTimeInNanos)) else throw new FutureTimeoutException("Futures timed out after [" + NANOS.toMillis(timeoutInNanos) + "] milliseconds")
_value
} finally {
_lock.unlock
}
}
def valueWithin(time: Long, unit: TimeUnit): Option[Either[Throwable, T]] = {
_lock.lock
try {
awaitUnsafe(unit.toNanos(time).min(timeoutInNanos - (currentTimeInNanos - _startTimeInNanos)))
_value
} finally {
_lock.unlock
}
} }
def await = { def await = {
_lock.lock _lock.lock
if (try { awaitUnsafe(timeoutInNanos - (currentTimeInNanos - _startTimeInNanos)) } finally { _lock.unlock }) this if (try { awaitUnsafe(timeLeft()) } finally { _lock.unlock }) this
else throw new FutureTimeoutException("Futures timed out after [" + NANOS.toMillis(timeoutInNanos) + "] milliseconds") else throw new FutureTimeoutException("Futures timed out after [" + NANOS.toMillis(timeoutInNanos) + "] milliseconds")
} }
@ -648,7 +710,7 @@ class DefaultCompletableFuture[T](timeout: Long, timeunit: TimeUnit) extends Com
} }
} }
def isExpired: Boolean = timeoutInNanos - (currentTimeInNanos - _startTimeInNanos) <= 0 def isExpired: Boolean = timeLeft() <= 0
def value: Option[Either[Throwable, T]] = { def value: Option[Either[Throwable, T]] = {
_lock.lock _lock.lock
@ -662,7 +724,7 @@ class DefaultCompletableFuture[T](timeout: Long, timeunit: TimeUnit) extends Com
def complete(value: Either[Throwable, T]): DefaultCompletableFuture[T] = { def complete(value: Either[Throwable, T]): DefaultCompletableFuture[T] = {
_lock.lock _lock.lock
val notifyTheseListeners = try { val notifyTheseListeners = try {
if (_value.isEmpty) { if (_value.isEmpty && !isExpired) { //Only complete if we aren't expired
_value = Some(value) _value = Some(value)
val existingListeners = _listeners val existingListeners = _listeners
_listeners = Nil _listeners = Nil
@ -673,8 +735,29 @@ class DefaultCompletableFuture[T](timeout: Long, timeunit: TimeUnit) extends Com
_lock.unlock _lock.unlock
} }
if (notifyTheseListeners.nonEmpty) if (notifyTheseListeners.nonEmpty) { // Steps to ensure we don't run into a stack-overflow situation
notifyTheseListeners.reverse foreach notify @tailrec def runCallbacks(rest: List[Future[T] => Unit], callbacks: Stack[() => Unit]) {
if (rest.nonEmpty) {
notifyCompleted(rest.head)
while (callbacks.nonEmpty) { callbacks.pop().apply() }
runCallbacks(rest.tail, callbacks)
}
}
val pending = Future.callbacksPendingExecution.get
if (pending.isDefined) { //Instead of nesting the calls to the callbacks (leading to stack overflow)
pending.get.push(() => { // Linearize/aggregate callbacks at top level and then execute
val doNotify = notifyCompleted _ //Hoist closure to avoid garbage
notifyTheseListeners foreach doNotify
})
} else {
try {
val callbacks = Stack[() => Unit]() // Allocate new aggregator for pending callbacks
Future.callbacksPendingExecution.set(Some(callbacks)) // Specify the callback aggregator
runCallbacks(notifyTheseListeners, callbacks) // Execute callbacks, if they trigger new callbacks, they are aggregated
} finally { Future.callbacksPendingExecution.set(None) } // Ensure cleanup
}
}
this this
} }
@ -683,19 +766,21 @@ class DefaultCompletableFuture[T](timeout: Long, timeunit: TimeUnit) extends Com
_lock.lock _lock.lock
val notifyNow = try { val notifyNow = try {
if (_value.isEmpty) { if (_value.isEmpty) {
_listeners ::= func if(!isExpired) { //Only add the listener if the future isn't expired
false _listeners ::= func
false
} else false //Will never run the callback since the future is expired
} else true } else true
} finally { } finally {
_lock.unlock _lock.unlock
} }
if (notifyNow) notify(func) if (notifyNow) notifyCompleted(func)
this this
} }
private def notify(func: Future[T] => Unit) { private def notifyCompleted(func: Future[T] => Unit) {
try { try {
func(this) func(this)
} catch { } catch {
@ -703,7 +788,8 @@ class DefaultCompletableFuture[T](timeout: Long, timeunit: TimeUnit) extends Com
} }
} }
private def currentTimeInNanos: Long = MILLIS.toNanos(System.currentTimeMillis) @inline private def currentTimeInNanos: Long = MILLIS.toNanos(System.currentTimeMillis)
@inline private def timeLeft(): Long = timeoutInNanos - (currentTimeInNanos - _startTimeInNanos)
} }
/** /**
@ -715,8 +801,7 @@ sealed class AlreadyCompletedFuture[T](suppliedValue: Either[Throwable, T]) exte
def complete(value: Either[Throwable, T]): CompletableFuture[T] = this def complete(value: Either[Throwable, T]): CompletableFuture[T] = this
def onComplete(func: Future[T] => Unit): Future[T] = { func(this); this } def onComplete(func: Future[T] => Unit): Future[T] = { func(this); this }
def awaitValue: Option[Either[Throwable, T]] = value def await(atMost: Duration): Future[T] = this
def valueWithin(time: Long, unit: TimeUnit): Option[Either[Throwable, T]] = value
def await : Future[T] = this def await : Future[T] = this
def awaitBlocking : Future[T] = this def awaitBlocking : Future[T] = this
def isExpired: Boolean = true def isExpired: Boolean = true

View file

@ -4,14 +4,13 @@
package akka.dispatch package akka.dispatch
import akka.actor.{Actor, ActorType, ActorRef, ActorInitializationException}
import akka.AkkaException import akka.AkkaException
import java.util.{Queue, List, Comparator, PriorityQueue} import java.util.{Comparator, PriorityQueue}
import java.util.concurrent._ import java.util.concurrent._
import akka.util._ import akka.util._
class MessageQueueAppendFailedException(message: String) extends AkkaException(message) class MessageQueueAppendFailedException(message: String, cause: Throwable = null) extends AkkaException(message, cause)
/** /**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
@ -30,9 +29,8 @@ trait MessageQueue {
*/ */
sealed trait MailboxType sealed trait MailboxType
case class UnboundedMailbox(val blocking: Boolean = false) extends MailboxType case class UnboundedMailbox() extends MailboxType
case class BoundedMailbox( case class BoundedMailbox(
val blocking: Boolean = false,
val capacity: Int = { if (Dispatchers.MAILBOX_CAPACITY < 0) Int.MaxValue else Dispatchers.MAILBOX_CAPACITY }, val capacity: Int = { if (Dispatchers.MAILBOX_CAPACITY < 0) Int.MaxValue else Dispatchers.MAILBOX_CAPACITY },
val pushTimeOut: Duration = Dispatchers.MAILBOX_PUSH_TIME_OUT) extends MailboxType { val pushTimeOut: Duration = Dispatchers.MAILBOX_PUSH_TIME_OUT) extends MailboxType {
if (capacity < 0) throw new IllegalArgumentException("The capacity for BoundedMailbox can not be negative") if (capacity < 0) throw new IllegalArgumentException("The capacity for BoundedMailbox can not be negative")
@ -40,46 +38,35 @@ case class BoundedMailbox(
} }
trait UnboundedMessageQueueSemantics extends MessageQueue { self: BlockingQueue[MessageInvocation] => trait UnboundedMessageQueueSemantics extends MessageQueue { self: BlockingQueue[MessageInvocation] =>
def blockDequeue: Boolean @inline final def enqueue(handle: MessageInvocation): Unit = this add handle
@inline final def dequeue(): MessageInvocation = this.poll()
final def enqueue(handle: MessageInvocation) {
this add handle
}
final def dequeue(): MessageInvocation = {
if (blockDequeue) this.take()
else this.poll()
}
} }
trait BoundedMessageQueueSemantics extends MessageQueue { self: BlockingQueue[MessageInvocation] => trait BoundedMessageQueueSemantics extends MessageQueue { self: BlockingQueue[MessageInvocation] =>
def blockDequeue: Boolean
def pushTimeOut: Duration def pushTimeOut: Duration
final def enqueue(handle: MessageInvocation) { final def enqueue(handle: MessageInvocation) {
if (pushTimeOut.length > 0 && pushTimeOut.toMillis > 0) { if (pushTimeOut.length > 0) {
if (!this.offer(handle, pushTimeOut.length, pushTimeOut.unit)) this.offer(handle, pushTimeOut.length, pushTimeOut.unit) || {
throw new MessageQueueAppendFailedException("Couldn't enqueue message " + handle + " to " + toString) throw new MessageQueueAppendFailedException("Couldn't enqueue message " + handle + " to " + toString) }
} else this put handle } else this put handle
} }
final def dequeue(): MessageInvocation = @inline final def dequeue(): MessageInvocation = this.poll()
if (blockDequeue) this.take()
else this.poll()
} }
class DefaultUnboundedMessageQueue(val blockDequeue: Boolean) extends class DefaultUnboundedMessageQueue extends
LinkedBlockingQueue[MessageInvocation] with LinkedBlockingQueue[MessageInvocation] with
UnboundedMessageQueueSemantics UnboundedMessageQueueSemantics
class DefaultBoundedMessageQueue(capacity: Int, val pushTimeOut: Duration, val blockDequeue: Boolean) extends class DefaultBoundedMessageQueue(capacity: Int, val pushTimeOut: Duration) extends
LinkedBlockingQueue[MessageInvocation](capacity) with LinkedBlockingQueue[MessageInvocation](capacity) with
BoundedMessageQueueSemantics BoundedMessageQueueSemantics
class UnboundedPriorityMessageQueue(val blockDequeue: Boolean, cmp: Comparator[MessageInvocation]) extends class UnboundedPriorityMessageQueue(cmp: Comparator[MessageInvocation]) extends
PriorityBlockingQueue[MessageInvocation](11, cmp) with PriorityBlockingQueue[MessageInvocation](11, cmp) with
UnboundedMessageQueueSemantics UnboundedMessageQueueSemantics
class BoundedPriorityMessageQueue(capacity: Int, val pushTimeOut: Duration, val blockDequeue: Boolean, cmp: Comparator[MessageInvocation]) extends class BoundedPriorityMessageQueue(capacity: Int, val pushTimeOut: Duration, cmp: Comparator[MessageInvocation]) extends
BoundedBlockingQueue[MessageInvocation](capacity, new PriorityQueue[MessageInvocation](11, cmp)) with BoundedBlockingQueue[MessageInvocation](capacity, new PriorityQueue[MessageInvocation](11, cmp)) with
BoundedMessageQueueSemantics BoundedMessageQueueSemantics

View file

@ -5,11 +5,11 @@
package akka.dispatch package akka.dispatch
import java.util.concurrent._ import java.util.concurrent._
import atomic. {AtomicInteger, AtomicBoolean, AtomicReference, AtomicLong} import java.util.concurrent.atomic.AtomicLong
import akka.event.EventHandler import akka.event.EventHandler
import akka.config.Configuration import akka.config.Configuration
import akka.config.Config.TIME_UNIT import akka.config.Config.TIME_UNIT
import akka.util.{Duration, Switch, ReentrantGuard, HashCode, ReflectiveAccess} import akka.util.{Duration, Switch, ReentrantGuard}
import java.util.concurrent.ThreadPoolExecutor.{AbortPolicy, CallerRunsPolicy, DiscardOldestPolicy, DiscardPolicy} import java.util.concurrent.ThreadPoolExecutor.{AbortPolicy, CallerRunsPolicy, DiscardOldestPolicy, DiscardPolicy}
import akka.actor._ import akka.actor._
@ -30,16 +30,18 @@ final case class MessageInvocation(val receiver: ActorRef,
} }
} }
final case class FutureInvocation(future: CompletableFuture[Any], function: () => Any) extends Runnable { final case class FutureInvocation[T](future: CompletableFuture[T], function: () => T, cleanup: () => Unit) extends Runnable {
val uuid = akka.actor.newUuid def run = {
future complete (try {
def run = future complete (try { Right(function())
Right(function.apply) } catch {
} catch { case e =>
case e => EventHandler.error(e, this, e.getMessage)
EventHandler.error(e, this, e.getMessage) Left(e)
Left(e) } finally {
}) cleanup()
})
}
} }
object MessageDispatcher { object MessageDispatcher {
@ -57,7 +59,7 @@ trait MessageDispatcher {
import MessageDispatcher._ import MessageDispatcher._
protected val uuids = new ConcurrentSkipListSet[Uuid] protected val uuids = new ConcurrentSkipListSet[Uuid]
protected val futures = new ConcurrentSkipListSet[Uuid] protected val futures = new AtomicLong(0L)
protected val guard = new ReentrantGuard protected val guard = new ReentrantGuard
protected val active = new Switch(false) protected val active = new Switch(false)
@ -84,15 +86,27 @@ trait MessageDispatcher {
private[akka] final def dispatchMessage(invocation: MessageInvocation): Unit = dispatch(invocation) private[akka] final def dispatchMessage(invocation: MessageInvocation): Unit = dispatch(invocation)
private[akka] final def dispatchFuture(invocation: FutureInvocation): Unit = { private[akka] final def dispatchFuture[T](block: () => T, timeout: Long): Future[T] = {
guard withGuard { futures.getAndIncrement()
futures add invocation.uuid try {
if (active.isOff) { active.switchOn { start } } val future = new DefaultCompletableFuture[T](timeout)
if (active.isOff)
guard withGuard { active.switchOn { start } }
executeFuture(FutureInvocation[T](future, block, futureCleanup))
future
} catch {
case e =>
futures.decrementAndGet
throw e
} }
invocation.future.onComplete { f => }
private val futureCleanup: () => Unit =
() => if (futures.decrementAndGet() == 0) {
guard withGuard { guard withGuard {
futures remove invocation.uuid if (futures.get == 0 && uuids.isEmpty) {
if (futures.isEmpty && uuids.isEmpty) {
shutdownSchedule match { shutdownSchedule match {
case UNSCHEDULED => case UNSCHEDULED =>
shutdownSchedule = SCHEDULED shutdownSchedule = SCHEDULED
@ -104,8 +118,6 @@ trait MessageDispatcher {
} }
} }
} }
executeFuture(invocation)
}
private[akka] def register(actorRef: ActorRef) { private[akka] def register(actorRef: ActorRef) {
if (actorRef.mailbox eq null) if (actorRef.mailbox eq null)
@ -122,7 +134,7 @@ trait MessageDispatcher {
private[akka] def unregister(actorRef: ActorRef) = { private[akka] def unregister(actorRef: ActorRef) = {
if (uuids remove actorRef.uuid) { if (uuids remove actorRef.uuid) {
actorRef.mailbox = null actorRef.mailbox = null
if (uuids.isEmpty && futures.isEmpty){ if (uuids.isEmpty && futures.get == 0){
shutdownSchedule match { shutdownSchedule match {
case UNSCHEDULED => case UNSCHEDULED =>
shutdownSchedule = SCHEDULED shutdownSchedule = SCHEDULED
@ -156,7 +168,7 @@ trait MessageDispatcher {
shutdownSchedule = SCHEDULED shutdownSchedule = SCHEDULED
Scheduler.scheduleOnce(this, timeoutMs, TimeUnit.MILLISECONDS) Scheduler.scheduleOnce(this, timeoutMs, TimeUnit.MILLISECONDS)
case SCHEDULED => case SCHEDULED =>
if (uuids.isEmpty() && futures.isEmpty) { if (uuids.isEmpty && futures.get == 0) {
active switchOff { active switchOff {
shutdown // shut down in the dispatcher's references is zero shutdown // shut down in the dispatcher's references is zero
} }
@ -188,17 +200,17 @@ trait MessageDispatcher {
*/ */
private[akka] def dispatch(invocation: MessageInvocation): Unit private[akka] def dispatch(invocation: MessageInvocation): Unit
private[akka] def executeFuture(invocation: FutureInvocation): Unit private[akka] def executeFuture(invocation: FutureInvocation[_]): Unit
/** /**
* Called one time every time an actor is attached to this dispatcher and this dispatcher was previously shutdown * Called one time every time an actor is attached to this dispatcher and this dispatcher was previously shutdown
*/ */
private[akka] def start: Unit private[akka] def start(): Unit
/** /**
* Called one time every time an actor is detached from this dispatcher and this dispatcher has no actors left attached * Called one time every time an actor is detached from this dispatcher and this dispatcher has no actors left attached
*/ */
private[akka] def shutdown: Unit private[akka] def shutdown(): Unit
/** /**
* Returns the size of the mailbox for the specified actor * Returns the size of the mailbox for the specified actor
@ -206,9 +218,9 @@ trait MessageDispatcher {
def mailboxSize(actorRef: ActorRef): Int def mailboxSize(actorRef: ActorRef): Int
/** /**
* Returns the size of the Future queue * Returns the amount of futures queued for execution
*/ */
def futureQueueSize: Int = futures.size def pendingFutures: Long = futures.get
} }
/** /**
@ -222,9 +234,8 @@ abstract class MessageDispatcherConfigurator {
def mailboxType(config: Configuration): MailboxType = { def mailboxType(config: Configuration): MailboxType = {
val capacity = config.getInt("mailbox-capacity", Dispatchers.MAILBOX_CAPACITY) val capacity = config.getInt("mailbox-capacity", Dispatchers.MAILBOX_CAPACITY)
// FIXME how do we read in isBlocking for mailbox? Now set to 'false'.
if (capacity < 1) UnboundedMailbox() if (capacity < 1) UnboundedMailbox()
else BoundedMailbox(false, capacity, Duration(config.getInt("mailbox-push-timeout-time", Dispatchers.MAILBOX_PUSH_TIME_OUT.toMillis.toInt), TIME_UNIT)) else BoundedMailbox(capacity, Duration(config.getInt("mailbox-push-timeout-time", Dispatchers.MAILBOX_PUSH_TIME_OUT.toMillis.toInt), TIME_UNIT))
} }
def configureThreadPool(config: Configuration, createDispatcher: => (ThreadPoolConfig) => MessageDispatcher): ThreadPoolConfigDispatcherBuilder = { def configureThreadPool(config: Configuration, createDispatcher: => (ThreadPoolConfig) => MessageDispatcher): ThreadPoolConfigDispatcherBuilder = {

View file

@ -4,13 +4,9 @@
package akka.dispatch package akka.dispatch
import akka.actor.{Actor, ActorRef} import akka.actor.{ActorRef}
import akka.config.Config.config
import akka.util.Duration import akka.util.Duration
import java.util.Queue
import java.util.concurrent.{ConcurrentLinkedQueue, BlockingQueue, TimeUnit, LinkedBlockingQueue}
import akka.actor
import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.atomic.AtomicReference
/** /**
@ -25,13 +21,13 @@ class ThreadBasedDispatcher(_actor: ActorRef, _mailboxType: MailboxType)
private[akka] val owner = new AtomicReference[ActorRef](_actor) private[akka] val owner = new AtomicReference[ActorRef](_actor)
def this(actor: ActorRef) = def this(actor: ActorRef) =
this(actor, UnboundedMailbox(true)) // For Java API this(actor, UnboundedMailbox()) // For Java API
def this(actor: ActorRef, capacity: Int) = def this(actor: ActorRef, capacity: Int) =
this(actor, BoundedMailbox(true, capacity)) //For Java API this(actor, BoundedMailbox(capacity)) //For Java API
def this(actor: ActorRef, capacity: Int, pushTimeOut: Duration) = //For Java API def this(actor: ActorRef, capacity: Int, pushTimeOut: Duration) = //For Java API
this(actor, BoundedMailbox(true, capacity, pushTimeOut)) this(actor, BoundedMailbox(capacity, pushTimeOut))
override def register(actorRef: ActorRef) = { override def register(actorRef: ActorRef) = {
val actor = owner.get() val actor = owner.get()

View file

@ -218,9 +218,9 @@ trait ExecutorServiceDelegate extends ExecutorService {
def execute(command: Runnable) = executor.execute(command) def execute(command: Runnable) = executor.execute(command)
def shutdown = executor.shutdown def shutdown() { executor.shutdown() }
def shutdownNow = executor.shutdownNow def shutdownNow() = executor.shutdownNow()
def isShutdown = executor.isShutdown def isShutdown = executor.isShutdown

View file

@ -109,15 +109,15 @@ object EventHandler extends ListenerManagement {
} }
defaultListeners foreach { listenerName => defaultListeners foreach { listenerName =>
try { try {
ReflectiveAccess.getClassFor[Actor](listenerName) map { clazz => ReflectiveAccess.getClassFor[Actor](listenerName) match {
addListener(Actor.actorOf(clazz, listenerName).start()) case r: Right[_, Class[Actor]] => addListener(Actor.actorOf(r.b, listenerName).start())
case l: Left[Exception,_] => throw l.a
} }
} catch { } catch {
case e: akka.actor.DeploymentAlreadyBoundException => // do nothing
case e: Exception => case e: Exception =>
throw new ConfigurationException( throw new ConfigurationException(
"Event Handler specified in config can't be loaded [" + listenerName + "Event Handler specified in config can't be loaded [" + listenerName +
"] due to [" + e.toString + "]") "] due to [" + e.toString + "]", e)
} }
} }
info(this, "Starting up EventHandler") info(this, "Starting up EventHandler")
@ -132,8 +132,8 @@ object EventHandler extends ListenerManagement {
* Shuts down all event handler listeners including the event handle dispatcher. * Shuts down all event handler listeners including the event handle dispatcher.
*/ */
def shutdown() { def shutdown() {
foreachListener(_.stop) foreachListener(_.stop())
EventHandlerDispatcher.shutdown EventHandlerDispatcher.shutdown()
} }
def notify(event: Any) { def notify(event: Any) {

View file

@ -16,6 +16,7 @@ import scala.reflect.BeanProperty
import java.net.InetSocketAddress import java.net.InetSocketAddress
import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.ConcurrentHashMap
import java.io.{PrintWriter, PrintStream} import java.io.{PrintWriter, PrintStream}
import java.lang.reflect.InvocationTargetException
trait RemoteModule { trait RemoteModule {
val UUID_PREFIX = "uuid:".intern val UUID_PREFIX = "uuid:".intern
@ -117,7 +118,7 @@ case class RemoteServerWriteFailed(
class RemoteClientException private[akka] ( class RemoteClientException private[akka] (
message: String, message: String,
@BeanProperty val client: RemoteClientModule, @BeanProperty val client: RemoteClientModule,
val remoteAddress: InetSocketAddress) extends AkkaException(message) val remoteAddress: InetSocketAddress, cause: Throwable = null) extends AkkaException(message, cause)
/** /**
* Thrown when the remote server actor dispatching fails for some reason. * Thrown when the remote server actor dispatching fails for some reason.
@ -146,7 +147,7 @@ abstract class RemoteSupport extends ListenerManagement with RemoteServerModule
handler handler
} }
def shutdown { def shutdown() {
eventHandler.stop() eventHandler.stop()
removeListener(eventHandler) removeListener(eventHandler)
this.shutdownClientModule() this.shutdownClientModule()
@ -282,7 +283,8 @@ trait RemoteServerModule extends RemoteModule {
def registerByUuid(actorRef: ActorRef): Unit def registerByUuid(actorRef: ActorRef): Unit
/** /**
* Register Remote Actor by a specific 'id' passed as argument. * Register Remote Actor by a specific 'id' passed as argument. The actor is registered by UUID rather than ID
* when prefixing the handle with the uuid: protocol.
* <p/> * <p/>
* NOTE: If you use this method to register your remote actor then you must unregister the actor by this ID yourself. * NOTE: If you use this method to register your remote actor then you must unregister the actor by this ID yourself.
*/ */

View file

@ -5,7 +5,6 @@
package akka.routing package akka.routing
import akka.actor.{Actor, ActorRef, PoisonPill} import akka.actor.{Actor, ActorRef, PoisonPill}
import java.util.concurrent.TimeUnit
/** /**
* Actor pooling * Actor pooling
@ -47,14 +46,13 @@ trait ActorPool {
*/ */
trait DefaultActorPool extends ActorPool { this: Actor => trait DefaultActorPool extends ActorPool { this: Actor =>
import ActorPool._ import ActorPool._
import collection.mutable.LinkedList
import akka.actor.MaximumNumberOfRestartsWithinTimeRangeReached import akka.actor.MaximumNumberOfRestartsWithinTimeRangeReached
protected var _delegates = Vector[ActorRef]() protected var _delegates = Vector[ActorRef]()
private var _lastCapacityChange = 0 private var _lastCapacityChange = 0
private var _lastSelectorCount = 0 private var _lastSelectorCount = 0
override def postStop = _delegates foreach { override def postStop() = _delegates foreach {
delegate => try { delegate => try {
delegate ! PoisonPill delegate ! PoisonPill
} catch { case e: Exception => } //Ignore any exceptions here } catch { case e: Exception => } //Ignore any exceptions here

View file

@ -21,7 +21,7 @@ class AkkaLoader {
* Boot initializes the specified bundles * Boot initializes the specified bundles
*/ */
def boot(withBanner: Boolean, b : Bootable): Unit = hasBooted switchOn { def boot(withBanner: Boolean, b : Bootable): Unit = hasBooted switchOn {
if (withBanner) printBanner if (withBanner) printBanner()
println("Starting Akka...") println("Starting Akka...")
b.onLoad b.onLoad
Thread.currentThread.setContextClassLoader(getClass.getClassLoader) Thread.currentThread.setContextClassLoader(getClass.getClassLoader)
@ -32,40 +32,62 @@ class AkkaLoader {
/* /*
* Shutdown, well, shuts down the bundles used in boot * Shutdown, well, shuts down the bundles used in boot
*/ */
def shutdown: Unit = hasBooted switchOff { def shutdown() {
println("Shutting down Akka...") hasBooted switchOff {
_bundles.foreach(_.onUnload) println("Shutting down Akka...")
_bundles = None _bundles.foreach(_.onUnload)
Actor.shutdownHook.run _bundles = None
println("Akka succesfully shut down") Actor.shutdownHook.run
println("Akka succesfully shut down")
}
} }
private def printBanner = { private def printBanner() {
println("==================================================") println("""
println(" t") ==============================================================================
println(" t t t")
println(" t t tt t") ZZ:
println(" tt t t tt t") ZZZZ
println(" t ttttttt t ttt t") ZZZZZZ
println(" t tt ttt t ttt t") ZZZ' ZZZ
println(" t t ttt t ttt t t") ~7 7ZZ' ZZZ
println(" tt t ttt ttt ttt t") :ZZZ: IZZ' ZZZ
println(" t t ttt ttt t tt t") ,OZZZZ.~ZZ? ZZZ
println(" t ttt ttt t t") ZZZZ' 'ZZZ$ ZZZ
println(" tt ttt ttt t") . $ZZZ ~ZZ$ ZZZ
println(" ttt ttt") .=Z?. .ZZZO ~ZZ7 OZZ
println(" tttttttt ttt ttt ttt ttt tttttttt") .ZZZZ7..:ZZZ~ 7ZZZ ZZZ~
println(" ttt tt ttt ttt ttt ttt ttt ttt") .$ZZZ$Z+.ZZZZ ZZZ: ZZZ$
println(" ttt ttt ttt ttt ttt ttt ttt ttt") .,ZZZZ?' =ZZO= .OZZ 'ZZZ
println(" ttt ttt ttt ttt ttt tt ttt ttt") .$ZZZZ+ .ZZZZ IZZZ ZZZ$
println(" tttt ttttttttt tttttttt tttt") .ZZZZZ' .ZZZZ' .ZZZ$ ?ZZZ
println(" ttttttttt ttt ttt ttt ttt ttttttttt") .ZZZZZZ' .OZZZ? ?ZZZ 'ZZZ$
println(" ttt ttt ttt ttt ttt ttt ttt ttt") .?ZZZZZZ' .ZZZZ? .ZZZ? 'ZZZO
println(" ttt ttt ttt ttt ttt ttt ttt ttt") .+ZZZZZZ?' .7ZZZZ' .ZZZZ :ZZZZ
println(" ttt tt ttt ttt ttt ttt ttt ttt") .ZZZZZZ$' .?ZZZZZ' .~ZZZZ 'ZZZZ.
println(" tttttttt ttt ttt ttt ttt tttttttt")
println("==================================================")
println(" Running version {}", Config.VERSION) NNNNN $NNNN+
println("==================================================") NNNNN $NNNN+
NNNNN $NNNN+
NNNNN $NNNN+
NNNNN $NNNN+
=NNNNNNNNND$ NNNNN DDDDDD: $NNNN+ DDDDDN NDDNNNNNNNN,
NNNNNNNNNNNNND NNNNN DNNNNN $NNNN+ 8NNNNN= :NNNNNNNNNNNNNN
NNNNN$ DNNNNN NNNNN $NNNNN~ $NNNN+ NNNNNN NNNNN, :NNNNN+
?DN~ NNNNN NNNNN MNNNNN $NNNN+:NNNNN7 $ND =NNNNN
DNNNNN NNNNNDNNNN$ $NNNNDNNNNN :DNNNNN
ZNDNNNNNNNNND NNNNNNNNNND, $NNNNNNNNNNN DNDNNNNNNNNNN
NNNNNNNDDINNNNN NNNNNNNNNNND $NNNNNNNNNNND ONNNNNNND8+NNNNN
:NNNND NNNNN NNNNNN DNNNN, $NNNNNO 7NNNND NNNNNO :NNNNN
DNNNN NNNNN NNNNN DNNNN $NNNN+ 8NNNNN NNNNN $NNNNN
DNNNNO NNNNNN NNNNN NNNNN $NNNN+ NNNNN$ NNNND, ,NNNNND
NNNNNNDDNNNNNNNN NNNNN =NNNNN $NNNN+ DNNNN? DNNNNNNDNNNNNNNND
NNNNNNNNN NNNN$ NNNNN 8NNNND $NNNN+ NNNNN= ,DNNNNNNND NNNNN$
==============================================================================
Running version %s
==============================================================================
""".format(Config.VERSION))
} }
} }

View file

@ -5,6 +5,6 @@
package akka.util package akka.util
trait Bootable { trait Bootable {
def onLoad {} def onLoad() {}
def onUnload {} def onUnload() {}
} }

View file

@ -48,11 +48,26 @@ object ReflectiveAccess {
} }
} }
lazy val clusterInstance: Option[Cluster] = getObjectFor("akka.cluster.Cluster$") lazy val clusterInstance: Option[Cluster] = getObjectFor("akka.cluster.Cluster$") match {
case Right(value) => Some(value)
case Left(exception) =>
EventHandler.debug(this, exception.toString)
None
}
lazy val clusterDeployerInstance: Option[ClusterDeployer] = getObjectFor("akka.cluster.ClusterDeployer$") lazy val clusterDeployerInstance: Option[ClusterDeployer] = getObjectFor("akka.cluster.ClusterDeployer$") match {
case Right(value) => Some(value)
case Left(exception) =>
EventHandler.debug(this, exception.toString)
None
}
lazy val serializerClass: Option[Class[_]] = getClassFor("akka.serialization.Serializer") lazy val serializerClass: Option[Class[_]] = getClassFor("akka.serialization.Serializer") match {
case Right(value) => Some(value)
case Left(exception) =>
EventHandler.debug(this, exception.toString)
None
}
lazy val node: ClusterNode = { lazy val node: ClusterNode = {
ensureEnabled() ensureEnabled()
@ -123,16 +138,20 @@ object ReflectiveAccess {
lazy val isEnabled = remoteSupportClass.isDefined lazy val isEnabled = remoteSupportClass.isDefined
def ensureEnabled() { def ensureEnabled() = {
if (!isEnabled) { if (!isEnabled) {
val e = new ModuleNotAvailableException( val e = new ModuleNotAvailableException("Can't load the remoting module, make sure that akka-remote.jar is on the classpath")
"Can't load the remoting module, make sure that akka-remote.jar is on the classpath") EventHandler.debug(this, e.toString)
EventHandler.debug(this, e.toString) throw e
throw e
} }
} }
val remoteSupportClass: Option[Class[_ <: RemoteSupport]] = getClassFor(TRANSPORT) val remoteSupportClass = getClassFor[RemoteSupport](TRANSPORT) match {
case Right(value) => Some(value)
case Left(exception) =>
EventHandler.debug(this, exception.toString)
None
}
protected[akka] val defaultRemoteSupport: Option[() => RemoteSupport] = protected[akka] val defaultRemoteSupport: Option[() => RemoteSupport] =
remoteSupportClass map { remoteClass => remoteSupportClass map { remoteClass =>
@ -140,9 +159,11 @@ object ReflectiveAccess {
remoteClass, remoteClass,
Array[Class[_]](), Array[Class[_]](),
Array[AnyRef]() Array[AnyRef]()
) getOrElse { ) match {
case Right(value) => value
case Left(exception) =>
val e = new ModuleNotAvailableException( val e = new ModuleNotAvailableException(
"Can't instantiate [%s] - make sure that akka-remote.jar is on the classpath".format(remoteClass.getName)) "Can't instantiate [%s] - make sure that akka-remote.jar is on the classpath".format(remoteClass.getName), exception)
EventHandler.debug(this, e.toString) EventHandler.debug(this, e.toString)
throw e throw e
} }
@ -172,7 +193,12 @@ object ReflectiveAccess {
} }
val typedActorObjectInstance: Option[TypedActorObject] = val typedActorObjectInstance: Option[TypedActorObject] =
getObjectFor("akka.actor.TypedActor$") getObjectFor[TypedActorObject]("akka.actor.TypedActor$") match {
case Right(value) => Some(value)
case Left(exception)=>
EventHandler.debug(this, exception.toString)
None
}
def resolveFutureIfMessageIsJoinPoint(message: Any, future: Future[_]): Boolean = { def resolveFutureIfMessageIsJoinPoint(message: Any, future: Future[_]): Boolean = {
ensureEnabled() ensureEnabled()
@ -188,94 +214,93 @@ object ReflectiveAccess {
def createInstance[T](clazz: Class[_], def createInstance[T](clazz: Class[_],
params: Array[Class[_]], params: Array[Class[_]],
args: Array[AnyRef]): Option[T] = try { args: Array[AnyRef]): Either[Exception,T] = try {
assert(clazz ne null) assert(clazz ne null)
assert(params ne null) assert(params ne null)
assert(args ne null) assert(args ne null)
val ctor = clazz.getDeclaredConstructor(params: _*) val ctor = clazz.getDeclaredConstructor(params: _*)
ctor.setAccessible(true) ctor.setAccessible(true)
Some(ctor.newInstance(args: _*).asInstanceOf[T]) Right(ctor.newInstance(args: _*).asInstanceOf[T])
} catch { } catch {
case e: java.lang.reflect.InvocationTargetException => case e: java.lang.reflect.InvocationTargetException =>
EventHandler.debug(this, e.getCause.toString) EventHandler.debug(this, e.getCause.toString)
None Left(e)
case e: Exception => case e: Exception =>
EventHandler.debug(this, e.toString) EventHandler.debug(this, e.toString)
None Left(e)
} }
def createInstance[T](fqn: String, def createInstance[T](fqn: String,
params: Array[Class[_]], params: Array[Class[_]],
args: Array[AnyRef], args: Array[AnyRef],
classloader: ClassLoader = loader): Option[T] = try { classloader: ClassLoader = loader): Either[Exception,T] = try {
assert(params ne null) assert(params ne null)
assert(args ne null) assert(args ne null)
getClassFor(fqn) match { getClassFor(fqn) match {
case Some(clazz) => case Right(value) =>
val ctor = clazz.getDeclaredConstructor(params: _*) val ctor = value.getDeclaredConstructor(params: _*)
ctor.setAccessible(true) ctor.setAccessible(true)
Some(ctor.newInstance(args: _*).asInstanceOf[T]) Right(ctor.newInstance(args: _*).asInstanceOf[T])
case None => None case Left(exception) => Left(exception) //We could just cast this to Either[Exception, T] but it's ugly
} }
} catch { } catch {
case e: Exception => case e: Exception =>
EventHandler.debug(this, e.toString) Left(e)
None
} }
def getObjectFor[T](fqn: String, classloader: ClassLoader = loader): Option[T] = try {//Obtains a reference to $MODULE$ //Obtains a reference to fqn.MODULE$
def getObjectFor[T](fqn: String, classloader: ClassLoader = loader): Either[Exception,T] = try {
getClassFor(fqn) match { getClassFor(fqn) match {
case Some(clazz) => case Right(value) =>
val instance = clazz.getDeclaredField("MODULE$") val instance = value.getDeclaredField("MODULE$")
instance.setAccessible(true) instance.setAccessible(true)
Option(instance.get(null).asInstanceOf[T]) val obj = instance.get(null)
case None => None if (obj eq null) Left(new NullPointerException) else Right(obj.asInstanceOf[T])
case Left(exception) => Left(exception) //We could just cast this to Either[Exception, T] but it's ugly
} }
} catch { } catch {
case e: ExceptionInInitializerError => case e: Exception =>
EventHandler.debug(this, e.toString) Left(e)
throw e
} }
def getClassFor[T](fqn: String, classloader: ClassLoader = loader): Option[Class[T]] = { def getClassFor[T](fqn: String, classloader: ClassLoader = loader): Either[Exception,Class[T]] = try {
assert(fqn ne null) assert(fqn ne null)
// First, use the specified CL // First, use the specified CL
val first = try { val first = try {
Option(classloader.loadClass(fqn).asInstanceOf[Class[T]]) Right(classloader.loadClass(fqn).asInstanceOf[Class[T]])
} catch { } catch {
case c: ClassNotFoundException => None case c: ClassNotFoundException => Left(c)
} }
if (first.isDefined) first if (first.isRight) first
else { else {
// Second option is to use the ContextClassLoader // Second option is to use the ContextClassLoader
val second = try { val second = try {
Option(Thread.currentThread.getContextClassLoader.loadClass(fqn).asInstanceOf[Class[T]]) Right(Thread.currentThread.getContextClassLoader.loadClass(fqn).asInstanceOf[Class[T]])
} catch { } catch {
case c: ClassNotFoundException => None case c: ClassNotFoundException => Left(c)
} }
if (second.isDefined) second if (second.isRight) second
else { else {
val third = try { val third = try {
// Don't try to use "loader" if we got the default "classloader" parameter if (classloader ne loader) Right(loader.loadClass(fqn).asInstanceOf[Class[T]]) else Left(null) //Horrid
if (classloader ne loader) Option(loader.loadClass(fqn).asInstanceOf[Class[T]])
else None
} catch { } catch {
case c: ClassNotFoundException => None case c: ClassNotFoundException => Left(c)
} }
if (third.isDefined) third if (third.isRight) third
else { else {
// Last option is Class.forName
try { try {
Option(Class.forName(fqn).asInstanceOf[Class[T]]) Right(Class.forName(fqn).asInstanceOf[Class[T]]) // Last option is Class.forName
} catch { } catch {
case c: ClassNotFoundException => None case c: ClassNotFoundException => Left(c)
} }
} }
} }
} }
} catch {
case e: Exception => Left(e)
} }
} }

View file

@ -16,7 +16,11 @@ PAPEROPT_letter = -D latex_paper_size=letter
ALLSPHINXOPTS = -d $(BUILDDIR)/doctrees $(PAPEROPT_$(PAPER)) $(SPHINXOPTS) . ALLSPHINXOPTS = -d $(BUILDDIR)/doctrees $(PAPEROPT_$(PAPER)) $(SPHINXOPTS) .
# Set python path to include local packages for pygments styles. # Set python path to include local packages for pygments styles.
PYTHONPATH += $(LOCALPACKAGES) ifneq (,$(PYTHONPATH))
PYTHONPATH := $(PYTHONPATH):$(LOCALPACKAGES)
else
PYTHONPATH := $(LOCALPACKAGES)
endif
export PYTHONPATH export PYTHONPATH
.PHONY: help clean pygments html singlehtml latex pdf .PHONY: help clean pygments html singlehtml latex pdf
@ -40,8 +44,11 @@ pygments:
@echo "Custom pygments styles have been installed." @echo "Custom pygments styles have been installed."
@echo @echo
html: pygments $(LOCALPACKAGES):
$(SPHINXBUILD) -b html $(ALLSPHINXOPTS) $(BUILDDIR)/html $(MAKE) pygments
html: $(LOCALPACKAGES)
$(SPHINXBUILD) -a -b html $(ALLSPHINXOPTS) $(BUILDDIR)/html
@echo @echo
@echo "Build finished. The HTML pages are in $(BUILDDIR)/html." @echo "Build finished. The HTML pages are in $(BUILDDIR)/html."

Binary file not shown.

Binary file not shown.

Before

Width:  |  Height:  |  Size: 5.8 KiB

After

Width:  |  Height:  |  Size: 7.3 KiB

Before After
Before After

Binary file not shown.

Before

Width:  |  Height:  |  Size: 6.6 KiB

After

Width:  |  Height:  |  Size: 7.1 KiB

Before After
Before After

View file

@ -32,18 +32,12 @@
{% block content %} {% block content %}
<div class="header"> <div class="header">
{%- block akkaheader %} {%- block akkaheader %}
{%- if theme_full_logo != "false" %}
<a href="{{ pathto('index') }}">
<img class="logo" src="{{ pathto('_static/' + logo, 1) }}" alt="Logo"/>
</a>
{%- else %}
{%- if logo -%} {%- if logo -%}
<img class="rightlogo" src="{{ pathto('_static/' + logo, 1) }}" alt="Logo"/> <a href="{{ pathto('index') }}"><img class="leftlogo" src="{{ pathto('_static/' + logo, 1) }}" alt="Logo"/></a>
{%- endif -%} {%- endif -%}
<h1 class="heading"><a href="{{ pathto('index') }}"> <h1 class="heading"><a href="{{ pathto('index') }}"><span>{{ shorttitle|e }}</span></a></h1>
<span>{{ shorttitle|e }}</span></a></h1> <h2 class="heading"><a href="{{ pathto('index') }}"><span>Version {{ version|e }}</span></a></h2>
<h2 class="heading"><span>{{ title|striptags|e }}</span></h2> <h2 class="rightheading"><span><a href="http://akka.io/docs/akka/snapshot/Akka.pdf">PDF</a></span></h2>
{%- endif %}
{%- endblock %} {%- endblock %}
</div> </div>
<div class="topnav"> <div class="topnav">

View file

@ -112,17 +112,26 @@ div.header h2 {
font-size: 1.3em; font-size: 1.3em;
font-weight: normal; font-weight: normal;
letter-spacing: 1px; letter-spacing: 1px;
text-transform: uppercase; color: {{ theme_headingcolor }};
color: #aaa;
border: 0; border: 0;
margin-top: -3px; margin-top: -3px;
padding: 0; padding: 0;
} }
div.header img.leftlogo {
float: left;
}
div.header img.rightlogo { div.header img.rightlogo {
float: right; float: right;
} }
div.header h2.rightheading {
position: relative;
top:-45px;
float: right;
}
div.title { div.title {
font-size: 1.3em; font-size: 1.3em;

View file

@ -0,0 +1,16 @@
.. _add-on-modules:
Add-on Modules
==============
Akka Modules consist of add-on modules outside the core of Akka:
- ``akka-kernel-1.1.jar`` -- Akka microkernel for running a bare-bones mini application server (embeds Jetty etc.)
- ``akka-amqp-1.1.jar`` -- AMQP integration
- ``akka-camel-1.1.jar`` -- Apache Camel Actors integration (it's the best way to have your Akka application communicate with the rest of the world)
- ``akka-camel-typed-1.1.jar`` -- Apache Camel Typed Actors integration
- ``akka-scalaz-1.1.jar`` -- Support for the Scalaz library
- ``akka-spring-1.1.jar`` -- Spring framework integration
- ``akka-osgi-dependencies-bundle-1.1.jar`` -- OSGi support
Documentation for Akka Modules is located `here <http://akka.io/docs/akka-modules/snapshot/>`_.

View file

@ -21,6 +21,8 @@ Videos
Articles Articles
-------- --------
`Scatter-Gather with Akka Dataflow <http://vasilrem.com/blog/software-development/scatter-gather-with-akka-dataflow/>`_
`Actor-Based Continuations with Akka and Swarm <http://www.earldouglas.com/actor-based-continuations-with-akka-and-swarm>`_ `Actor-Based Continuations with Akka and Swarm <http://www.earldouglas.com/actor-based-continuations-with-akka-and-swarm>`_
`Mimicking Twitter Using an Akka-Based Event-Driven Architecture <http://www.earldouglas.com/mimicking-twitter-using-an-akka-based-event-driven-architecture>`_ `Mimicking Twitter Using an Akka-Based Event-Driven Architecture <http://www.earldouglas.com/mimicking-twitter-using-an-akka-based-event-driven-architecture>`_

View file

@ -0,0 +1,35 @@
Benchmarks
==========
Scalability, Throughput and Latency benchmark
---------------------------------------------
.. image:: ../images/benchmark-akka-sample-trading-throughput.png
Simple Trading system.
- `Here is the result with some graphs <https://spreadsheets.google.com/ccc?key=0AqkhZTxa6-dOdERaQnNvOEZpMDdnazRWOVNHMWIxZ0E&hl=en&authkey=CLyksoEI#gid=0>`_
- `Here is the article <http://blog.jayway.com/2010/08/10/yet-another-akka-benchmark/>`_
- `Here is the code <http://github.com/patriknw/akka-sample-trading>`_
Compares:
- Synchronous Scala solution
- Scala library Actors
- Fire-forget
- Request-reply
- Akka
- Request-reply
- Fire-forget with default dispatcher
- Fire-forget with Hawt dispatcher
Performance benchmark
---------------------
Benchmarking Akka against:
- Scala Library Actors
- Raw Java concurrency
- Jetlang (Java actors lib) `<http://github.com/jboner/akka-bench>`_

View file

@ -0,0 +1,15 @@
Additional Information
======================
.. toctree::
:maxdepth: 2
add-on-modules
articles
benchmarks
recipies
external-sample-projects
companies-using-akka
third-party-integrations
language-bindings
stability-matrix

View file

@ -0,0 +1,33 @@
Feature Stability Matrix
========================
Akka is comprised of a number if modules, with different levels of maturity and in different parts of their lifecycle, the matrix below gives you get current stability level of the modules.
Explanation of the different levels of stability
------------------------------------------------
* **Solid** - Proven solid in heavy production usage
* **Stable** - Ready for use in production environment
* **In progress** - Not enough feedback/use to claim it's ready for production use
================================ ============ ============ ============
Feature Solid Stable In progress
================================ ============ ============ ============
Actors (Scala) Solid
Actors (Java) Solid
Typed Actors (Scala) Solid
Typed Actors (Java) Solid
STM (Scala) Solid
STM (Java) Solid
Transactors (Scala) Solid
Transactors (Java) Solid
Remote Actors (Scala) Solid
Remote Actors (Java) Solid
Camel Solid
AMQP Solid
HTTP Solid
Integration Guice Stable
Integration Spring Stable
Scheduler Solid
Redis Pub Sub In progress
================================ ============ ============ ============

View file

@ -2,19 +2,20 @@ Third-party Integrations
======================== ========================
The Play! Framework The Play! Framework
=================== -------------------
Dustin Whitney has done an Akka integration module for the `Play! framework <http://www.playframework.org/>`_. Dustin Whitney has done an Akka integration module for the `Play! framework <http://www.playframework.org/>`_.
Detailed instructions here: `<http://github.com/dwhitney/akka/blob/master/README.textile>`_. Detailed instructions here: `<http://github.com/dwhitney/akka/blob/master/README.textile>`_.
There are three screencasts: There are three screencasts:
# Using Play! with Akka STM: `<http://vimeo.com/10764693>`_
# Using Play! with Akka Actors: `<http://vimeo.com/10792173>`_ - Using Play! with Akka STM: `<http://vimeo.com/10764693>`_
# Using Play! with Akka Remote Actors: `<http://vimeo.com/10793443>`_ - Using Play! with Akka Actors: `<http://vimeo.com/10792173>`_
- Using Play! with Akka Remote Actors: `<http://vimeo.com/10793443>`_
The Pinky REST/MVC Framework The Pinky REST/MVC Framework
============================ ----------------------------
Peter Hausel has done an Akka integration module for the `Pinky framework <http://wiki.github.com/pk11/pinky/>`_. Peter Hausel has done an Akka integration module for the `Pinky framework <http://wiki.github.com/pk11/pinky/>`_.

View file

@ -1,23 +1,20 @@
#########
Utilities
#########
.. sidebar:: Contents
.. contents:: :local:
This section of the manual describes miscellaneous utilities which are provided
by Akka and used in multiple places.
.. _Duration: .. _Duration:
########
Duration Duration
======== ########
Module stability: **SOLID**
Durations are used throughout the Akka library, wherefore this concept is Durations are used throughout the Akka library, wherefore this concept is
represented by a special data type, :class:`Duration`. Values of this type may represented by a special data type, :class:`Duration`. Values of this type may
represent infinite (:obj:`Duration.Inf`, :obj:`Duration.MinusInf`) or finite represent infinite (:obj:`Duration.Inf`, :obj:`Duration.MinusInf`) or finite
durations, where the latter are constructable using a mini-DSL: durations.
Scala
=====
In Scala durations are constructable using a mini-DSL and support all expected operations:
.. code-block:: scala .. code-block:: scala
@ -27,6 +24,8 @@ durations, where the latter are constructable using a mini-DSL:
val threemillis = 3.millis val threemillis = 3.millis
val diff = fivesec - threemillis val diff = fivesec - threemillis
assert (diff < fivesec) assert (diff < fivesec)
val fourmillis = threemillis * 4 / 3 // though you cannot write it the other way around
val n = threemillis / (1 millisecond)
.. note:: .. note::
@ -35,6 +34,9 @@ durations, where the latter are constructable using a mini-DSL:
if the time unit is the last token on a line, otherwise semi-colon inference if the time unit is the last token on a line, otherwise semi-colon inference
might go wrong, depending on what starts the next line. might go wrong, depending on what starts the next line.
Java
====
Java provides less syntactic sugar, so you have to spell out the operations as Java provides less syntactic sugar, so you have to spell out the operations as
method calls instead: method calls instead:

View file

@ -0,0 +1,8 @@
Common utilities
==========================
.. toctree::
:maxdepth: 2
scheduler
duration

View file

@ -0,0 +1,23 @@
Scheduler
=========
Module stability: **SOLID**
``Akka`` has a little scheduler written using actors.
This can be convenient if you want to schedule some periodic task for maintenance or similar.
It allows you to register a message that you want to be sent to a specific actor at a periodic interval.
Here is an example:
-------------------
.. code-block:: scala
import akka.actor.Scheduler
//Sends messageToBeSent to receiverActor after initialDelayBeforeSending and then after each delayBetweenMessages
Scheduler.schedule(receiverActor, messageToBeSent, initialDelayBeforeSending, delayBetweenMessages, timeUnit)
//Sends messageToBeSent to receiverActor after delayUntilSend
Scheduler.scheduleOnce(receiverActor, messageToBeSent, delayUntilSend, timeUnit)

View file

@ -13,12 +13,12 @@ extensions = ['sphinx.ext.todo', 'includecode']
templates_path = ['_templates'] templates_path = ['_templates']
source_suffix = '.rst' source_suffix = '.rst'
master_doc = 'index' master_doc = 'index'
exclude_patterns = ['_build', 'pending'] exclude_patterns = ['_build', 'pending', 'disabled']
project = u'Akka' project = u'Akka'
copyright = u'2009-2011, Scalable Solutions AB' copyright = u'2011, Typesafe Inc'
version = '1.1' version = '1.2-SNAPSHOT'
release = '1.1' release = '1.2-SNAPSHOT'
pygments_style = 'simple' pygments_style = 'simple'
highlight_language = 'scala' highlight_language = 'scala'
@ -28,9 +28,6 @@ show_authors = True
# -- Options for HTML output --------------------------------------------------- # -- Options for HTML output ---------------------------------------------------
html_theme = 'akka' html_theme = 'akka'
html_theme_options = {
'full_logo': 'true'
}
html_theme_path = ['_sphinx/themes'] html_theme_path = ['_sphinx/themes']
html_title = 'Akka Documentation' html_title = 'Akka Documentation'

View file

@ -0,0 +1,175 @@
.. highlightlang:: none
.. _building-akka:
###############
Building Akka
###############
This page describes how to build and run Akka from the latest source code.
.. contents:: :local:
Get the source code
===================
Akka uses `Git`_ and is hosted at `Github`_.
.. _Git: http://git-scm.com
.. _Github: http://github.com
You first need Git installed on your machine. You can then clone the source
repositories:
- Akka repository from http://github.com/jboner/akka
- Akka Modules repository from http://github.com/jboner/akka-modules
For example::
git clone git://github.com/jboner/akka.git
git clone git://github.com/jboner/akka-modules.git
If you have already cloned the repositories previously then you can update the
code with ``git pull``::
git pull origin master
SBT - Simple Build Tool
=======================
Akka is using the excellent `SBT`_ build system. So the first thing you have to
do is to download and install SBT. You can read more about how to do that in the
`SBT setup`_ documentation.
.. _SBT: http://code.google.com/p/simple-build-tool
.. _SBT setup: http://code.google.com/p/simple-build-tool/wiki/Setup
The SBT commands that you'll need to build Akka are all included below. If you
want to find out more about SBT and using it for your own projects do read the
`SBT documentation`_.
.. _SBT documentation: http://code.google.com/p/simple-build-tool/wiki/RunningSbt
The Akka SBT build file is ``project/build/AkkaProject.scala`` with some
properties defined in ``project/build.properties``.
Building Akka
=============
First make sure that you are in the akka code directory::
cd akka
Fetching dependencies
---------------------
SBT does not fetch dependencies automatically. You need to manually do this with
the ``update`` command::
sbt update
Once finished, all the dependencies for Akka will be in the ``lib_managed``
directory under each module: akka-actor, akka-stm, and so on.
*Note: you only need to run update the first time you are building the code,
or when the dependencies have changed.*
Building
--------
To compile all the Akka core modules use the ``compile`` command::
sbt compile
You can run all tests with the ``test`` command::
sbt test
If compiling and testing are successful then you have everything working for the
latest Akka development version.
Publish to local Ivy repository
-------------------------------
If you want to deploy the artifacts to your local Ivy repository (for example,
to use from an SBT project) use the ``publish-local`` command::
sbt publish-local
Publish to local Maven repository
---------------------------------
If you want to deploy the artifacts to your local Maven repository use::
sbt publish-local publish
SBT interactive mode
--------------------
Note that in the examples above we are calling ``sbt compile`` and ``sbt test``
and so on. SBT also has an interactive mode. If you just run ``sbt`` you enter
the interactive SBT prompt and can enter the commands directly. This saves
starting up a new JVM instance for each command and can be much faster and more
convenient.
For example, building Akka as above is more commonly done like this::
% sbt
[info] Building project akka 1.2-SNAPSHOT against Scala 2.9.0
[info] using AkkaParentProject with sbt 0.7.6 and Scala 2.7.7
> update
[info]
[info] == akka-actor / update ==
...
[success] Successful.
[info]
[info] Total time ...
> compile
...
> test
...
SBT batch mode
--------------
It's also possible to combine commands in a single call. For example, updating,
testing, and publishing Akka to the local Ivy repository can be done with::
sbt update test publish-local
Building Akka Modules
=====================
See the Akka Modules documentation.
.. _dependencies:
Dependencies
============
If you are managing dependencies by hand you can find the dependencies for each
module by looking in the ``lib_managed`` directories. For example, this will
list all compile dependencies (providing you have the source code and have run
``sbt update``)::
cd akka
ls -1 */lib_managed/compile
You can also look at the Ivy dependency resolution information that is created
on ``sbt update`` and found in ``~/.ivy2/cache``. For example, the
``.ivy2/cache/se.scalablesolutions.akka-akka-remote-compile.xml`` file contains
the resolution information for the akka-remote module compile dependencies. If
you open this file in a web browser you will get an easy to navigate view of
dependencies.

View file

@ -1,3 +1,5 @@
.. _developer_guidelines:
Developer Guidelines Developer Guidelines
==================== ====================
@ -6,25 +8,25 @@ Code Style
The Akka code style follows `this document <http://davetron5000.github.com/scala-style/ScalaStyleGuide.pdf>`_ . The Akka code style follows `this document <http://davetron5000.github.com/scala-style/ScalaStyleGuide.pdf>`_ .
Here is a code style settings file for IntelliJ IDEA. Here is a code style settings file for ``IntelliJ IDEA``:
`<file:akka-intellij-code-style.jar>`_ `Download <../_static/akka-intellij-code-style.jar>`_
Please follow the code style. Look at the code around you and mimic. Please follow the code style. Look at the code around you and mimic.
Testing Testing
------- -------
All code that is checked in should have tests. All testing is done with ScalaTest and ScalaCheck. All code that is checked in **should** have tests. All testing is done with ``ScalaTest`` and ``ScalaCheck``.
* Name tests as *Test.scala if they do not depend on any external stuff. That keeps surefire happy. * Name tests as **Test.scala** if they do not depend on any external stuff. That keeps surefire happy.
* Name tests as *Spec.scala if they have external dependencies. * Name tests as **Spec.scala** if they have external dependencies.
There is a testing standard that should be followed: `Ticket001Spec <@https://github.com/jboner/akka/blob/master/akka-actor/src/test/scala/akka/ticket/Ticket001Spec.scala>`_ There is a testing standard that should be followed: `Ticket001Spec <https://github.com/jboner/akka/blob/master/akka-actor-tests/src/test/scala/akka/ticket/Ticket001Spec.scala>`_
Actor TestKit Actor TestKit
^^^^^^^^^^^^^ ^^^^^^^^^^^^^
There is a useful test kit for testing actors: `akka.util.TestKit <@https://github.com/jboner/akka/tree/master/akka-actor/src/main/scala/akka/util/TestKit.scala>`_. It enables assertions concerning replies received and their timing, there is more documentation in the `<TestKit>`_ module. There is a useful test kit for testing actors: `akka.util.TestKit <https://github.com/jboner/akka/tree/master/akka-testkit/src/main/scala/akka/testkit/TestKit.scala>`_. It enables assertions concerning replies received and their timing, there is more documentation in the :ref:`akka-testkit` module.
NetworkFailureTest NetworkFailureTest
^^^^^^^^^^^^^^^^^^ ^^^^^^^^^^^^^^^^^^

View file

@ -3,10 +3,14 @@
.. _documentation: .. _documentation:
############### #########################
Documentation Documentation Guidelines
############### #########################
.. sidebar:: Contents
.. contents:: :local:
The Akka documentation uses `reStructuredText`_ as its markup language and is The Akka documentation uses `reStructuredText`_ as its markup language and is
built using `Sphinx`_. built using `Sphinx`_.
@ -67,3 +71,83 @@ For example::
Here is a reference to "akka section": :ref:`akka-section` which will have the Here is a reference to "akka section": :ref:`akka-section` which will have the
name "Akka Section". name "Akka Section".
Build the documentation
=======================
First install `Sphinx`_. See below.
Building
--------
::
cd akka-docs
make html
open _build/html/index.html
make pdf
open _build/latex/Akka.pdf
Installing Sphinx on OS X
-------------------------
Install `Homebrew <https://github.com/mxcl/homebrew>`_
Install Python and pip:
::
brew install python
/usr/local/share/python/easy_install pip
Add the Homebrew Python path to your $PATH:
::
/usr/local/Cellar/python/2.7.1/bin
More information in case of trouble:
https://github.com/mxcl/homebrew/wiki/Homebrew-and-Python
Install sphinx:
::
pip install sphinx
Add sphinx_build to your $PATH:
::
/usr/local/share/python
Install BasicTeX package from:
http://www.tug.org/mactex/morepackages.html
Add texlive bin to $PATH:
::
/usr/local/texlive/2010basic/bin/universal-darwin
Add missing tex packages:
::
sudo tlmgr update --self
sudo tlmgr install titlesec
sudo tlmgr install framed
sudo tlmgr install threeparttable
sudo tlmgr install wrapfig
sudo tlmgr install helvetic
sudo tlmgr install courier
Link the akka pygments style:
::
cd /usr/local/Cellar/python/2.7.1/lib/python2.7/site-packages/pygments/styles
ln -s /path/to/akka/akka-docs/themes/akka/pygments/akka.py akka.py

View file

@ -4,4 +4,8 @@ Information for Developers
.. toctree:: .. toctree::
:maxdepth: 2 :maxdepth: 2
building-akka
developer-guidelines
documentation documentation
team

28
akka-docs/dev/team.rst Normal file
View file

@ -0,0 +1,28 @@
.. _team:
Team
=====
=================== ========================== ====================================
Name Role Email
=================== ========================== ====================================
Jonas Bonér Founder, Despot, Committer jonas AT jonasboner DOT com
Viktor Klang Bad cop, Committer viktor DOT klang AT gmail DOT com
Debasish Ghosh Committer dghosh AT acm DOT org
Ross McDonald Alumni rossajmcd AT gmail DOT com
Eckhart Hertzler Alumni
Mikael Högqvist Alumni
Tim Perrett Alumni
Jeanfrancois Arcand Alumni jfarcand AT apache DOT org
Martin Krasser Committer krasserm AT googlemail DOT com
Jan Van Besien Alumni
Michael Kober Alumni
Peter Vlugter Committer
Peter Veentjer Committer
Irmo Manie Committer
Heiko Seeberger Committer
Hiram Chirino Committer
Scott Clasen Committer
Roland Kuhn Committer
Patrik Nordwall Committer patrik DOT nordwall AT gmail DOT com
=================== ========================== ====================================

View file

@ -91,11 +91,11 @@ object Pi extends App {
} }
//#master-receive //#master-receive
override def preStart { override def preStart() {
start = now start = now
} }
override def postStop { override def postStop() {
// tell the world that the calculation is complete // tell the world that the calculation is complete
println( println(
"\n\tPi estimate: \t\t%s\n\tCalculation time: \t%s millis" "\n\tPi estimate: \t\t%s\n\tCalculation time: \t%s millis"

View file

@ -19,19 +19,22 @@ We will be using an algorithm that is called "embarrassingly parallel" which jus
Here is the formula for the algorithm we will use: Here is the formula for the algorithm we will use:
.. image:: pi-formula.png .. image:: ../images/pi-formula.png
In this particular algorithm the master splits the series into chunks which are sent out to each worker actor to be processed. When each worker has processed its chunk it sends a result back to the master which aggregates the total result. In this particular algorithm the master splits the series into chunks which are sent out to each worker actor to be processed. When each worker has processed its chunk it sends a result back to the master which aggregates the total result.
Tutorial source code Tutorial source code
-------------------- --------------------
If you want don't want to type in the code and/or set up an SBT project then you can check out the full tutorial from the Akka GitHub repository. It is in the ``akka-tutorials/akka-tutorial-first`` module. You can also browse it online `here <https://github.com/jboner/akka/tree/master/akka-tutorials/akka-tutorial-first>`_, with the actual source code `here <https://github.com/jboner/akka/blob/master/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala>`_. If you want don't want to type in the code and/or set up an SBT project then you can check out the full tutorial from the Akka GitHub repository. It is in the ``akka-tutorials/akka-tutorial-first`` module. You can also browse it online `here`__, with the actual source code `here`__.
__ https://github.com/jboner/akka/tree/master/akka-tutorials/akka-tutorial-first
__ https://github.com/jboner/akka/blob/master/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala
Prerequisites Prerequisites
------------- -------------
This tutorial assumes that you have Jave 1.6 or later installed on you machine and ``java`` on your ``PATH``. You also need to know how to run commands in a shell (ZSH, Bash, DOS etc.) and a decent text editor or IDE to type in the Scala code. This tutorial assumes that you have Java 1.6 or later installed on you machine and ``java`` on your ``PATH``. You also need to know how to run commands in a shell (ZSH, Bash, DOS etc.) and a decent text editor or IDE to type in the Scala code.
Downloading and installing Akka Downloading and installing Akka
------------------------------- -------------------------------
@ -77,7 +80,7 @@ Akka is very modular and has many JARs for containing different features. The co
- ``akka-slf4j-1.1.jar`` -- SLF4J Event Handler Listener - ``akka-slf4j-1.1.jar`` -- SLF4J Event Handler Listener
- ``akka-testkit-1.1.jar`` -- Toolkit for testing Actors - ``akka-testkit-1.1.jar`` -- Toolkit for testing Actors
We also have Akka Modules containing add-on modules outside the core of Akka. You can download the Akka Modules distribution from TODO. It contains Akka core as well. We will not be needing any modules there today, but for your information the module JARs are these: We also have Akka Modules containing add-on modules outside the core of Akka. You can download the Akka Modules distribution from `<http://akka.io/downloads/>`_. It contains Akka core as well. We will not be needing any modules there today, but for your information the module JARs are these:
- ``akka-kernel-1.1.jar`` -- Akka microkernel for running a bare-bones mini application server (embeds Jetty etc.) - ``akka-kernel-1.1.jar`` -- Akka microkernel for running a bare-bones mini application server (embeds Jetty etc.)
- ``akka-amqp-1.1.jar`` -- AMQP integration - ``akka-amqp-1.1.jar`` -- AMQP integration
@ -92,16 +95,16 @@ Downloading and installing Scala
To build and run the tutorial sample from the command line, you have to install the Scala distribution. If you prefer to use SBT to build and run the sample then you can skip this section and jump to the next one. To build and run the tutorial sample from the command line, you have to install the Scala distribution. If you prefer to use SBT to build and run the sample then you can skip this section and jump to the next one.
Scala can be downloaded from `http://www.scala-lang.org/downloads <http://www.scala-lang.org/downloads>`_. Browse there and download the Scala 2.9.0.RC1 release. If you pick the ``tgz`` or ``zip`` distribution then just unzip it where you want it installed. If you pick the IzPack Installer then double click on it and follow the instructions. Scala can be downloaded from `http://www.scala-lang.org/downloads <http://www.scala-lang.org/downloads>`_. Browse there and download the Scala 2.9.0 release. If you pick the ``tgz`` or ``zip`` distribution then just unzip it where you want it installed. If you pick the IzPack Installer then double click on it and follow the instructions.
You also need to make sure that the ``scala-2.9.0.RC1/bin`` (if that is the directory where you installed Scala) is on your ``PATH``:: You also need to make sure that the ``scala-2.9.0/bin`` (if that is the directory where you installed Scala) is on your ``PATH``::
$ export PATH=$PATH:scala-2.9.0.RC1/bin $ export PATH=$PATH:scala-2.9.0/bin
You can test your installation by invoking scala:: You can test your installation by invoking scala::
$ scala -version $ scala -version
Scala code runner version 2.9.0.RC1 -- Copyright 2002-2011, LAMP/EPFL Scala code runner version 2.9.0.final -- Copyright 2002-2011, LAMP/EPFL
Looks like we are all good. Finally let's create a source file ``Pi.scala`` for the tutorial and put it in the root of the Akka distribution in the ``tutorial`` directory (you have to create it first). Looks like we are all good. Finally let's create a source file ``Pi.scala`` for the tutorial and put it in the root of the Akka distribution in the ``tutorial`` directory (you have to create it first).
@ -130,7 +133,7 @@ If you have not already done so, now is the time to create an SBT project for ou
Name: Tutorial 1 Name: Tutorial 1
Organization: Hakkers Inc Organization: Hakkers Inc
Version [1.0]: Version [1.0]:
Scala version [2.9.0.RC1]: Scala version [2.9.0]:
sbt version [0.7.6.RC0]: sbt version [0.7.6.RC0]:
Now we have the basis for an SBT project. Akka has an SBT Plugin making it very easy to use Akka is an SBT-based project so let's use that. Now we have the basis for an SBT project. Akka has an SBT Plugin making it very easy to use Akka is an SBT-based project so let's use that.

View file

@ -0,0 +1,125 @@
Configuration
=============
.. sidebar:: Contents
.. contents:: :local:
.. _-Dakka.config:
.. _-Dakka.home:
Specifying the configuration file
---------------------------------
If you don't specify a configuration file then Akka uses default values, corresponding to the ``akka-reference.conf``
that you see below. You can specify your own configuration file to override any property in the reference config.
You only have to define the properties that differ from the default configuration.
The location of the config file to use can be specified in various ways:
* Define the ``-Dakka.config=...`` system property parameter with a file path to configuration file.
* Put an ``akka.conf`` file in the root of the classpath.
* Define the ``AKKA_HOME`` environment variable pointing to the root of the Akka
distribution. The config is taken from the ``AKKA_HOME/config/akka.conf``. You
can also point to the AKKA_HOME by specifying the ``-Dakka.home=...`` system
property parameter.
If several of these ways to specify the config file are used at the same time the precedence is the order as given above,
i.e. you can always redefine the location with the ``-Dakka.config=...`` system property.
Defining the configuration file
-------------------------------
Here is the reference configuration file:
.. literalinclude:: ../../config/akka-reference.conf
:language: none
A custom ``akka.conf`` might look like this:
::
# In this file you can override any option defined in the 'akka-reference.conf' file.
# Copy in all or parts of the 'akka-reference.conf' file and modify as you please.
akka {
event-handlers = ["akka.event.slf4j.Slf4jEventHandler"]
# Comma separated list of the enabled modules.
enabled-modules = ["camel", "remote"]
# These boot classes are loaded (and created) automatically when the Akka Microkernel boots up
# Can be used to bootstrap your application(s)
# Should be the FQN (Fully Qualified Name) of the boot class which needs to have a default constructor
boot = ["sample.camel.Boot",
"sample.myservice.Boot"]
actor {
throughput = 10 # Throughput for ExecutorBasedEventDrivenDispatcher, set to 1 for complete fairness
}
remote {
server {
port = 2562 # The port clients should connect to. Default is 2552 (AKKA)
}
}
}
.. _-Dakka.mode:
Specifying files for different modes
------------------------------------
You can use different configuration files for different purposes by specifying a mode option, either as
``-Dakka.mode=...`` system property or as ``AKKA_MODE=...`` environment variable. For example using DEBUG log level
when in development mode. Run with ``-Dakka.mode=dev`` and place the following ``akka.dev.conf`` in the root of
the classpath.
akka.dev.conf:
::
akka {
event-handler-level = "DEBUG"
}
The mode option works in the same way when using configuration files in ``AKKA_HOME/config/`` directory.
The mode option is not used when specifying the configuration file with ``-Dakka.config=...`` system property.
Including files
---------------
Sometimes it can be useful to include another configuration file, for example if you have one ``akka.conf`` with all
environment independent settings and then override some settings for specific modes.
akka.dev.conf:
::
include "akka.conf"
akka {
event-handler-level = "DEBUG"
}
.. _-Dakka.output.config.source:
Showing Configuration Source
----------------------------
If the system property ``akka.output.config.source`` is set to anything but
null, then the source from which Akka reads its configuration is printed to the
console during application startup.
Summary of System Properties
----------------------------
* :ref:`akka.home <-Dakka.home>` (``AKKA_HOME``): where Akka searches for configuration
* :ref:`akka.config <-Dakka.config>`: explicit configuration file location
* :ref:`akka.mode <-Dakka.mode>` (``AKKA_MODE``): modify configuration file name for multiple profiles
* :ref:`akka.output.config.source <-Dakka.output.config.source>`: whether to print configuration source to console

View file

@ -1,3 +1,5 @@
.. _event-handler:
Event Handler Event Handler
============= =============
@ -12,11 +14,12 @@ You can configure which event handlers should be registered at boot time. That i
.. code-block:: ruby .. code-block:: ruby
akka { akka {
event-handlers = ["akka.event.EventHandler$DefaultListener"] # event handlers to register at boot time (EventHandler$DefaultListener logs to STDOUT) # event handlers to register at boot time (EventHandler$DefaultListener logs to STDOUT)
event-handlers = ["akka.event.EventHandler$DefaultListener"]
event-handler-level = "DEBUG" # Options: ERROR, WARNING, INFO, DEBUG event-handler-level = "DEBUG" # Options: ERROR, WARNING, INFO, DEBUG
} }
The default one logs to STDOUT and is registered by default. It is not intended to be used for production. There is also an SLF4J event handler available in the 'akka-slf4j.jar' module. Read more about it `here <slf4j>`_. The default one logs to STDOUT and is registered by default. It is not intended to be used for production. There is also an :ref:`slf4j` event handler available in the 'akka-slf4j' module.
Example of creating a listener from Scala (from Java you just have to create an 'UntypedActor' and create a handler for these messages): Example of creating a listener from Scala (from Java you just have to create an 'UntypedActor' and create a handler for these messages):
@ -88,9 +91,10 @@ The methods take a call-by-name parameter for the message to avoid object alloca
From Java you need to nest the call in an if statement to achieve the same thing. From Java you need to nest the call in an if statement to achieve the same thing.
`<code format="scala">`_ .. code-block:: java
if (EventHandler.isDebugEnabled()) {
EventHandler.debug(this, String.format("Processing took %s ms", duration)); if (EventHandler.isDebugEnabled()) {
} EventHandler.debug(this, String.format("Processing took %s ms", duration));
}
`<code>`_

View file

@ -4,5 +4,8 @@ General
.. toctree:: .. toctree::
:maxdepth: 2 :maxdepth: 2
migration-guides jmm
util configuration
event-handler
slf4j

36
akka-docs/general/jmm.rst Normal file
View file

@ -0,0 +1,36 @@
Akka and the Java Memory Model
================================
Prior to Java 5, the Java Memory Model (JMM) was broken. It was possible to get all kinds of strange results like unpredictable merged writes made by concurrent executing threads, unexpected reordering of instructions, and even final fields were not guaranteed to be final. With Java 5 and JSR-133, the Java Memory Model is clearly specified. This specification makes it possible to write code that performs, but doesn't cause concurrency problems. The Java Memory Model is specified in 'happens before'-rules, e.g.:
* **monitor lock rule**: a release of a lock happens before every subsequent acquire of the same lock.
* **volatile variable rule**: a write of a volatile variable happens before every subsequent read of the same volatile variable
The 'happens before'-rules clearly specify which visibility guarantees are provided on memory and which re-orderings are allowed. Without these rules it would not be possible to write concurrent and performant code in Java.
Actors and the Java Memory Model
--------------------------------
With the Actors implementation in Akka, there are 2 ways multiple threads can execute actions on shared memory over time:
* if a message is send to an actor (e.g. by another actor). In most cases messages are immutable, but if that message is not a properly constructed immutable object, without happens before rules, the system still could be subject to instruction re-orderings and visibility problems (so a possible source of concurrency errors).
* if an actor makes changes to its internal state in one 'receive' method and access that state while processing another message. With the actors model you don't get any guarantee that the same thread will be executing the same actor for different messages. Without a happens before relation between these actions, there could be another source of concurrency errors.
To solve the 2 problems above, Akka adds the following 2 'happens before'-rules to the JMM:
* **the actor send rule**: where the send of the message to an actor happens before the receive of the **same** actor.
* **the actor subsequent processing rule**: where processing of one message happens before processing of the next message by the **same** actor.
Both rules only apply for the same actor instance and are not valid if different actors are used.
STM and the Java Memory Model
-----------------------------
The Akka STM also provides a happens before rule called:
* **the transaction rule**: a commit on a transaction happens before every subsequent start of a transaction where there is at least 1 shared reference.
How these rules are realized in Akka, is an implementation detail and can change over time (the exact details could even depend on the used configuration) but they will lift on the other JMM rules like the monitor lock rule or the volatile variable rule. Essentially this means that you, the Akka user, do not need to worry about adding synchronization to provide such a happens before relation, because it is the responsibility of Akka. So you have your hands free to deal with your problems and not that of the framework.

View file

@ -1,41 +0,0 @@
Migration Guide 1.0.x to 1.1.x
===================================
**Akka has now moved to Scala 2.9.x**
Akka HTTP
---------
# akka.servlet.Initializer has been moved to ``akka-kernel`` to be able to have ``akka-http`` not depend on ``akka-remote``, if you don't want to use the class for kernel, just create your own version of ``akka.servlet.Initializer``, it's just a couple of lines of code and there is instructions here: `Akka Http Docs <http>`_
# akka.http.ListWriter has been removed in full, if you use it and want to keep using it, here's the code: `ListWriter <https://github.com/jboner/akka/blob/v1.0/akka-http/src/main/scala/akka/http/ListWriter.scala>`_
# Jersey-server is now a "provided" dependency for ``akka-http``, so you'll need to add the dependency to your project, it's built against Jersey 1.3
Akka Actor
----------
# is now dependency free, with the exception of the dependency on the ``scala-library.jar``
# does not bundle any logging anymore, but you can subscribe to events within Akka by registering an event handler on akka.event.EventHandler or by specifying the ``FQN`` of an Actor in the akka.conf under akka.event-handlers; there is an ``akka-slf4j`` module which still provides the Logging trait and a default ``SLF4J`` logger adapter.
Don't forget to add a SLF4J backend though, we recommend:
.. code-block:: scala
lazy val logback = "ch.qos.logback" % "logback-classic" % "0.9.28" % "runtime"
# If you used HawtDispatcher and want to continue using it, you need to include akka-dispatcher-extras.jar from Akka Modules, in your akka.conf you need to specify: ``akka.dispatch.HawtDispatcherConfigurator`` instead of ``HawtDispatcher``
# FSM: the onTransition method changed from Function1 to PartialFunction; there is an implicit conversion for the precise types in place, but it may be necessary to add an underscore if you are passing an eta-expansion (using a method as function value).
Akka Typed Actor
----------------
All methods starting with 'get*' are deprecated and will be removed in post 1.1 release.
Akka Remote
-----------
# ``UnparsebleException`` has been renamed to ``CannotInstantiateRemoteExceptionDueToRemoteProtocolParsingErrorException(exception, classname, message)``
Akka Testkit
------------
The TestKit moved into the akka-testkit subproject and correspondingly into the :code:`akka.testkit` package.

View file

@ -1,7 +1,15 @@
.. _slf4j:
SLF4J SLF4J
===== =====
This module is available in the 'akka-slf4j.jar'. It has one single dependency; the slf4j-api jar. This module is available in the 'akka-slf4j.jar'. It has one single dependency; the slf4j-api jar. In runtime you
also need a SLF4J backend, we recommend:
.. code-block:: scala
lazy val logback = "ch.qos.logback" % "logback-classic" % "0.9.28" % "runtime"
Event Handler Event Handler
------------- -------------
@ -15,5 +23,5 @@ This module includes a SLF4J Event Handler that works with Akka's standard Event
event-handler-level = "DEBUG" event-handler-level = "DEBUG"
} }
Read more about how to use the event handler `here <http://doc.akka.io/event-handler>`_. Read more about how to use the :ref:`event-handler`.

Binary file not shown.

After

Width:  |  Height:  |  Size: 14 KiB

View file

Before

Width:  |  Height:  |  Size: 119 KiB

After

Width:  |  Height:  |  Size: 119 KiB

Before After
Before After

Binary file not shown.

After

Width:  |  Height:  |  Size: 71 KiB

View file

Before

Width:  |  Height:  |  Size: 64 KiB

After

Width:  |  Height:  |  Size: 64 KiB

Before After
Before After

View file

Before

Width:  |  Height:  |  Size: 53 KiB

After

Width:  |  Height:  |  Size: 53 KiB

Before After
Before After

View file

Before

Width:  |  Height:  |  Size: 84 KiB

After

Width:  |  Height:  |  Size: 84 KiB

Before After
Before After

View file

Before

Width:  |  Height:  |  Size: 134 KiB

After

Width:  |  Height:  |  Size: 134 KiB

Before After
Before After

View file

Before

Width:  |  Height:  |  Size: 1.5 KiB

After

Width:  |  Height:  |  Size: 1.5 KiB

Before After
Before After

View file

Before

Width:  |  Height:  |  Size: 28 KiB

After

Width:  |  Height:  |  Size: 28 KiB

Before After
Before After

View file

Before

Width:  |  Height:  |  Size: 124 KiB

After

Width:  |  Height:  |  Size: 124 KiB

Before After
Before After

View file

@ -6,10 +6,29 @@ Contents
intro/index intro/index
general/index general/index
common/index
scala/index scala/index
java/index
dev/index dev/index
project/index
additional/index
Links Links
===== =====
* `Support <http://scalablesolutions.se>`_ * :ref:`migration`
* `Downloads <http://akka.io/downloads/>`_
* `Source Code <http://github.com/jboner/akka>`_
* :ref:`scaladoc`
* :ref:`other-doc`
* `Akka Modules Documentation <http://akka.io/docs/akka-modules/snapshot/>`_
* :ref:`issue_tracking`
* :ref:`support`

View file

@ -1,340 +0,0 @@
Building Akka
=============
This page describes how to build and run Akka from the latest source code.
.. contents:: :local:
Get the source code
-------------------
Akka uses `Git <http://git-scm.com>`_ and is hosted at `Github
<http://github.com>`_.
You first need Git installed on your machine. You can then clone the source
repositories:
- Akka repository from `<http://github.com/jboner/akka>`_
- Akka Modules repository from `<http://github.com/jboner/akka-modules>`_
For example::
git clone git://github.com/jboner/akka.git
git clone git://github.com/jboner/akka-modules.git
If you have already cloned the repositories previously then you can update the
code with ``git pull``::
git pull origin master
SBT - Simple Build Tool
-----------------------
Akka is using the excellent `SBT <http://code.google.com/p/simple-build-tool>`_
build system. So the first thing you have to do is to download and install
SBT. You can read more about how to do that `here
<http://code.google.com/p/simple-build-tool/wiki/Setup>`_ .
The SBT commands that you'll need to build Akka are all included below. If you
want to find out more about SBT and using it for your own projects do read the
`SBT documentation
<http://code.google.com/p/simple-build-tool/wiki/RunningSbt>`_.
The Akka SBT build file is ``project/build/AkkaProject.scala`` with some
properties defined in ``project/build.properties``.
Building Akka
-------------
First make sure that you are in the akka code directory::
cd akka
Fetching dependencies
^^^^^^^^^^^^^^^^^^^^^
SBT does not fetch dependencies automatically. You need to manually do this with
the ``update`` command::
sbt update
Once finished, all the dependencies for Akka will be in the ``lib_managed``
directory under each module: akka-actor, akka-stm, and so on.
*Note: you only need to run update the first time you are building the code,
or when the dependencies have changed.*
Building
^^^^^^^^
To compile all the Akka core modules use the ``compile`` command::
sbt compile
You can run all tests with the ``test`` command::
sbt test
If compiling and testing are successful then you have everything working for the
latest Akka development version.
Publish to local Ivy repository
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
If you want to deploy the artifacts to your local Ivy repository (for example,
to use from an SBT project) use the ``publish-local`` command::
sbt publish-local
Publish to local Maven repository
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
If you want to deploy the artifacts to your local Maven repository use::
sbt publish-local publish
SBT interactive mode
^^^^^^^^^^^^^^^^^^^^
Note that in the examples above we are calling ``sbt compile`` and ``sbt test``
and so on. SBT also has an interactive mode. If you just run ``sbt`` you enter
the interactive SBT prompt and can enter the commands directly. This saves
starting up a new JVM instance for each command and can be much faster and more
convenient.
For example, building Akka as above is more commonly done like this:
.. code-block:: none
% sbt
[info] Building project akka 1.1-SNAPSHOT against Scala 2.9.0.RC1
[info] using AkkaParentProject with sbt 0.7.6.RC0 and Scala 2.7.7
> update
[info]
[info] == akka-actor / update ==
...
[success] Successful.
[info]
[info] Total time ...
> compile
...
> test
...
SBT batch mode
^^^^^^^^^^^^^^
It's also possible to combine commands in a single call. For example, updating,
testing, and publishing Akka to the local Ivy repository can be done with::
sbt update test publish-local
Building Akka Modules
---------------------
To build Akka Modules first build and publish Akka to your local Ivy repository
as described above. Or using::
cd akka
sbt update publish-local
Then you can build Akka Modules using the same steps as building Akka. First
update to get all dependencies (including the Akka core modules), then compile,
test, or publish-local as needed. For example::
cd akka-modules
sbt update publish-local
Microkernel distribution
^^^^^^^^^^^^^^^^^^^^^^^^
To build the Akka Modules microkernel (the same as the Akka Modules distribution
download) use the ``dist`` command::
sbt dist
The distribution zip can be found in the dist directory and is called
``akka-modules-{version}.zip``.
To run the microkernel, unzip the zip file, change into the unzipped directory,
set the ``AKKA_HOME`` environment variable, and run the main jar file. For
example:
.. code-block:: none
unzip dist/akka-modules-1.1-SNAPSHOT.zip
cd akka-modules-1.1-SNAPSHOT
export AKKA_HOME=`pwd`
java -jar akka-modules-1.1-SNAPSHOT.jar
The microkernel will boot up and install the sample applications that reside in
the distribution's ``deploy`` directory. You can deploy your own applications
into the ``deploy`` directory as well.
Scripts
-------
Linux/Unix init script
^^^^^^^^^^^^^^^^^^^^^^
Here is a Linux/Unix init script that can be very useful:
http://github.com/jboner/akka/blob/master/scripts/akka-init-script.sh
Copy and modify as needed.
Simple startup shell script
^^^^^^^^^^^^^^^^^^^^^^^^^^^
This little script might help a bit. Just make sure you have the Akka
distribution in the '$AKKA_HOME/dist' directory and then invoke this script to
start up the kernel. The distribution is created in the './dist' dir for you if
you invoke 'sbt dist'.
http://github.com/jboner/akka/blob/master/scripts/run_akka.sh
Copy and modify as needed.
Dependencies
------------
If you are managing dependencies by hand you can find out what all the compile
dependencies are for each module by looking in the ``lib_managed/compile``
directories. For example, you can run this to create a listing of dependencies
(providing you have the source code and have run ``sbt update``)::
cd akka
ls -1 */lib_managed/compile
Dependencies used by the Akka core modules
------------------------------------------
akka-actor
^^^^^^^^^^
* No dependencies
akka-stm
^^^^^^^^
* Depends on akka-actor
* multiverse-alpha-0.6.2.jar
akka-typed-actor
^^^^^^^^^^^^^^^^
* Depends on akka-stm
* aopalliance-1.0.jar
* aspectwerkz-2.2.3.jar
* guice-all-2.0.jar
akka-remote
^^^^^^^^^^^
* Depends on akka-typed-actor
* commons-codec-1.4.jar
* commons-io-2.0.1.jar
* dispatch-json_2.8.1-0.7.8.jar
* guice-all-2.0.jar
* h2-lzf-1.0.jar
* jackson-core-asl-1.7.1.jar
* jackson-mapper-asl-1.7.1.jar
* junit-4.8.1.jar
* netty-3.2.3.Final.jar
* objenesis-1.2.jar
* protobuf-java-2.3.0.jar
* sjson_2.8.1-0.9.1.jar
akka-http
^^^^^^^^^
* Depends on akka-remote
* jsr250-api-1.0.jar
* jsr311-api-1.1.jar
Dependencies used by the Akka modules
-------------------------------------
akka-amqp
^^^^^^^^^
* Depends on akka-remote
* commons-cli-1.1.jar
* amqp-client-1.8.1.jar
akka-camel
^^^^^^^^^^
* Depends on akka-actor
* camel-core-2.7.0.jar
* commons-logging-api-1.1.jar
* commons-management-1.0.jar
akka-camel-typed
^^^^^^^^^^^^^^^^
* Depends on akka-typed-actor
* camel-core-2.7.0.jar
* commons-logging-api-1.1.jar
* commons-management-1.0.jar
akka-spring
^^^^^^^^^^^
* Depends on akka-camel
* akka-camel-typed
* commons-logging-1.1.1.jar
* spring-aop-3.0.4.RELEASE.jar
* spring-asm-3.0.4.RELEASE.jar
* spring-beans-3.0.4.RELEASE.jar
* spring-context-3.0.4.RELEASE.jar
* spring-core-3.0.4.RELEASE.jar
* spring-expression-3.0.4.RELEASE.jar
akka-scalaz
^^^^^^^^^^^
* Depends on akka-actor
* hawtdispatch-1.1.jar
* hawtdispatch-scala-1.1.jar
* scalaz-core_2.8.1-6.0-SNAPSHOT.jar
akka-kernel
^^^^^^^^^^^
* Depends on akka-http, akka-amqp, and akka-spring
* activation-1.1.jar
* asm-3.1.jar
* jaxb-api-2.1.jar
* jaxb-impl-2.1.12.jar
* jersey-core-1.3.jar
* jersey-json-1.3.jar
* jersey-scala-1.3.jar
* jersey-server-1.3.jar
* jettison-1.1.jar
* jetty-continuation-7.1.6.v20100715.jar
* jetty-http-7.1.6.v20100715.jar
* jetty-io-7.1.6.v20100715.jar
* jetty-security-7.1.6.v20100715.jar
* jetty-server-7.1.6.v20100715.jar
* jetty-servlet-7.1.6.v20100715.jar
* jetty-util-7.1.6.v20100715.jar
* jetty-xml-7.1.6.v20100715.jar
* servlet-api-2.5.jar
* stax-api-1.0.1.jar

View file

@ -1,31 +0,0 @@
Configuration
=============
Specifying the configuration file
---------------------------------
If you don't specify a configuration file then Akka uses default values. If
you want to override these then you should edit the ``akka.conf`` file in the
``AKKA_HOME/config`` directory. This config inherits from the
``akka-reference.conf`` file that you see below. Use your ``akka.conf`` to override
any property in the reference config.
The config can be specified in various ways:
* Define the ``-Dakka.config=...`` system property option
* Put an ``akka.conf`` file on the classpath
* Define the ``AKKA_HOME`` environment variable pointing to the root of the Akka
distribution. The config is taken from the ``AKKA_HOME/config`` directory. You
can also point to the AKKA_HOME by specifying the ``-Dakka.home=...`` system
property option.
Defining the configuration file
-------------------------------
Here is the reference configuration file:
.. literalinclude:: ../../config/akka-reference.conf
:language: none

View file

@ -0,0 +1,74 @@
.. _deployment-scenarios:
###################################
Use-case and Deployment Scenarios
###################################
How can I use and deploy Akka?
==============================
Akka can be used in two different ways:
- As a library: used as a regular JAR on the classpath and/or in a web app, to
be put into ``WEB-INF/lib``
- As a microkernel: stand-alone microkernel, embedding a servlet container along
with many other services
Using Akka as library
---------------------
This is most likely what you want if you are building Web applications. There
are several ways you can use Akka in Library mode by adding more and more
modules to the stack.
Actors as services
^^^^^^^^^^^^^^^^^^
The simplest way you can use Akka is to use the actors as services in your Web
application. All thats needed to do that is to put the Akka charts as well as
its dependency jars into ``WEB-INF/lib``. You also need to put the ``akka.conf``
config file in the ``$AKKA_HOME/config`` directory. Now you can create your
Actors as regular services referenced from your Web application. You should also
be able to use the Remoting service, e.g. be able to make certain Actors remote
on other hosts. Please note that remoting service does not speak HTTP over port
80, but a custom protocol over the port is specified in ``akka.conf``.
Using Akka as a stand alone microkernel
---------------------------------------
Akka can also be run as a stand-alone microkernel. It implements a full
enterprise stack. See the :ref:`add-on-modules` for more information.
Using the Akka sbt plugin to package your application
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
The Akka sbt plugin can create a full Akka microkernel deployment for your sbt
project.
To use the plugin, first add a plugin definition to your SBT project by creating
``project/plugins/Plugins.scala`` with::
import sbt._
class Plugins(info: ProjectInfo) extends PluginDefinition(info) {
val akkaRepo = "Akka Repo" at "http://akka.io/repository"
val akkaPlugin = "se.scalablesolutions.akka" % "akka-sbt-plugin" % "1.1"
}
Then mix the ``AkkaKernelProject`` trait into your project definition. For
example::
class MyProject(info: ProjectInfo) extends DefaultProject(info) with AkkaKernelProject
This will automatically add all the Akka dependencies needed for a microkernel
deployment (download them with ``sbt update``).
Place your config files in ``src/main/config``.
To build a microkernel deployment use the ``dist`` task::
sbt dist

View file

@ -1,10 +1,12 @@
.. _getting-started-first-java:
Getting Started Tutorial (Java): First Chapter Getting Started Tutorial (Java): First Chapter
============================================== ==============================================
Introduction Introduction
------------ ------------
Welcome to the first tutorial on how to get started with Akka and Java. We assume that you already know what Akka and Java are and will now focus on the steps necessary to start your first project. Welcome to the first tutorial on how to get started with `Akka <http://akka.io>`_ and Java. We assume that you already know what Akka and Java are and will now focus on the steps necessary to start your first project.
There are two variations of this first tutorial: There are two variations of this first tutorial:
@ -19,14 +21,25 @@ We will be using an algorithm that is called "embarrassingly parallel" which jus
Here is the formula for the algorithm we will use: Here is the formula for the algorithm we will use:
.. image:: pi-formula.png .. image:: ../images/pi-formula.png
In this particular algorithm the master splits the series into chunks which are sent out to each worker actor to be processed. When each worker has processed its chunk it sends a result back to the master which aggregates the total result. In this particular algorithm the master splits the series into chunks which are sent out to each worker actor to be processed. When each worker has processed its chunk it sends a result back to the master which aggregates the total result.
Tutorial source code Tutorial source code
-------------------- --------------------
If you want don't want to type in the code and/or set up a Maven project then you can check out the full tutorial from the Akka GitHub repository. It is in the ``akka-tutorials/akka-tutorial-first`` module. You can also browse it online `here <https://github.com/jboner/akka/tree/master/akka-tutorials/akka-tutorial-first>`_, with the actual source code `here <https://github.com/jboner/akka/blob/master/akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java>`_. If you want don't want to type in the code and/or set up a Maven project then you can check out the full tutorial from the Akka GitHub repository. It is in the ``akka-tutorials/akka-tutorial-first`` module. You can also browse it online `here`__, with the actual source code `here`__.
__ https://github.com/jboner/akka/tree/master/akka-tutorials/akka-tutorial-first
__ https://github.com/jboner/akka/blob/master/akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java
To check out the code using Git invoke the following::
$ git clone git://github.com/jboner/akka.git
Then you can navigate down to the tutorial::
$ cd akka/akka-tutorials/akka-tutorial-first
Prerequisites Prerequisites
------------- -------------
@ -45,39 +58,49 @@ You can test your installation by invoking ``java``::
Java(TM) SE Runtime Environment (build 1.6.0_24-b07-334-10M3326) Java(TM) SE Runtime Environment (build 1.6.0_24-b07-334-10M3326)
Java HotSpot(TM) 64-Bit Server VM (build 19.1-b02-334, mixed mode) Java HotSpot(TM) 64-Bit Server VM (build 19.1-b02-334, mixed mode)
Downloading and installing Akka Downloading and installing Akka
------------------------------- -------------------------------
To build and run the tutorial sample from the command line, you have to download Akka. If you prefer to use SBT to build and run the sample then you can skip this section and jump to the next one. To build and run the tutorial sample from the command line, you have to download
Akka. If you prefer to use SBT to build and run the sample then you can skip
this section and jump to the next one.
Let's get the ``akka-1.1`` distribution of Akka core (not Akka Modules) from `http://akka.io/downloads <http://akka.io/downloads/>`_. Once you have downloaded the distribution unzip it in the folder you would like to have Akka installed in, in my case I choose to install it in ``/Users/jboner/tools/``, simply by unzipping it to this directory. Let's get the ``akka-actors-1.1.zip`` distribution of Akka from
http://akka.io/downloads/ which includes everything we need for this
tutorial. Once you have downloaded the distribution unzip it in the folder you
would like to have Akka installed in. In my case I choose to install it in
``/Users/jboner/tools/``, simply by unzipping it to this directory.
You need to do one more thing in order to install Akka properly: set the ``AKKA_HOME`` environment variable to the root of the distribution. In my case I'm opening up a shell, navigating down to the distribution, and setting the ``AKKA_HOME`` variable:: You need to do one more thing in order to install Akka properly: set the
``AKKA_HOME`` environment variable to the root of the distribution. In my case
I'm opening up a shell, navigating down to the distribution, and setting the
``AKKA_HOME`` variable::
$ cd /Users/jboner/tools/akka-1.1 $ cd /Users/jboner/tools/akka-actors-1.1
$ export AKKA_HOME=`pwd` $ export AKKA_HOME=`pwd`
$ echo $AKKA_HOME $ echo $AKKA_HOME
/Users/jboner/tools/akka-1.1 /Users/jboner/tools/akka-actors-1.1
The distribution looks like this:: The distribution looks like this::
$ ls -l $ ls -1
total 16944 config
drwxr-xr-x 7 jboner staff 238 Apr 6 11:15 . doc
drwxr-xr-x 28 jboner staff 952 Apr 6 11:16 .. lib
drwxr-xr-x 17 jboner staff 578 Apr 6 11:16 deploy src
drwxr-xr-x 26 jboner staff 884 Apr 6 11:16 dist
drwxr-xr-x 3 jboner staff 102 Apr 6 11:15 lib_managed
-rwxr-xr-x 1 jboner staff 8674105 Apr 6 11:15 scala-library.jar
drwxr-xr-x 4 jboner staff 136 Apr 6 11:16 scripts
- In the ``dist`` directory we have the Akka JARs, including sources and docs. - In the ``config`` directory we have the Akka conf files.
- In the ``lib_managed/compile`` directory we have Akka's dependency JARs. - In the ``doc`` directory we have the documentation, API, doc JARs, and also
- In the ``deploy`` directory we have the sample JARs. the source files for the tutorials.
- In the ``scripts`` directory we have scripts for running Akka. - In the ``lib`` directory we have the Scala and Akka JARs.
- Finally ``scala-library.jar`` is the JAR for the latest Scala distribution that Akka depends on. - In the ``src`` directory we have the source JARs for Akka.
The only JAR we will need for this tutorial (apart from the ``scala-library.jar`` JAR) is the ``akka-actor-1.1.jar`` JAR in the ``dist`` directory. This is a self-contained JAR with zero dependencies and contains everything we need to write a system using Actors.
The only JAR we will need for this tutorial (apart from the
``scala-library.jar`` JAR) is the ``akka-actor-1.1.jar`` JAR in the ``lib/akka``
directory. This is a self-contained JAR with zero dependencies and contains
everything we need to write a system using Actors.
Akka is very modular and has many JARs for containing different features. The core distribution has seven modules: Akka is very modular and has many JARs for containing different features. The core distribution has seven modules:
@ -89,7 +112,10 @@ Akka is very modular and has many JARs for containing different features. The co
- ``akka-slf4j-1.1.jar`` -- SLF4J Event Handler Listener for logging with SLF4J - ``akka-slf4j-1.1.jar`` -- SLF4J Event Handler Listener for logging with SLF4J
- ``akka-testkit-1.1.jar`` -- Toolkit for testing Actors - ``akka-testkit-1.1.jar`` -- Toolkit for testing Actors
We also have Akka Modules containing add-on modules outside the core of Akka. You can download the Akka Modules distribution from TODO. It contains Akka core as well. We will not be needing any modules there today, but for your information the module JARs are these: We also have Akka Modules containing add-on modules outside the core of
Akka. You can download the Akka Modules distribution from `<http://akka.io/downloads/>`_. It contains Akka
core as well. We will not be needing any modules there today, but for your
information the module JARs are these:
- ``akka-kernel-1.1.jar`` -- Akka microkernel for running a bare-bones mini application server (embeds Jetty etc.) - ``akka-kernel-1.1.jar`` -- Akka microkernel for running a bare-bones mini application server (embeds Jetty etc.)
- ``akka-amqp-1.1.jar`` -- AMQP integration - ``akka-amqp-1.1.jar`` -- AMQP integration
@ -99,6 +125,7 @@ We also have Akka Modules containing add-on modules outside the core of Akka. Yo
- ``akka-spring-1.1.jar`` -- Spring framework integration - ``akka-spring-1.1.jar`` -- Spring framework integration
- ``akka-osgi-dependencies-bundle-1.1.jar`` -- OSGi support - ``akka-osgi-dependencies-bundle-1.1.jar`` -- OSGi support
Downloading and installing Maven Downloading and installing Maven
-------------------------------- --------------------------------
@ -138,7 +165,9 @@ Here is the layout that Maven created::
As you can see we already have a Java source file called ``App.java``, let's now rename it to ``Pi.java``. As you can see we already have a Java source file called ``App.java``, let's now rename it to ``Pi.java``.
We also need to edit the ``pom.xml`` build file. Let's add the dependency we need as well as the Maven repository it should download it from. It should now look something like this:: We also need to edit the ``pom.xml`` build file. Let's add the dependency we need as well as the Maven repository it should download it from. It should now look something like this:
.. code-block:: xml
<?xml version="1.0" encoding="UTF-8"?> <?xml version="1.0" encoding="UTF-8"?>
<project xmlns="http://maven.apache.org/POM/4.0.0" <project xmlns="http://maven.apache.org/POM/4.0.0"
@ -165,7 +194,7 @@ We also need to edit the ``pom.xml`` build file. Let's add the dependency we nee
<repository> <repository>
<id>Akka</id> <id>Akka</id>
<name>Akka Maven2 Repository</name> <name>Akka Maven2 Repository</name>
<url>http://www.scalablesolutions.se/akka/repository/</url> <url>http://akka.io/repository/</url>
</repository> </repository>
</repositories> </repositories>
@ -265,7 +294,7 @@ Now we can create the worker actor. This is done by extending in the ``UntypedA
Work work = (Work) message; Work work = (Work) message;
// perform the work // perform the work
double result = calculatePiFor(work.getStart(), work.getNrOfElements()) double result = calculatePiFor(work.getStart(), work.getNrOfElements());
// reply with the result // reply with the result
getContext().replyUnsafe(new Result(result)); getContext().replyUnsafe(new Result(result));
@ -465,7 +494,7 @@ Now the only thing that is left to implement is the runner that should bootstrap
pi.calculate(4, 10000, 10000); pi.calculate(4, 10000, 10000);
} }
public void calculate(int nrOfWorkers, int nrOfElements, int nrOfMessages) public void calculate(final int nrOfWorkers, final int nrOfElements, final int nrOfMessages)
throws Exception { throws Exception {
// this latch is only plumbing to know when the calculation is completed // this latch is only plumbing to know when the calculation is completed
@ -661,7 +690,7 @@ Before we package it up and run it, let's take a look at the full code now, with
// ================== // ==================
// ===== Run it ===== // ===== Run it =====
// ================== // ==================
public void calculate(int nrOfWorkers, int nrOfElements, int nrOfMessages) public void calculate(final int nrOfWorkers, final int nrOfElements, final int nrOfMessages)
throws Exception { throws Exception {
// this latch is only plumbing to know when the calculation is completed // this latch is only plumbing to know when the calculation is completed
@ -682,35 +711,42 @@ Before we package it up and run it, let's take a look at the full code now, with
} }
} }
Run it as a command line application Run it as a command line application
------------------------------------ ------------------------------------
To build and run the tutorial from the command line, you need to have the Scala library JAR on the classpath. If you have not typed in (or copied) the code for the tutorial as
``$AKKA_HOME/tutorial/akka/tutorial/first/java/Pi.java`` then now is the
time. When that's done open up a shell and step in to the Akka distribution
(``cd $AKKA_HOME``).
Scala can be downloaded from `http://www.scala-lang.org/downloads <http://www.scala-lang.org/downloads>`_. Browse there and download the Scala 2.9.0.RC1 release. If you pick the ``tgz`` or ``zip`` distribution then just unzip it where you want it installed. If you pick the IzPack Installer then double click on it and follow the instructions. First we need to compile the source file. That is done with Java's compiler
``javac``. Our application depends on the ``akka-actor-1.1.jar`` and the
``scala-library.jar`` JAR files, so let's add them to the compiler classpath
when we compile the source::
The ``scala-library.jar`` resides in the ``scala-2.9.0.RC1/lib`` directory. Copy that to your project directory. $ javac -cp lib/scala-library.jar:lib/akka/akka-actor-1.1.jar tutorial/akka/tutorial/first/java/Pi.java
If you have not typed in (or copied) the code for the tutorial as ``$AKKA_HOME/tutorial/akka/tutorial/first/java/Pi.java`` then now is the time. When that's done open up a shell and step in to the Akka distribution (``cd $AKKA_HOME``). When we have compiled the source file we are ready to run the application. This
is done with ``java`` but yet again we need to add the ``akka-actor-1.1.jar``
First we need to compile the source file. That is done with Java's compiler ``javac``. Our application depends on the ``akka-actor-1.1.jar`` and the ``scala-library.jar`` JAR files, so let's add them to the compiler classpath when we compile the source:: and the ``scala-library.jar`` JAR files to the classpath as well as the classes
we compiled ourselves::
$ javac -cp dist/akka-actor-1.1.jar:scala-library.jar tutorial/Pi.scala
When we have compiled the source file we are ready to run the application. This is done with ``java`` but yet again we need to add the ``akka-actor-1.1.jar`` and the ``scala-library.jar`` JAR files to the classpath as well as the classes we compiled ourselves::
$ java \ $ java \
-cp dist/akka-actor-1.1.jar:scala-library.jar:tutorial \ -cp lib/scala-library.jar:lib/akka/akka-actor-1.1.jar:tutorial \
akka.tutorial.java.first.Pi akka.tutorial.java.first.Pi
AKKA_HOME is defined as [/Users/jboner/src/akka-stuff/akka-core] AKKA_HOME is defined as [/Users/jboner/tools/akka-actors-1.1]
loading config from [/Users/jboner/src/akka-stuff/akka-core/config/akka.conf]. loading config from [/Users/jboner/tools/akka-actors-1.1/config/akka.conf].
Pi estimate: 3.1435501812459323 Pi estimate: 3.1435501812459323
Calculation time: 822 millis Calculation time: 822 millis
Yippee! It is working. Yippee! It is working.
If you have not defined the ``AKKA_HOME`` environment variable then Akka can't find the ``akka.conf`` configuration file and will print out a ``Cant load akka.conf`` warning. This is ok since it will then just use the defaults. If you have not defined the ``AKKA_HOME`` environment variable then Akka can't
find the ``akka.conf`` configuration file and will print out a ``Cant load
akka.conf`` warning. This is ok since it will then just use the defaults.
Run it inside Maven Run it inside Maven
------------------- -------------------
@ -719,7 +755,7 @@ If you used Maven, then you can run the application directly inside Maven. First
$ mvn compile $ mvn compile
When this in done we can run our application directly inside SBT:: When this in done we can run our application directly inside Maven::
$ mvn exec:java -Dexec.mainClass="akka.tutorial.first.java.Pi" $ mvn exec:java -Dexec.mainClass="akka.tutorial.first.java.Pi"
... ...

View file

@ -1,10 +1,12 @@
.. _getting-started-first-scala-eclipse:
Getting Started Tutorial (Scala with Eclipse): First Chapter Getting Started Tutorial (Scala with Eclipse): First Chapter
============================================================ ============================================================
Introduction Introduction
------------ ------------
Welcome to the first tutorial on how to get started with Akka and Scala. We assume that you already know what Akka and Scala are and will now focus on the steps necessary to start your first project. We will be using `Eclipse <http://www.eclipse.org/downloads/>`_, and the `Scala plugin for Eclipse <http://www.scala-ide.org/>`_. Welcome to the first tutorial on how to get started with `Akka <http://akka.io>`_ and `Scala <http://scala-lang.org>`_. We assume that you already know what Akka and Scala are and will now focus on the steps necessary to start your first project. We will be using `Eclipse <http://www.eclipse.org/downloads/>`_, and the `Scala plugin for Eclipse <http://www.scala-ide.org/>`_.
The sample application that we will create is using actors to calculate the value of Pi. Calculating Pi is a CPU intensive operation and we will utilize Akka Actors to write a concurrent solution that scales out to multi-core processors. This sample will be extended in future tutorials to use Akka Remote Actors to scale out on multiple machines in a cluster. The sample application that we will create is using actors to calculate the value of Pi. Calculating Pi is a CPU intensive operation and we will utilize Akka Actors to write a concurrent solution that scales out to multi-core processors. This sample will be extended in future tutorials to use Akka Remote Actors to scale out on multiple machines in a cluster.
@ -12,14 +14,17 @@ We will be using an algorithm that is called "embarrassingly parallel" which jus
Here is the formula for the algorithm we will use: Here is the formula for the algorithm we will use:
.. image:: pi-formula.png .. image:: ../images/pi-formula.png
In this particular algorithm the master splits the series into chunks which are sent out to each worker actor to be processed. When each worker has processed its chunk it sends a result back to the master which aggregates the total result. In this particular algorithm the master splits the series into chunks which are sent out to each worker actor to be processed. When each worker has processed its chunk it sends a result back to the master which aggregates the total result.
Tutorial source code Tutorial source code
-------------------- --------------------
If you want don't want to type in the code and/or set up an SBT project then you can check out the full tutorial from the Akka GitHub repository. It is in the ``akka-tutorials/akka-tutorial-first`` module. You can also browse it online `here <https://github.com/jboner/akka/tree/master/akka-tutorials/akka-tutorial-first>`_, with the actual source code `here <https://github.com/jboner/akka/blob/master/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala>`_. If you want don't want to type in the code and/or set up an SBT project then you can check out the full tutorial from the Akka GitHub repository. It is in the ``akka-tutorials/akka-tutorial-first`` module. You can also browse it online `here`__, with the actual source code `here`__.
__ https://github.com/jboner/akka/tree/master/akka-tutorials/akka-tutorial-first
__ https://github.com/jboner/akka/blob/master/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala
Prerequisites Prerequisites
------------- -------------
@ -38,39 +43,49 @@ You can test your installation by invoking ``java``::
Java(TM) SE Runtime Environment (build 1.6.0_24-b07-334-10M3326) Java(TM) SE Runtime Environment (build 1.6.0_24-b07-334-10M3326)
Java HotSpot(TM) 64-Bit Server VM (build 19.1-b02-334, mixed mode) Java HotSpot(TM) 64-Bit Server VM (build 19.1-b02-334, mixed mode)
Downloading and installing Akka Downloading and installing Akka
------------------------------- -------------------------------
To build and run the tutorial sample from the command line, you have to download Akka. If you prefer to use SBT to build and run the sample then you can skip this section and jump to the next one. To build and run the tutorial sample from the command line, you have to download
Akka. If you prefer to use SBT to build and run the sample then you can skip
this section and jump to the next one.
Let's get the ``akka-1.1`` distribution of Akka core (not Akka Modules) from `http://akka.io/downloads <http://akka.io/downloads/>`_. Once you have downloaded the distribution unzip it in the folder you would like to have Akka installed in, in my case I choose to install it in ``/Users/jboner/tools/``, simply by unzipping it to this directory. Let's get the ``akka-actors-1.1.zip`` distribution of Akka from
http://akka.io/downloads/ which includes everything we need for this
tutorial. Once you have downloaded the distribution unzip it in the folder you
would like to have Akka installed in. In my case I choose to install it in
``/Users/jboner/tools/``, simply by unzipping it to this directory.
You need to do one more thing in order to install Akka properly: set the ``AKKA_HOME`` environment variable to the root of the distribution. In my case I'm opening up a shell, navigating down to the distribution, and setting the ``AKKA_HOME`` variable:: You need to do one more thing in order to install Akka properly: set the
``AKKA_HOME`` environment variable to the root of the distribution. In my case
I'm opening up a shell, navigating down to the distribution, and setting the
``AKKA_HOME`` variable::
$ cd /Users/jboner/tools/akka-1.1 $ cd /Users/jboner/tools/akka-actors-1.1
$ export AKKA_HOME=`pwd` $ export AKKA_HOME=`pwd`
$ echo $AKKA_HOME $ echo $AKKA_HOME
/Users/jboner/tools/akka-1.1 /Users/jboner/tools/akka-actors-1.1
The distribution looks like this:: The distribution looks like this::
$ ls -l $ ls -1
total 16944 config
drwxr-xr-x 7 jboner staff 238 Apr 6 11:15 . doc
drwxr-xr-x 28 jboner staff 952 Apr 6 11:16 .. lib
drwxr-xr-x 17 jboner staff 578 Apr 6 11:16 deploy src
drwxr-xr-x 26 jboner staff 884 Apr 6 11:16 dist
drwxr-xr-x 3 jboner staff 102 Apr 6 11:15 lib_managed
-rwxr-xr-x 1 jboner staff 8674105 Apr 6 11:15 scala-library.jar
drwxr-xr-x 4 jboner staff 136 Apr 6 11:16 scripts
- In the ``dist`` directory we have the Akka JARs, including sources and docs. - In the ``config`` directory we have the Akka conf files.
- In the ``lib_managed/compile`` directory we have Akka's dependency JARs. - In the ``doc`` directory we have the documentation, API, doc JARs, and also
- In the ``deploy`` directory we have the sample JARs. the source files for the tutorials.
- In the ``scripts`` directory we have scripts for running Akka. - In the ``lib`` directory we have the Scala and Akka JARs.
- Finally ``scala-library.jar`` is the JAR for the latest Scala distribution that Akka depends on. - In the ``src`` directory we have the source JARs for Akka.
The only JAR we will need for this tutorial (apart from the ``scala-library.jar`` JAR) is the ``akka-actor-1.1.jar`` JAR in the ``dist`` directory. This is a self-contained JAR with zero dependencies and contains everything we need to write a system using Actors.
The only JAR we will need for this tutorial (apart from the
``scala-library.jar`` JAR) is the ``akka-actor-1.1.jar`` JAR in the ``lib/akka``
directory. This is a self-contained JAR with zero dependencies and contains
everything we need to write a system using Actors.
Akka is very modular and has many JARs for containing different features. The core distribution has seven modules: Akka is very modular and has many JARs for containing different features. The core distribution has seven modules:
@ -79,10 +94,13 @@ Akka is very modular and has many JARs for containing different features. The co
- ``akka-remote-1.1.jar`` -- Remote Actors - ``akka-remote-1.1.jar`` -- Remote Actors
- ``akka-stm-1.1.jar`` -- STM (Software Transactional Memory), transactors and transactional datastructures - ``akka-stm-1.1.jar`` -- STM (Software Transactional Memory), transactors and transactional datastructures
- ``akka-http-1.1.jar`` -- Akka Mist for continuation-based asynchronous HTTP and also Jersey integration - ``akka-http-1.1.jar`` -- Akka Mist for continuation-based asynchronous HTTP and also Jersey integration
- ``akka-slf4j-1.1.jar`` -- SLF4J Event Handler Listener - ``akka-slf4j-1.1.jar`` -- SLF4J Event Handler Listener for logging with SLF4J
- ``akka-testkit-1.1.jar`` -- Toolkit for testing Actors - ``akka-testkit-1.1.jar`` -- Toolkit for testing Actors
We also have Akka Modules containing add-on modules outside the core of Akka. You can download the Akka Modules distribution from TODO. It contains Akka core as well. We will not be needing any modules there today, but for your information the module JARs are these: We also have Akka Modules containing add-on modules outside the core of
Akka. You can download the Akka Modules distribution from `<http://akka.io/downloads/>`_. It contains Akka
core as well. We will not be needing any modules there today, but for your
information the module JARs are these:
- ``akka-kernel-1.1.jar`` -- Akka microkernel for running a bare-bones mini application server (embeds Jetty etc.) - ``akka-kernel-1.1.jar`` -- Akka microkernel for running a bare-bones mini application server (embeds Jetty etc.)
- ``akka-amqp-1.1.jar`` -- AMQP integration - ``akka-amqp-1.1.jar`` -- AMQP integration
@ -92,6 +110,7 @@ We also have Akka Modules containing add-on modules outside the core of Akka. Yo
- ``akka-spring-1.1.jar`` -- Spring framework integration - ``akka-spring-1.1.jar`` -- Spring framework integration
- ``akka-osgi-dependencies-bundle-1.1.jar`` -- OSGi support - ``akka-osgi-dependencies-bundle-1.1.jar`` -- OSGi support
Downloading and installing the Scala IDE for Eclipse Downloading and installing the Scala IDE for Eclipse
---------------------------------------------------- ----------------------------------------------------
@ -99,19 +118,19 @@ If you want to use Eclipse for coding your Akka tutorial, you need to install th
You can install this plugin using the regular update mechanism. First choose a version of the IDE from `http://download.scala-ide.org <http://download.scala-ide.org>`_. We recommend you choose 2.0.x, which comes with Scala 2.9. Copy the corresponding URL and then choose ``Help/Install New Software`` and paste the URL you just copied. You should see something similar to the following image. You can install this plugin using the regular update mechanism. First choose a version of the IDE from `http://download.scala-ide.org <http://download.scala-ide.org>`_. We recommend you choose 2.0.x, which comes with Scala 2.9. Copy the corresponding URL and then choose ``Help/Install New Software`` and paste the URL you just copied. You should see something similar to the following image.
.. image:: install-beta2-updatesite.png .. image:: ../images/install-beta2-updatesite.png
Make sure you select both the ``JDT Weaving for Scala`` and the ``Scala IDE for Eclipse`` plugins. The other plugin is optional, and contains the source code of the plugin itself. Make sure you select both the ``JDT Weaving for Scala`` and the ``Scala IDE for Eclipse`` plugins. The other plugin is optional, and contains the source code of the plugin itself.
Once the installation is finished, you need to restart Eclipse. The first time the plugin starts it will open a diagnostics window and offer to fix several settings, such as the delay for content assist (code-completion) or the shown completion proposal types. Once the installation is finished, you need to restart Eclipse. The first time the plugin starts it will open a diagnostics window and offer to fix several settings, such as the delay for content assist (code-completion) or the shown completion proposal types.
.. image:: diagnostics-window.png .. image:: ../images/diagnostics-window.png
Accept the recommended settings, and follow the instructions if you need to increase the heap size of Eclipse. Accept the recommended settings, and follow the instructions if you need to increase the heap size of Eclipse.
Check that the installation succeeded by creating a new Scala project (``File/New>Scala Project``), and typing some code. You should have content-assist, hyperlinking to definitions, instant error reporting, and so on. Check that the installation succeeded by creating a new Scala project (``File/New>Scala Project``), and typing some code. You should have content-assist, hyperlinking to definitions, instant error reporting, and so on.
.. image:: example-code.png .. image:: ../images/example-code.png
You are ready to code now! You are ready to code now!
@ -120,16 +139,16 @@ Downloading and installing Scala
To build and run the tutorial sample from the command line, you have to install the Scala distribution. If you prefer to use Eclipse to build and run the sample then you can skip this section and jump to the next one. To build and run the tutorial sample from the command line, you have to install the Scala distribution. If you prefer to use Eclipse to build and run the sample then you can skip this section and jump to the next one.
Scala can be downloaded from `http://www.scala-lang.org/downloads <http://www.scala-lang.org/downloads>`_. Browse there and download the Scala 2.9.0.RC1 release. If you pick the ``tgz`` or ``zip`` distribution then just unzip it where you want it installed. If you pick the IzPack Installer then double click on it and follow the instructions. Scala can be downloaded from `http://www.scala-lang.org/downloads <http://www.scala-lang.org/downloads>`_. Browse there and download the Scala 2.9.0 release. If you pick the ``tgz`` or ``zip`` distribution then just unzip it where you want it installed. If you pick the IzPack Installer then double click on it and follow the instructions.
You also need to make sure that the ``scala-2.9.0.RC1/bin`` (if that is the directory where you installed Scala) is on your ``PATH``:: You also need to make sure that the ``scala-2.9.0/bin`` (if that is the directory where you installed Scala) is on your ``PATH``::
$ export PATH=$PATH:scala-2.9.0.RC1/bin $ export PATH=$PATH:scala-2.9.0/bin
You can test your installation by invoking scala:: You can test your installation by invoking scala::
$ scala -version $ scala -version
Scala code runner version 2.9.0.RC1 -- Copyright 2002-2011, LAMP/EPFL Scala code runner version 2.9.0.final -- Copyright 2002-2011, LAMP/EPFL
Looks like we are all good. Finally let's create a source file ``Pi.scala`` for the tutorial and put it in the root of the Akka distribution in the ``tutorial`` directory (you have to create it first). Looks like we are all good. Finally let's create a source file ``Pi.scala`` for the tutorial and put it in the root of the Akka distribution in the ``tutorial`` directory (you have to create it first).
@ -140,12 +159,12 @@ Creating an Akka project in Eclipse
If you have not already done so, now is the time to create an Eclipse project for our tutorial. Use the ``New Scala Project`` wizard and accept the default settings. Once the project is open, we need to add the akka libraries to the *build path*. Right click on the project and choose ``Properties``, then click on ``Java Build Path``. Go to ``Libraries`` and click on ``Add External Jars..``, then navigate to the location where you installed akka and choose ``akka-actor.jar``. You should see something similar to this: If you have not already done so, now is the time to create an Eclipse project for our tutorial. Use the ``New Scala Project`` wizard and accept the default settings. Once the project is open, we need to add the akka libraries to the *build path*. Right click on the project and choose ``Properties``, then click on ``Java Build Path``. Go to ``Libraries`` and click on ``Add External Jars..``, then navigate to the location where you installed akka and choose ``akka-actor.jar``. You should see something similar to this:
.. image:: build-path.png .. image:: ../images/build-path.png
Using SBT in Eclipse Using SBT in Eclipse
^^^^^^^^^^^^^^^^^^^^ ^^^^^^^^^^^^^^^^^^^^
If you are an `SBT <http://code.google.com/p/simple-build-tool/>`_ user, you can follow the :doc:`Akka Tutorial in Scala </intro/getting-started-first-scala>` and additionally install the ``sbt-eclipse`` plugin. This adds support for generating Eclipse project files from your SBT project. You need to update your SBT plugins definition in ``project/plugins``:: If you are an `SBT <http://code.google.com/p/simple-build-tool/>`_ user, you can follow the :ref:`getting-started-first-scala-download-sbt` instruction and additionally install the ``sbt-eclipse`` plugin. This adds support for generating Eclipse project files from your SBT project. You need to update your SBT plugins definition in ``project/plugins``::
import sbt._ import sbt._
@ -171,7 +190,7 @@ and then update your SBT project definition by mixing in ``Eclipsify`` in your p
Then run the ``eclipse`` target to generate the Eclipse project:: Then run the ``eclipse`` target to generate the Eclipse project::
dragos@dragos-imac pi $ sbt eclipse dragos@dragos-imac pi $ sbt eclipse
[info] Building project AkkaPi 1.0 against Scala 2.9.0.RC1 [info] Building project AkkaPi 1.0 against Scala 2.9.0
[info] using MySbtProject with sbt 0.7.4 and Scala 2.7.7 [info] using MySbtProject with sbt 0.7.4 and Scala 2.7.7
[info] [info]
[info] == eclipse == [info] == eclipse ==
@ -186,7 +205,7 @@ Then run the ``eclipse`` target to generate the Eclipse project::
Next you need to import this project in Eclipse, by choosing ``Eclipse/Import.. Existing Projects into Workspace``. Navigate to the directory where you defined your SBT project and choose import: Next you need to import this project in Eclipse, by choosing ``Eclipse/Import.. Existing Projects into Workspace``. Navigate to the directory where you defined your SBT project and choose import:
.. image:: import-project.png .. image:: ../images/import-project.png
Now we have the basis for an Akka Eclipse application, so we can.. Now we have the basis for an Akka Eclipse application, so we can..
@ -234,7 +253,7 @@ Now we can create the worker actor. Create a new class called ``Worker`` as bef
The ``Actor`` trait is defined in ``akka.actor`` and you can either import it explicitly, or let Eclipse do it for you when it cannot resolve the ``Actor`` trait. The quick fix option (``Ctrl-F1``) will offer two options: The ``Actor`` trait is defined in ``akka.actor`` and you can either import it explicitly, or let Eclipse do it for you when it cannot resolve the ``Actor`` trait. The quick fix option (``Ctrl-F1``) will offer two options:
.. image:: quickfix.png .. image:: ../images/quickfix.png
Choose the Akka Actor and move on. Choose the Akka Actor and move on.
@ -307,11 +326,11 @@ Here is the master actor::
def receive = { ... } def receive = { ... }
override def preStart { override def preStart() {
start = System.currentTimeMillis start = System.currentTimeMillis
} }
override def postStop { override def postStop() {
// tell the world that the calculation is complete // tell the world that the calculation is complete
println( println(
"\n\tPi estimate: \t\t%s\n\tCalculation time: \t%s millis" "\n\tPi estimate: \t\t%s\n\tCalculation time: \t%s millis"
@ -393,8 +412,8 @@ Run it from Eclipse
Eclipse builds your project on every save when ``Project/Build Automatically`` is set. If not, bring you project up to date by clicking ``Project/Build Project``. If there are no compilation errors, you can right-click in the editor where ``Pi`` is defined, and choose ``Run as.. /Scala application``. If everything works fine, you should see:: Eclipse builds your project on every save when ``Project/Build Automatically`` is set. If not, bring you project up to date by clicking ``Project/Build Project``. If there are no compilation errors, you can right-click in the editor where ``Pi`` is defined, and choose ``Run as.. /Scala application``. If everything works fine, you should see::
AKKA_HOME is defined as [/Users/jboner/tools/akka-modules-1.1-M1/] AKKA_HOME is defined as [/Users/jboner/tools/akka-actors-1.1]
loading config from [/Users/jboner/tools/akka-modules-1.1-M1/config/akka.conf]. loading config from [/Users/jboner/tools/akka-actors-1.1/config/akka.conf].
Pi estimate: 3.1435501812459323 Pi estimate: 3.1435501812459323
Calculation time: 858 millis Calculation time: 858 millis
@ -403,7 +422,7 @@ If you have not defined an the ``AKKA_HOME`` environment variable then Akka can'
You can also define a new Run configuration, by going to ``Run/Run Configurations``. Create a new ``Scala application`` and choose the tutorial project and the main class to be ``akkatutorial.Pi``. You can pass additional command line arguments to the JVM on the ``Arguments`` page, for instance to define where ``akka.conf`` is: You can also define a new Run configuration, by going to ``Run/Run Configurations``. Create a new ``Scala application`` and choose the tutorial project and the main class to be ``akkatutorial.Pi``. You can pass additional command line arguments to the JVM on the ``Arguments`` page, for instance to define where ``akka.conf`` is:
.. image:: run-config.png .. image:: ../images/run-config.png
Once you finished your run configuration, click ``Run``. You should see the same output in the ``Console`` window. You can use the same configuration for debugging the application, by choosing ``Run/Debug History`` or just ``Debug As``. Once you finished your run configuration, click ``Run``. You should see the same output in the ``Console`` window. You can use the same configuration for debugging the application, by choosing ``Run/Debug History`` or just ``Debug As``.

View file

@ -1,3 +1,5 @@
.. _getting-started-first-scala:
Getting Started Tutorial (Scala): First Chapter Getting Started Tutorial (Scala): First Chapter
=============================================== ===============================================
@ -19,19 +21,30 @@ We will be using an algorithm that is called "embarrassingly parallel" which jus
Here is the formula for the algorithm we will use: Here is the formula for the algorithm we will use:
.. image:: pi-formula.png .. image:: ../images/pi-formula.png
In this particular algorithm the master splits the series into chunks which are sent out to each worker actor to be processed. When each worker has processed its chunk it sends a result back to the master which aggregates the total result. In this particular algorithm the master splits the series into chunks which are sent out to each worker actor to be processed. When each worker has processed its chunk it sends a result back to the master which aggregates the total result.
Tutorial source code Tutorial source code
-------------------- --------------------
If you want don't want to type in the code and/or set up an SBT project then you can check out the full tutorial from the Akka GitHub repository. It is in the ``akka-tutorials/akka-tutorial-first`` module. You can also browse it online `here <https://github.com/jboner/akka/tree/master/akka-tutorials/akka-tutorial-first>`_, with the actual source code `here <https://github.com/jboner/akka/blob/master/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala>`_. If you want don't want to type in the code and/or set up an SBT project then you can check out the full tutorial from the Akka GitHub repository. It is in the ``akka-tutorials/akka-tutorial-first`` module. You can also browse it online `here`__, with the actual source code `here`__.
__ https://github.com/jboner/akka/tree/master/akka-tutorials/akka-tutorial-first
__ https://github.com/jboner/akka/blob/master/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala
To check out the code using Git invoke the following::
$ git clone git://github.com/jboner/akka.git
Then you can navigate down to the tutorial::
$ cd akka/akka-tutorials/akka-tutorial-first
Prerequisites Prerequisites
------------- -------------
This tutorial assumes that you have Jave 1.6 or later installed on you machine and ``java`` on your ``PATH``. You also need to know how to run commands in a shell (ZSH, Bash, DOS etc.) and a decent text editor or IDE to type in the Scala code. This tutorial assumes that you have Java 1.6 or later installed on you machine and ``java`` on your ``PATH``. You also need to know how to run commands in a shell (ZSH, Bash, DOS etc.) and a decent text editor or IDE to type in the Scala code.
You need to make sure that ``$JAVA_HOME`` environment variable is set to the root of the Java distribution. You also need to make sure that the ``$JAVA_HOME/bin`` is on your ``PATH``:: You need to make sure that ``$JAVA_HOME`` environment variable is set to the root of the Java distribution. You also need to make sure that the ``$JAVA_HOME/bin`` is on your ``PATH``::
@ -45,39 +58,49 @@ You can test your installation by invoking ``java``::
Java(TM) SE Runtime Environment (build 1.6.0_24-b07-334-10M3326) Java(TM) SE Runtime Environment (build 1.6.0_24-b07-334-10M3326)
Java HotSpot(TM) 64-Bit Server VM (build 19.1-b02-334, mixed mode) Java HotSpot(TM) 64-Bit Server VM (build 19.1-b02-334, mixed mode)
Downloading and installing Akka Downloading and installing Akka
------------------------------- -------------------------------
To build and run the tutorial sample from the command line, you have to download Akka. If you prefer to use SBT to build and run the sample then you can skip this section and jump to the next one. To build and run the tutorial sample from the command line, you have to download
Akka. If you prefer to use SBT to build and run the sample then you can skip
this section and jump to the next one.
Let's get the ``akka-1.1`` distribution of Akka core (not Akka Modules) from `http://akka.io/downloads <http://akka.io/downloads/>`_. Once you have downloaded the distribution unzip it in the folder you would like to have Akka installed in, in my case I choose to install it in ``/Users/jboner/tools/``, simply by unzipping it to this directory. Let's get the ``akka-actors-1.1.zip`` distribution of Akka from
http://akka.io/downloads/ which includes everything we need for this
tutorial. Once you have downloaded the distribution unzip it in the folder you
would like to have Akka installed in. In my case I choose to install it in
``/Users/jboner/tools/``, simply by unzipping it to this directory.
You need to do one more thing in order to install Akka properly: set the ``AKKA_HOME`` environment variable to the root of the distribution. In my case I'm opening up a shell, navigating down to the distribution, and setting the ``AKKA_HOME`` variable:: You need to do one more thing in order to install Akka properly: set the
``AKKA_HOME`` environment variable to the root of the distribution. In my case
I'm opening up a shell, navigating down to the distribution, and setting the
``AKKA_HOME`` variable::
$ cd /Users/jboner/tools/akka-1.1 $ cd /Users/jboner/tools/akka-actors-1.1
$ export AKKA_HOME=`pwd` $ export AKKA_HOME=`pwd`
$ echo $AKKA_HOME $ echo $AKKA_HOME
/Users/jboner/tools/akka-1.1 /Users/jboner/tools/akka-actors-1.1
The distribution looks like this:: The distribution looks like this::
$ ls -l $ ls -1
total 16944 config
drwxr-xr-x 7 jboner staff 238 Apr 6 11:15 . doc
drwxr-xr-x 28 jboner staff 952 Apr 6 11:16 .. lib
drwxr-xr-x 17 jboner staff 578 Apr 6 11:16 deploy src
drwxr-xr-x 26 jboner staff 884 Apr 6 11:16 dist
drwxr-xr-x 3 jboner staff 102 Apr 6 11:15 lib_managed
-rwxr-xr-x 1 jboner staff 8674105 Apr 6 11:15 scala-library.jar
drwxr-xr-x 4 jboner staff 136 Apr 6 11:16 scripts
- In the ``dist`` directory we have the Akka JARs, including sources and docs. - In the ``config`` directory we have the Akka conf files.
- In the ``lib_managed/compile`` directory we have Akka's dependency JARs. - In the ``doc`` directory we have the documentation, API, doc JARs, and also
- In the ``deploy`` directory we have the sample JARs. the source files for the tutorials.
- In the ``scripts`` directory we have scripts for running Akka. - In the ``lib`` directory we have the Scala and Akka JARs.
- Finally ``scala-library.jar`` is the JAR for the latest Scala distribution that Akka depends on. - In the ``src`` directory we have the source JARs for Akka.
The only JAR we will need for this tutorial (apart from the ``scala-library.jar`` JAR) is the ``akka-actor-1.1.jar`` JAR in the ``dist`` directory. This is a self-contained JAR with zero dependencies and contains everything we need to write a system using Actors.
The only JAR we will need for this tutorial (apart from the
``scala-library.jar`` JAR) is the ``akka-actor-1.1.jar`` JAR in the ``lib/akka``
directory. This is a self-contained JAR with zero dependencies and contains
everything we need to write a system using Actors.
Akka is very modular and has many JARs for containing different features. The core distribution has seven modules: Akka is very modular and has many JARs for containing different features. The core distribution has seven modules:
@ -86,10 +109,13 @@ Akka is very modular and has many JARs for containing different features. The co
- ``akka-remote-1.1.jar`` -- Remote Actors - ``akka-remote-1.1.jar`` -- Remote Actors
- ``akka-stm-1.1.jar`` -- STM (Software Transactional Memory), transactors and transactional datastructures - ``akka-stm-1.1.jar`` -- STM (Software Transactional Memory), transactors and transactional datastructures
- ``akka-http-1.1.jar`` -- Akka Mist for continuation-based asynchronous HTTP and also Jersey integration - ``akka-http-1.1.jar`` -- Akka Mist for continuation-based asynchronous HTTP and also Jersey integration
- ``akka-slf4j-1.1.jar`` -- SLF4J Event Handler Listener - ``akka-slf4j-1.1.jar`` -- SLF4J Event Handler Listener for logging with SLF4J
- ``akka-testkit-1.1.jar`` -- Toolkit for testing Actors - ``akka-testkit-1.1.jar`` -- Toolkit for testing Actors
We also have Akka Modules containing add-on modules outside the core of Akka. You can download the Akka Modules distribution from TODO. It contains Akka core as well. We will not be needing any modules there today, but for your information the module JARs are these: We also have Akka Modules containing add-on modules outside the core of
Akka. You can download the Akka Modules distribution from `<http://akka.io/downloads/>`_. It contains Akka
core as well. We will not be needing any modules there today, but for your
information the module JARs are these:
- ``akka-kernel-1.1.jar`` -- Akka microkernel for running a bare-bones mini application server (embeds Jetty etc.) - ``akka-kernel-1.1.jar`` -- Akka microkernel for running a bare-bones mini application server (embeds Jetty etc.)
- ``akka-amqp-1.1.jar`` -- AMQP integration - ``akka-amqp-1.1.jar`` -- AMQP integration
@ -99,26 +125,29 @@ We also have Akka Modules containing add-on modules outside the core of Akka. Yo
- ``akka-spring-1.1.jar`` -- Spring framework integration - ``akka-spring-1.1.jar`` -- Spring framework integration
- ``akka-osgi-dependencies-bundle-1.1.jar`` -- OSGi support - ``akka-osgi-dependencies-bundle-1.1.jar`` -- OSGi support
Downloading and installing Scala Downloading and installing Scala
-------------------------------- --------------------------------
To build and run the tutorial sample from the command line, you have to install the Scala distribution. If you prefer to use SBT to build and run the sample then you can skip this section and jump to the next one. To build and run the tutorial sample from the command line, you have to install the Scala distribution. If you prefer to use SBT to build and run the sample then you can skip this section and jump to the next one.
Scala can be downloaded from `http://www.scala-lang.org/downloads <http://www.scala-lang.org/downloads>`_. Browse there and download the Scala 2.9.0.RC1 release. If you pick the ``tgz`` or ``zip`` distribution then just unzip it where you want it installed. If you pick the IzPack Installer then double click on it and follow the instructions. Scala can be downloaded from `http://www.scala-lang.org/downloads <http://www.scala-lang.org/downloads>`_. Browse there and download the Scala 2.9.0 release. If you pick the ``tgz`` or ``zip`` distribution then just unzip it where you want it installed. If you pick the IzPack Installer then double click on it and follow the instructions.
You also need to make sure that the ``scala-2.9.0.RC1/bin`` (if that is the directory where you installed Scala) is on your ``PATH``:: You also need to make sure that the ``scala-2.9.0/bin`` (if that is the directory where you installed Scala) is on your ``PATH``::
$ export PATH=$PATH:scala-2.9.0.RC1/bin $ export PATH=$PATH:scala-2.9.0/bin
You can test your installation by invoking scala:: You can test your installation by invoking scala::
$ scala -version $ scala -version
Scala code runner version 2.9.0.RC1 -- Copyright 2002-2011, LAMP/EPFL Scala code runner version 2.9.0.final -- Copyright 2002-2011, LAMP/EPFL
Looks like we are all good. Finally let's create a source file ``Pi.scala`` for the tutorial and put it in the root of the Akka distribution in the ``tutorial`` directory (you have to create it first). Looks like we are all good. Finally let's create a source file ``Pi.scala`` for the tutorial and put it in the root of the Akka distribution in the ``tutorial`` directory (you have to create it first).
Some tools require you to set the ``SCALA_HOME`` environment variable to the root of the Scala distribution, however Akka does not require that. Some tools require you to set the ``SCALA_HOME`` environment variable to the root of the Scala distribution, however Akka does not require that.
.. _getting-started-first-scala-download-sbt:
Downloading and installing SBT Downloading and installing SBT
------------------------------ ------------------------------
@ -140,7 +169,7 @@ If you have not already done so, now is the time to create an SBT project for ou
Name: Tutorial 1 Name: Tutorial 1
Organization: Hakkers Inc Organization: Hakkers Inc
Version [1.0]: Version [1.0]:
Scala version [2.9.0.RC1]: Scala version [2.9.0]:
sbt version [0.7.6.RC0]: sbt version [0.7.6.RC0]:
Now we have the basis for an SBT project. Akka has an SBT Plugin making it very easy to use Akka is an SBT-based project so let's use that. Now we have the basis for an SBT project. Akka has an SBT Plugin making it very easy to use Akka is an SBT-based project so let's use that.
@ -158,9 +187,7 @@ Now we need to create a project definition using our Akka SBT plugin. We do that
import sbt._ import sbt._
class TutorialOneProject(info: ProjectInfo) extends DefaultProject(info) with AkkaProject { class TutorialOneProject(info: ProjectInfo) extends DefaultProject(info) with AkkaProject
val akkaRepo = "Akka Repo" at "http://akka.io/repository"
}
The magic is in mixing in the ``AkkaProject`` trait. The magic is in mixing in the ``AkkaProject`` trait.
@ -173,8 +200,11 @@ Not needed in this tutorial, but if you would like to use additional Akka module
So, now we are all set. Just one final thing to do; make SBT download the dependencies it needs. That is done by invoking:: So, now we are all set. Just one final thing to do; make SBT download the dependencies it needs. That is done by invoking::
> reload
> update > update
The first reload command is needed because we have changed the project definition since the sbt session started.
SBT itself needs a whole bunch of dependencies but our project will only need one; ``akka-actor-1.1.jar``. SBT downloads that as well. SBT itself needs a whole bunch of dependencies but our project will only need one; ``akka-actor-1.1.jar``. SBT downloads that as well.
Start writing the code Start writing the code
@ -291,11 +321,11 @@ Here is the master actor::
def receive = { ... } def receive = { ... }
override def preStart { override def preStart() {
start = System.currentTimeMillis start = System.currentTimeMillis
} }
override def postStop { override def postStop() {
// tell the world that the calculation is complete // tell the world that the calculation is complete
println( println(
"\n\tPi estimate: \t\t%s\n\tCalculation time: \t%s millis" "\n\tPi estimate: \t\t%s\n\tCalculation time: \t%s millis"
@ -451,11 +481,11 @@ But before we package it up and run it, let's take a look at the full code now,
if (nrOfResults == nrOfMessages) self.stop() if (nrOfResults == nrOfMessages) self.stop()
} }
override def preStart { override def preStart() {
start = System.currentTimeMillis start = System.currentTimeMillis
} }
override def postStop { override def postStop() {
// tell the world that the calculation is complete // tell the world that the calculation is complete
println( println(
"\n\tPi estimate: \t\t%s\n\tCalculation time: \t%s millis" "\n\tPi estimate: \t\t%s\n\tCalculation time: \t%s millis"
@ -491,15 +521,15 @@ If you have not typed in (or copied) the code for the tutorial as ``$AKKA_HOME/t
First we need to compile the source file. That is done with Scala's compiler ``scalac``. Our application depends on the ``akka-actor-1.1.jar`` JAR file, so let's add that to the compiler classpath when we compile the source:: First we need to compile the source file. That is done with Scala's compiler ``scalac``. Our application depends on the ``akka-actor-1.1.jar`` JAR file, so let's add that to the compiler classpath when we compile the source::
$ scalac -cp dist/akka-actor-1.1.jar tutorial/Pi.scala $ scalac -cp lib/akka/akka-actor-1.1.jar tutorial/Pi.scala
When we have compiled the source file we are ready to run the application. This is done with ``java`` but yet again we need to add the ``akka-actor-1.1.jar`` JAR file to the classpath, and this time we also need to add the Scala runtime library ``scala-library.jar`` and the classes we compiled ourselves:: When we have compiled the source file we are ready to run the application. This is done with ``java`` but yet again we need to add the ``akka-actor-1.1.jar`` JAR file to the classpath, and this time we also need to add the Scala runtime library ``scala-library.jar`` and the classes we compiled ourselves::
$ java \ $ java \
-cp dist/akka-actor-1.1.jar:scala-library.jar:tutorial \ -cp lib/scala-library.jar:lib/akka/akka-actor-1.1.jar:. \
akka.tutorial.first.scala.Pi akka.tutorial.first.scala.Pi
AKKA_HOME is defined as [/Users/jboner/src/akka-stuff/akka-core] AKKA_HOME is defined as [/Users/jboner/tools/akka-actors-1.1]
loading config from [/Users/jboner/src/akka-stuff/akka-core/config/akka.conf]. loading config from [/Users/jboner/tools/akka-actors-1.1/config/akka.conf].
Pi estimate: 3.1435501812459323 Pi estimate: 3.1435501812459323
Calculation time: 858 millis Calculation time: 858 millis

View file

@ -0,0 +1,198 @@
Getting Started
===============
.. sidebar:: Contents
.. contents:: :local:
The best way to start learning Akka is to try the Getting Started Tutorial, which comes in several flavours
depending on you development environment preferences:
- :ref:`getting-started-first-java` for Java development, either
- as standalone project, running from the command line,
- or as Maven project and running it from within Maven
- :ref:`getting-started-first-scala` for Scala development, either
- as standalone project, running from the command line,
- or as SBT (Simple Build Tool) project and running it from within SBT
- :ref:`getting-started-first-scala-eclipse` for Scala development with Eclipse
The Getting Started Tutorial describes everything you need to get going, and you don't need to read the rest of
this page if you study the tutorial. For later look back reference this page describes the
essential parts for getting started with different development environments.
Prerequisites
-------------
Akka requires that you have `Java 1.6 <http://www.oracle.com/technetwork/java/javase/downloads/index.html>`_ or
later installed on you machine.
Download
--------
There are several ways to download Akka. You can download the full distribution with microkernel, which includes
all modules. You can download just the core distribution. Or you can use a build tool like Maven or SBT to download
dependencies from the Akka Maven repository.
Modules
-------
Akka is split up into two different parts:
* Akka - The core modules. Reflects all the sections under :ref:`scala-api` and :ref:`java-api`.
* Akka Modules - The microkernel and add-on modules, described in :ref:`add-on-modules`.
Akka is very modular and has many JARs for containing different features. The core distribution has seven modules:
- ``akka-actor-1.1.jar`` -- Standard Actors
- ``akka-typed-actor-1.1.jar`` -- Typed Actors
- ``akka-remote-1.1.jar`` -- Remote Actors
- ``akka-stm-1.1.jar`` -- STM (Software Transactional Memory), transactors and transactional datastructures
- ``akka-http-1.1.jar`` -- Akka Mist for continuation-based asynchronous HTTP and also Jersey integration
- ``akka-slf4j-1.1.jar`` -- SLF4J Event Handler Listener
- ``akka-testkit-1.1.jar`` -- Toolkit for testing Actors
We also have Akka Modules containing add-on modules outside the core of Akka.
- ``akka-kernel-1.1.jar`` -- Akka microkernel for running a bare-bones mini application server (embeds Jetty etc.)
- ``akka-amqp-1.1.jar`` -- AMQP integration
- ``akka-camel-1.1.jar`` -- Apache Camel Actors integration (it's the best way to have your Akka application communicate with the rest of the world)
- ``akka-camel-typed-1.1.jar`` -- Apache Camel Typed Actors integration
- ``akka-scalaz-1.1.jar`` -- Support for the Scalaz library
- ``akka-spring-1.1.jar`` -- Spring framework integration
- ``akka-osgi-dependencies-bundle-1.1.jar`` -- OSGi support
How to see the JARs dependencies of each Akka module is described in the :ref:`dependencies` section. Worth noting
is that ``akka-actor`` has zero external dependencies (apart from the ``scala-library.jar`` JAR).
Using a release distribution
----------------------------
Download the release you need, Akka core or Akka Modules, from `<http://akka.io/downloads>`_ and unzip it.
Microkernel
^^^^^^^^^^^
The Akka Modules distribution includes the microkernel. To run the microkernel:
* Set the AKKA_HOME environment variable to the root of the Akka distribution.
* To start the kernel use the scripts in the ``bin`` directory and deploy all samples applications from ``./deploy`` dir.
More information is available in the documentation of the Microkernel in :ref:`add-on-modules`.
Using a build tool
------------------
Akka can be used with build tools that support Maven repositories. The Akka Maven repository can be found at `<http://akka.io/repository>`_.
Using Akka with Maven
---------------------
Information about how to use Akka with Maven, including how to create an Akka Maven project from scratch,
can be found in the :ref:`getting-started-first-java`.
Summary of the essential parts for using Akka with Maven:
1) Add this repository to your ``pom.xml``:
.. code-block:: xml
<repository>
<id>Akka</id>
<name>Akka Maven2 Repository</name>
<url>http://akka.io/repository/ </url>
</repository>
2) Add the Akka dependencies. For example, here is the dependency for Akka Actor 1.1:
.. code-block:: xml
<dependency>
<groupId>se.scalablesolutions.akka</groupId>
<artifactId>akka-actor</artifactId>
<version>1.1</version>
</dependency>
Using Akka with SBT
-------------------
Information about how to use Akka with SBT, including how to create an Akka SBT project from scratch,
can be found in the :ref:`getting-started-first-scala`.
Summary of the essential parts for using Akka with SBT:
1) Akka has an SBT plugin which makes it very easy to get started with Akka and SBT.
The Scala version in your SBT project needs to match the version that Akka is built against. For Akka 1.1 this is
Scala version 2.9.0.
To use the plugin, first add a plugin definition to your SBT project by creating project/plugins/Plugins.scala with:
.. code-block:: scala
import sbt._
class Plugins(info: ProjectInfo) extends PluginDefinition(info) {
val akkaRepo = "Akka Repo" at "http://akka.io/repository"
val akkaPlugin = "se.scalablesolutions.akka" % "akka-sbt-plugin" % "1.1"
}
*Note: the plugin version matches the Akka version provided. The current release is 1.1.*
2) Then mix the AkkaProject trait into your project definition. For example:
.. code-block:: scala
class MyProject(info: ProjectInfo) extends DefaultProject(info) with AkkaProject
*Note: This adds akka-actor as a dependency by default.*
If you also want to include other Akka modules there is a convenience method: ``akkaModule``. For example, you can add extra Akka modules by adding any of the following lines to your project class:
.. code-block:: scala
val akkaStm = akkaModule("stm")
val akkaTypedActor = akkaModule("typed-actor")
val akkaRemote = akkaModule("remote")
val akkaHttp = akkaModule("http")
val akkaAmqp = akkaModule("amqp")
val akkaCamel = akkaModule("camel")
val akkaCamelTyped = akkaModule("camel-typed")
val akkaSpring = akkaModule("spring")
Using Akka with Eclipse
-----------------------
Information about how to use Akka with Eclipse, including how to create an Akka Eclipse project from scratch,
can be found in the :ref:`getting-started-first-scala-eclipse`.
Using Akka with IntelliJ IDEA
-----------------------------
Setup SBT project and then use `sbt-idea <https://github.com/mpeltonen/sbt-idea>`_ to generate IntelliJ IDEA project.
Build from sources
------------------
Akka uses Git and is hosted at `Github <http://github.com>`_.
* Akka: clone the Akka repository from `<http://github.com/jboner/akka>`_
* Akka Modules: clone the Akka Modules repository from `<http://github.com/jboner/akka-modules>`_
Continue reading the page on :ref:`building-akka`
Need help?
----------
If you have questions you can get help on the `Akka Mailing List <http://groups.google.com/group/akka-user>`_.
You can also ask for `commercial support <http://typesafe.com>`_.
Thanks for being a part of the Akka community.

View file

@ -4,9 +4,12 @@ Introduction
.. toctree:: .. toctree::
:maxdepth: 2 :maxdepth: 2
what-is-akka
why-akka why-akka
getting-started
getting-started-first-scala getting-started-first-scala
getting-started-first-scala-eclipse getting-started-first-scala-eclipse
getting-started-first-java getting-started-first-java
building-akka deployment-scenarios
configuration use-cases

View file

@ -0,0 +1,48 @@
Examples of use-cases for Akka
==============================
There is a great discussion on use-cases for Akka with some good write-ups by production users `here <http://stackoverflow.com/questions/4493001/good-use-case-for-akka/4494512#4494512>`_
Here are some of the areas where Akka is being deployed into production
-----------------------------------------------------------------------
**Transaction processing (Online Gaming, Finance/Banking, Trading, Statistics, Betting, Social Media, Telecom)**
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Scale up, scale out, fault-tolerance / HA
**Service backend (any industry, any app)**
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Service REST, SOAP, Cometd, WebSockets etc
Act as message hub / integration layer
Scale up, scale out, fault-tolerance / HA
**Concurrency/parallelism (any app)**
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Correct
Simple to work with and understand
Just add the jars to your existing JVM project (use Scala, Java, Groovy or JRuby)
**Simulation**
^^^^^^^^^^^^^^
Master/Worker, Compute Grid, MapReduce etc.
**Batch processing (any industry)**
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Camel integration to hook up with batch data sources
Actors divide and conquer the batch workloads
**Communications Hub (Telecom, Web media, Mobile media)**
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Scale up, scale out, fault-tolerance / HA
**Gaming and Betting (MOM, online gaming, betting)**
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Scale up, scale out, fault-tolerance / HA
**Business Intelligence/Data Mining/general purpose crunching**
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Scale up, scale out, fault-tolerance / HA
**Complex Event Stream Processing**
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Scale up, scale out, fault-tolerance / HA

View file

@ -0,0 +1,62 @@
.. _what-is-akka:
###############
What is Akka?
###############
**Simpler Scalability, Fault-Tolerance, Concurrency & Remoting through Actors**
We believe that writing correct concurrent, fault-tolerant and scalable
applications is too hard. Most of the time it's because we are using the wrong
tools and the wrong level of abstraction. Akka is here to change that. Using the
Actor Model together with Software Transactional Memory we raise the abstraction
level and provide a better platform to build correct concurrent and scalable
applications. For fault-tolerance we adopt the Let it crash/Embrace failure
model which have been used with great success in the telecom industry to build
applications that self-heals, systems that never stop. Actors also provides the
abstraction for transparent distribution and the basis for truly scalable and
fault-tolerant applications. Akka is Open Source and available under the Apache
2 License.
Download from http://akka.io/downloads/
Akka implements a unique hybrid
===============================
- :ref:`untyped-actors-java`, which gives you:
- Simple and high-level abstractions for concurrency and parallelism.
- Asynchronous, non-blocking and highly performant event-driven programming model.
- Very lightweight event-driven processes (create ~6.5 million actors on 4GB RAM).
- :ref:`fault-tolerance-java` through supervisor hierarchies with "let-it-crash"
semantics. Excellent for writing highly fault-tolerant systems that never
stop, systems that self-heal.
- :ref:`stm-java` (STM). (Distributed transactions coming soon).
- :ref:`transactors-java`: combine actors and STM into transactional
actors. Allows you to compose atomic message flows with automatic retry and
rollback.
- :ref:`remote-actors-java`: highly performant distributed actors with remote
supervision and error management.
- :ref:`java-api` and :ref:`scala-api`
Akka can be used in two different ways
======================================
- As a library: used by a web app, to be put into WEB-INF/lib or as a regular
JAR on your classpath.
- As a microkernel: stand-alone kernel, embedding a servlet container and all
the other modules.
See the :ref:`deployment-scenarios` for details.

View file

@ -53,16 +53,15 @@ And that's all in the ApacheV2-licensed open source project. On top of that we
have a commercial product called Cloudy Akka which provides the following have a commercial product called Cloudy Akka which provides the following
features: features:
#. Dynamically clustered ActorRegistry with both automatic and manual migration #. Management through Dashboard, JMX and REST
of actors #. Monitoring through Dashboard, JMX and SNMP
#. Dapper-style tracing of messages across components and remote nodes
#. A configurable alert system
#. Real-time statistics
#. Very low overhead monitoring agents (should always be on in production)
#. Consolidation of statistics and logging information to a single node
#. Data analysis through Hadoop
#. Storage of statistics data for later processing
#. Provisioning and rolling upgrades through a dashboard
#. Cluster membership and cluster event subscriptions Read more `here <http://typesafe.com/products/typesafe-subscription>`_.
#. Durable actor mailboxes of different sizes and shapes - file-backed,
Redis-backed, ZooKeeper-backed, Beanstalkd-backed and with AMQP and JMS-based
in the works
#. Monitoring influenced by Dapper for cross-machine message tracing and
JMX-exposed statistics
Read more `here <http://scalablesolutions.se/products.html>`_.

View file

@ -8,12 +8,12 @@ ActorRegistry: Finding Actors
Actors can be looked up using the 'akka.actor.Actors.registry()' object. Through this registry you can look up actors by: Actors can be looked up using the 'akka.actor.Actors.registry()' object. Through this registry you can look up actors by:
* uuid com.eaio.uuid.UUID this uses the uuid field in the Actor class, returns the actor reference for the actor with specified uuid, if one exists, otherwise None * uuid com.eaio.uuid.UUID this uses the ``uuid`` field in the Actor class, returns the actor reference for the actor with specified uuid, if one exists, otherwise None
* id string this uses the id field in the Actor class, which can be set by the user (default is the class name), returns all actor references to actors with specified id * id string this uses the ``id`` field in the Actor class, which can be set by the user (default is the class name), returns all actor references to actors with specified id
* parameterized type - returns a 'ActorRef[]' with all actors that are a subtype of this specific type * parameterized type - returns a ``ActorRef[]`` with all actors that are a subtype of this specific type
* specific actor class - returns a 'ActorRef[]' with all actors of this exact class * specific actor class - returns a ``ActorRef[]`` with all actors of this exact class
Actors are automatically registered in the ActorRegistry when they are started and removed when they are stopped. But you can explicitly register and unregister ActorRef's if you need to using the 'register' and 'unregister' methods. Actors are automatically registered in the ActorRegistry when they are started and removed when they are stopped. But you can explicitly register and unregister ActorRef's if you need to using the ``register`` and ``unregister`` methods.
Here is a summary of the API for finding actors: Here is a summary of the API for finding actors:
@ -31,7 +31,7 @@ You can shut down all Actors in the system by invoking:
registry().shutdownAll(); registry().shutdownAll();
If you want to know when a new Actor is added or to or removed from the registry, you can use the subscription API. You can register an Actor that should be notified when an event happens in the ActorRegistry: If you want to know when a new Actor is added to or removed from the registry, you can use the subscription API on the registry. You can register an Actor that should be notified when an event happens in the ActorRegistry:
.. code-block:: java .. code-block:: java

View file

@ -1,6 +1,10 @@
Dataflow Concurrency (Java) Dataflow Concurrency (Java)
=========================== ===========================
.. sidebar:: Contents
.. contents:: :local:
Introduction Introduction
------------ ------------
@ -13,6 +17,7 @@ Dataflow concurrency is deterministic. This means that it will always behave the
The best way to learn how to program with dataflow variables is to read the fantastic book `Concepts, Techniques, and Models of Computer Programming <http://www.info.ucl.ac.be/%7Epvr/book.html>`_. By Peter Van Roy and Seif Haridi. The best way to learn how to program with dataflow variables is to read the fantastic book `Concepts, Techniques, and Models of Computer Programming <http://www.info.ucl.ac.be/%7Epvr/book.html>`_. By Peter Van Roy and Seif Haridi.
The documentation is not as complete as it should be, something we will improve shortly. For now, besides above listed resources on dataflow concurrency, I recommend you to read the documentation for the GPars implementation, which is heavily influenced by the Akka implementation: The documentation is not as complete as it should be, something we will improve shortly. For now, besides above listed resources on dataflow concurrency, I recommend you to read the documentation for the GPars implementation, which is heavily influenced by the Akka implementation:
* `<http://gpars.codehaus.org/Dataflow>`_ * `<http://gpars.codehaus.org/Dataflow>`_
* `<http://www.gpars.org/guide/guide/7.%20Dataflow%20Concurrency.html>`_ * `<http://www.gpars.org/guide/guide/7.%20Dataflow%20Concurrency.html>`_
@ -138,6 +143,7 @@ Shows how to shutdown dataflow variables and bind threads to values to be able t
Example in Akka: Example in Akka:
.. code-block:: java .. code-block:: java
import static akka.dataflow.DataFlow.*; import static akka.dataflow.DataFlow.*;
import akka.japi.Effect; import akka.japi.Effect;

View file

@ -1,6 +1,10 @@
Dispatchers (Java) Dispatchers (Java)
================== ==================
.. sidebar:: Contents
.. contents:: :local:
Module stability: **SOLID** Module stability: **SOLID**
The Dispatcher is an important piece that allows you to configure the right semantics and parameters for optimal performance, throughput and scalability. Different Actors have different needs. The Dispatcher is an important piece that allows you to configure the right semantics and parameters for optimal performance, throughput and scalability. Different Actors have different needs.
@ -125,10 +129,10 @@ Setting this to a higher number will increase throughput but lower fairness, and
If you don't define a the 'throughput' option in the configuration file then the default value of '5' will be used. If you don't define a the 'throughput' option in the configuration file then the default value of '5' will be used.
Browse the `ScalaDoc <scaladoc>`_ or look at the code for all the options available. Browse the :ref:`scaladoc` or look at the code for all the options available.
Priority event-based Priority event-based
^^^^^^^^^^^ ^^^^^^^^^^^^^^^^^^^^
Sometimes it's useful to be able to specify priority order of messages, that is done by using PriorityExecutorBasedEventDrivenDispatcher and supply Sometimes it's useful to be able to specify priority order of messages, that is done by using PriorityExecutorBasedEventDrivenDispatcher and supply
a java.util.Comparator[MessageInvocation] or use a akka.dispatch.PriorityGenerator (recommended): a java.util.Comparator[MessageInvocation] or use a akka.dispatch.PriorityGenerator (recommended):
@ -137,7 +141,7 @@ Creating a PriorityExecutorBasedEventDrivenDispatcher using PriorityGenerator:
.. code-block:: java .. code-block:: java
package some.package; package some.pkg;
import akka.actor.*; import akka.actor.*;
import akka.dispatch.*; import akka.dispatch.*;
@ -249,13 +253,14 @@ For the 'ExecutorBasedEventDrivenDispatcher' and the 'ExecutorBasedWorkStealingD
For the 'ThreadBasedDispatcher', it is non-shareable between actors, and associates a dedicated Thread with the actor. For the 'ThreadBasedDispatcher', it is non-shareable between actors, and associates a dedicated Thread with the actor.
Making it bounded (by specifying a capacity) is optional, but if you do, you need to provide a pushTimeout (default is 10 seconds). When trying to send a message to the Actor it will throw a MessageQueueAppendFailedException("BlockingMessageTransferQueue transfer timed out") if the message cannot be added to the mailbox within the time specified by the pushTimeout. Making it bounded (by specifying a capacity) is optional, but if you do, you need to provide a pushTimeout (default is 10 seconds). When trying to send a message to the Actor it will throw a MessageQueueAppendFailedException("BlockingMessageTransferQueue transfer timed out") if the message cannot be added to the mailbox within the time specified by the pushTimeout.
`<code format="java">`_ .. code-block:: java
class MyActor extends UntypedActor {
public MyActor() { class MyActor extends UntypedActor {
int mailboxCapacity = 100; public MyActor() {
Duration pushTimeout = new FiniteDuration(10, TimeUnit.SECONDS); int mailboxCapacity = 100;
getContext().setDispatcher(Dispatchers.newThreadBasedDispatcher(getContext(), mailboxCapacity, pushTimeout)); Duration pushTimeout = new FiniteDuration(10, TimeUnit.SECONDS);
getContext().setDispatcher(Dispatchers.newThreadBasedDispatcher(getContext(), mailboxCapacity, pushTimeout));
}
...
} }
...
}
`<code>`_

View file

@ -1,6 +1,12 @@
.. _fault-tolerance-java:
Fault Tolerance Through Supervisor Hierarchies (Java) Fault Tolerance Through Supervisor Hierarchies (Java)
===================================================== =====================================================
.. sidebar:: Contents
.. contents:: :local:
Module stability: **SOLID** Module stability: **SOLID**
The "let it crash" approach to fault/error handling, implemented by linking actors, is very different to what Java and most non-concurrency oriented languages/frameworks have adopted. Its a way of dealing with failure that is designed for concurrent and distributed systems. The "let it crash" approach to fault/error handling, implemented by linking actors, is very different to what Java and most non-concurrency oriented languages/frameworks have adopted. Its a way of dealing with failure that is designed for concurrent and distributed systems.
@ -10,14 +16,14 @@ Concurrency
Throwing an exception in concurrent code (lets assume we are using non-linked actors), will just simply blow up the thread that currently executes the actor. Throwing an exception in concurrent code (lets assume we are using non-linked actors), will just simply blow up the thread that currently executes the actor.
# There is no way to find out that things went wrong (apart from inspecting the stack trace). - There is no way to find out that things went wrong (apart from inspecting the stack trace).
# There is nothing you can do about it. - There is nothing you can do about it.
Here actors provide a clean way of getting notification of the error and do something about it. Here actors provide a clean way of getting notification of the error and do something about it.
Linking actors also allow you to create sets of actors where you can be sure that either: Linking actors also allow you to create sets of actors where you can be sure that either:
# All are dead - All are dead
# None are dead - None are dead
This is very useful when you have thousands of concurrent actors. Some actors might have implicit dependencies and together implement a service, computation, user session etc. This is very useful when you have thousands of concurrent actors. Some actors might have implicit dependencies and together implement a service, computation, user session etc.
@ -56,8 +62,8 @@ Restart callbacks
There are two different callbacks that an UntypedActor or TypedActor can hook in to: There are two different callbacks that an UntypedActor or TypedActor can hook in to:
* Pre restart - Pre restart
* Post restart - Post restart
These are called prior to and after the restart upon failure and can be used to clean up and reset/reinitialize state upon restart. This is important in order to reset the component failure and leave the component in a fresh and stable state before consuming further messages. These are called prior to and after the restart upon failure and can be used to clean up and reset/reinitialize state upon restart. This is important in order to reset the component failure and leave the component in a fresh and stable state before consuming further messages.
@ -66,8 +72,8 @@ Defining a supervisor's restart strategy
Both the Typed Actor supervisor configuration and the Actor supervisor configuration take a FaultHandlingStrategy instance which defines the fault management. The different strategies are: Both the Typed Actor supervisor configuration and the Actor supervisor configuration take a FaultHandlingStrategy instance which defines the fault management. The different strategies are:
* AllForOne - AllForOne
* OneForOne - OneForOne
These have the semantics outlined in the section above. These have the semantics outlined in the section above.
@ -86,8 +92,8 @@ Defining actor life-cycle
The other common configuration element is the LifeCycle which defines the life-cycle. The supervised actor can define one of two different life-cycle configurations: The other common configuration element is the LifeCycle which defines the life-cycle. The supervised actor can define one of two different life-cycle configurations:
* Permanent: which means that the actor will always be restarted. - Permanent: which means that the actor will always be restarted.
* Temporary: which means that the actor will **not** be restarted, but it will be shut down through the regular shutdown process so the 'postStop' callback function will called. - Temporary: which means that the actor will **not** be restarted, but it will be shut down through the regular shutdown process so the 'postStop' callback function will called.
Here is an example of how to define the life-cycle: Here is an example of how to define the life-cycle:
@ -126,7 +132,7 @@ The Actors supervision can be declaratively defined by creating a Supervis
Supervisors created like this are implicitly instantiated and started. Supervisors created like this are implicitly instantiated and started.
To configure a handler function for when the actor underlying the supervisor receives a MaximumNumberOfRestartsWithinTimeRangeReached message, you can specify To configure a handler function for when the actor underlying the supervisor receives a MaximumNumberOfRestartsWithinTimeRangeReached message, you can specify
a Procedure2<ActorRef,MaximumNumberOfRestartsWithinTimeRangeReached> when creating the SupervisorConfig. This handler will be called with the ActorRef of the supervisor and the a Procedure2<ActorRef,MaximumNumberOfRestartsWithinTimeRangeReached> when creating the SupervisorConfig. This handler will be called with the ActorRef of the supervisor and the
MaximumNumberOfRestartsWithinTimeRangeReached message. MaximumNumberOfRestartsWithinTimeRangeReached message.
.. code-block:: java .. code-block:: java
@ -254,10 +260,13 @@ The supervising Actor also needs to define a fault handler that defines the rest
The different options are: The different options are:
* AllForOneStrategy(trapExit, maxNrOfRetries, withinTimeRange) - AllForOneStrategy(trapExit, maxNrOfRetries, withinTimeRange)
* trapExit is an Array of classes inheriting from Throwable, they signal which types of exceptions this actor will handle
* OneForOneStrategy(trapExit, maxNrOfRetries, withinTimeRange) - trapExit is an Array of classes inheriting from Throwable, they signal which types of exceptions this actor will handle
* trapExit is an Array of classes inheriting from Throwable, they signal which types of exceptions this actor will handle
- OneForOneStrategy(trapExit, maxNrOfRetries, withinTimeRange)
- trapExit is an Array of classes inheriting from Throwable, they signal which types of exceptions this actor will handle
Here is an example: Here is an example:
@ -346,8 +355,8 @@ Supervised actors have the option to reply to the initial sender within preResta
} }
} }
* A reply within preRestart or postRestart must be a safe reply via getContext().replySafe() because a getContext().replyUnsafe() will throw an exception when the actor is restarted without having failed. This can be the case in context of AllForOne restart strategies. - A reply within preRestart or postRestart must be a safe reply via getContext().replySafe() because a getContext().replyUnsafe() will throw an exception when the actor is restarted without having failed. This can be the case in context of AllForOne restart strategies.
* A reply within postStop must be a safe reply via getContext().replySafe() because a getContext().replyUnsafe() will throw an exception when the actor has been stopped by the application (and not by a supervisor) after successful execution of receive (or no execution at all). - A reply within postStop must be a safe reply via getContext().replySafe() because a getContext().replyUnsafe() will throw an exception when the actor has been stopped by the application (and not by a supervisor) after successful execution of receive (or no execution at all).
Handling too many actor restarts within a specific time limit Handling too many actor restarts within a specific time limit
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
@ -366,10 +375,10 @@ Now, what happens if this limit is reached?
What will happen is that the failing actor will send a system message to its supervisor called 'MaximumNumberOfRestartsWithinTimeRangeReached' with the following these properties: What will happen is that the failing actor will send a system message to its supervisor called 'MaximumNumberOfRestartsWithinTimeRangeReached' with the following these properties:
* victim: ActorRef - victim: ActorRef
* maxNrOfRetries: int - maxNrOfRetries: int
* withinTimeRange: int - withinTimeRange: int
* lastExceptionCausingRestart: Throwable - lastExceptionCausingRestart: Throwable
If you want to be able to take action upon this event (highly recommended) then you have to create a message handle for it in the supervisor. If you want to be able to take action upon this event (highly recommended) then you have to create a message handle for it in the supervisor.
@ -479,6 +488,7 @@ If the parent TypedActor (supervisor) wants to be able to do handle failing chil
For convenience there is an overloaded link that takes trapExit and faultHandler for the supervisor as arguments. Here is an example: For convenience there is an overloaded link that takes trapExit and faultHandler for the supervisor as arguments. Here is an example:
.. code-block:: java .. code-block:: java
import static akka.actor.TypedActor.*; import static akka.actor.TypedActor.*;
import static akka.config.Supervision.*; import static akka.config.Supervision.*;

269
akka-docs/java/futures.rst Normal file
View file

@ -0,0 +1,269 @@
.. _futures-java:
Futures (Java)
===============
.. sidebar:: Contents
.. contents:: :local:
Introduction
------------
In Akka, a `Future <http://en.wikipedia.org/wiki/Futures_and_promises>`_ is a data structure used to retrieve the result of some concurrent operation. This operation is usually performed by an ``Actor`` or by the ``Dispatcher`` directly. This result can be accessed synchronously (blocking) or asynchronously (non-blocking).
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:
.. code-block:: java
Future[Object] future = actorRef.sendRequestReplyFuture[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`_.
Use Directly
------------
A common use case within Akka is to have some computation performed concurrently without needing the extra utility of an ``UntypedActor``. If you find yourself creating a pool of ``UntypedActor``\s for the sole reason of performing a calculation in parallel, there is an easier (and faster) way:
.. code-block:: java
import akka.dispatch.Future;
import static akka.dispatch.Futures.future;
import java.util.concurrent.Callable;
Future<String> f = future(new Callable<String>() {
public String call() {
return "Hello" + "World!";
}
});
String result = f.get(); //Blocks until timeout, default timeout is set in akka.conf, otherwise 5 seconds
In the above code the block passed to ``future`` will be executed by the default ``Dispatcher``, with the return value of the block used to complete the ``Future`` (in this case, the result would be the string: "HelloWorld"). Unlike a ``Future`` that is returned from an ``UntypedActor``, this ``Future`` is properly typed, and we also avoid the overhead of managing an ``UntypedActor``.
Functional Futures
------------------
A recent addition to Akka's ``Future`` is several monadic methods that are very similar to the ones used by ``Scala``'s collections. These allow you to create 'pipelines' or 'streams' that the result will travel through.
Future is a Monad
^^^^^^^^^^^^^^^^^
The first method for working with ``Future`` functionally is ``map``. This method takes a ``Function`` which performs some operation on the result of the ``Future``, and returning a new result. The return value of the ``map`` method is another ``Future`` that will contain the new result:
.. code-block:: java
import akka.dispatch.Future;
import static akka.dispatch.Futures.future;
import static akka.japi.Function;
import java.util.concurrent.Callable;
Future<String> f1 = future(new Callable<String>() {
public String call() {
return "Hello" + "World";
}
});
Future<Integer> f2 = f1.map(new Function<String, Integer>() {
public Integer apply(String s) {
return s.length();
}
});
Integer result = f2.get();
In this example we are joining two strings together within a Future. Instead of waiting for f1 to complete, we apply our function that calculates the length of the string using the ``map`` method. Now we have a second Future, f2, that will eventually contain an ``Integer``. When our original ``Future``, f1, completes, it will also apply our function and complete the second Future with it's result. When we finally ``get`` the result, it will contain the number 10. Our original Future still contains the string "HelloWorld" and is unaffected by the ``map``.
Something to note when using these methods: if the ``Future`` is still being processed when one of these methods are called, it will be the completing thread that actually does the work. If the ``Future`` is already complete though, it will be run in our current thread. For example:
.. code-block:: java
import akka.dispatch.Future;
import static akka.dispatch.Futures.future;
import static akka.japi.Function;
import java.util.concurrent.Callable;
Future<String> f1 = future(new Callable<String>() {
public String call() {
Thread.sleep(1000);
return "Hello" + "World";
}
});
Future<Integer> f2 = f1.map(new Function<String, Integer>() {
public Integer apply(String s) {
return s.length();
}
});
Integer result = f2.get();
The original ``Future`` will take at least 1 second to execute now, which means it is still being processed at the time we call ``map``. The function we provide gets stored within the ``Future`` and later executed automatically by the dispatcher when the result is ready.
If we do the opposite:
.. code-block:: java
import akka.dispatch.Future;
import static akka.dispatch.Futures.future;
import static akka.japi.Function;
import java.util.concurrent.Callable;
Future<String> f1 = future(new Callable<String>() {
public String call() {
return "Hello" + "World";
}
});
Thread.sleep(1000);
Future<Integer> f2 = f1.map(new Function<String, Integer>() {
public Integer apply(String s) {
return s.length();
}
});
Integer result = f2.get();
Our little string has been processed long before our 1 second sleep has finished. Because of this, the dispatcher has moved onto other messages that need processing and can no longer calculate the length of the string for us, instead it gets calculated in the current thread just as if we weren't using a ``Future``.
Normally this works quite well as it means there is very little overhead to running a quick function. If there is a possibility of the function taking a non-trivial amount of time to process it might be better to have this done concurrently, and for that we use ``flatMap``:
.. code-block:: java
import akka.dispatch.Future;
import static akka.dispatch.Futures.future;
import static akka.japi.Function;
import java.util.concurrent.Callable;
Future<String> f1 = future(new Callable<String>() {
public String call() {
return "Hello" + "World";
}
});
Future<Integer> f2 = f1.flatMap(new Function<String, Future<Integer>>() {
public Future<Integer> apply(final String s) {
return future(
new Callable<Integer>() {
public Integer call() {
return s.length();
}
});
}
});
Integer result = f2.get();
Now our second Future is executed concurrently as well. This technique can also be used to combine the results of several Futures into a single calculation, which will be better explained in the following sections.
Composing Futures
^^^^^^^^^^^^^^^^^
It is very often desirable to be able to combine different Futures with eachother, below are some examples on how that can be done in a non-blocking fashion.
.. code-block:: java
import akka.dispatch.Future;
import static akka.dispatch.Futures.sequence;
import akka.japi.Function;
import java.lang.Iterable;
Iterable<Future<Integer>> listOfFutureInts = ... //Some source generating a sequence of Future<Integer>:s
// now we have a Future[Iterable[Int]]
Future<Iterable<Integer>> futureListOfInts = sequence(listOfFutureInts);
// Find the sum of the odd numbers
Long totalSum = futureListOfInts.map(
new Function<LinkedList<Integer>, Long>() {
public Long apply(LinkedList<Integer> ints) {
long sum = 0;
for(Integer i : ints)
sum += i;
return sum;
}
}).get();
To better explain what happened in the example, ``Future.sequence`` is taking the ``Iterable<Future<Integer>>`` and turning it into a ``Future<Iterable<Integer>>``. We can then use ``map`` to work with the ``Iterable<Integer>`` directly, and we aggregate the sum of the ``Iterable``.
The ``traverse`` method is similar to ``sequence``, but it takes a sequence of ``A``s and applies a function from ``A`` to ``Future<B>`` and returns a ``Future<Iterable<B>>``, enabling parallel ``map`` over the sequence, if you use ``Futures.future`` to create the ``Future``.
.. code-block:: java
import akka.dispatch.Future;
import static akka.dispatch.Futures.traverse;
import static akka.dispatch.Futures.future;
import java.lang.Iterable;
import akka.japi.Function;
Iterable<String> listStrings = ... //Just a sequence of Strings
Future<Iterable<String>> result = traverse(listStrings, new Function<String,Future<String>>(){
public Future<String> apply(final String r) {
return future(new Callable<String>() {
public String call() {
return r.toUpperCase();
}
});
}
});
result.get(); //Returns the sequence of strings as upper case
It's as simple as that!
Then there's a method that's called ``fold`` that takes a start-value, a sequence of ``Future``:s and a function from the type of the start-value, a timeout, and the type of the futures and returns something with the same type as the start-value, and then applies the function to all elements in the sequence of futures, non-blockingly, the execution will run on the Thread of the last completing Future in the sequence.
.. code-block:: java
import akka.dispatch.Future;
import static akka.dispatch.Futures.fold;
import java.lang.Iterable;
import akka.japi.Function2;
Iterable<Future<String>> futures = ... //A sequence of Futures, in this case Strings
Future<String> result = fold("", 15000, futures, new Function2<String, String, String>(){ //Start value is the empty string, timeout is 15 seconds
public String apply(String r, String t) {
return r + t; //Just concatenate
}
});
result.get(); // Will produce a String that says "testtesttesttest"(... and so on).
That's all it takes!
If the sequence passed to ``fold`` is empty, it will return the start-value, in the case above, that will be 0. In some cases you don't have a start-value and you're able to use the value of the first completing Future in the sequence as the start-value, you can use ``reduce``, it works like this:
.. code-block:: java
import akka.dispatch.Future;
import static akka.dispatch.Futures.reduce;
import java.util.Iterable;
import akka.japi.Function2;
Iterable<Future<String>> futures = ... //A sequence of Futures, in this case Strings
Future<String> result = reduce(futures, 15000, new Function2<String, String, String>(){ //Timeout is 15 seconds
public String apply(String r, String t) {
return r + t; //Just concatenate
}
});
result.get(); // Will produce a String that says "testtesttesttest"(... and so on).
Same as with ``fold``, the execution will be done by the Thread that completes the last of the Futures, you can also parallize it by chunking your futures into sub-sequences and reduce them, and then reduce the reduced results again.
This is just a sample of what can be done.
Exceptions
----------
Since the result of a ``Future`` is created concurrently to the rest of the program, exceptions must be handled differently. It doesn't matter if an ``UntypedActor`` or the dispatcher is completing the ``Future``, if an ``Exception`` is caught the ``Future`` will contain it instead of a valid result. If a ``Future`` does contain an ``Exception``, calling ``get`` will cause it to be thrown again so it can be handled properly.

View file

@ -45,6 +45,8 @@ Retrieve the external Guice dependency
-------------------------------------- --------------------------------------
The external dependency can be retrieved like this: The external dependency can be retrieved like this:
`<code format="java">`_
Ext ext = manager.getExternalDependency(Ext.class); .. code-block:: java
`<code>`_
Ext ext = manager.getExternalDependency(Ext.class);

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