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

This commit is contained in:
Jonas Bonér 2012-02-16 09:50:49 +01:00
commit 1b35d59b67
119 changed files with 2330 additions and 958 deletions

View file

@ -4,14 +4,21 @@
package akka.actor
import java.io.File
import com.typesafe.config.Config
import com.typesafe.config.ConfigFactory
import com.typesafe.config.ConfigParseOptions
import akka.util.Timeout
import akka.util.duration._
@deprecated("use ActorSystem instead", "2.0")
object GlobalActorSystem extends ActorSystemImpl("GlobalSystem", OldConfigurationLoader.defaultConfig) {
start()
/**
* Timeout used in `OldFuture.get` and default implicit ask timeout.
* Hard coded since the migration kit is not intended to be used for production anyway.
*/
val AwaitTimeout = Timeout(5 seconds)
}
/**

View file

@ -18,13 +18,13 @@ class OldFuture[T](future: Future[T]) {
@deprecated("use akka.dispatch.Await.result instead", "2.0")
def get: T = try {
Await.result(future, GlobalActorSystem.settings.ActorTimeout.duration)
Await.result(future, GlobalActorSystem.AwaitTimeout.duration)
} catch {
case e: TimeoutException throw new FutureTimeoutException(e.getMessage, e)
}
@deprecated("use akka.dispatch.Await.ready instead", "2.0")
def await: Future[T] = await(GlobalActorSystem.settings.ActorTimeout.duration)
def await: Future[T] = await(GlobalActorSystem.AwaitTimeout.duration)
@deprecated("use akka.dispatch.Await.ready instead", "2.0")
def await(atMost: Duration) = try {

View file

@ -14,7 +14,7 @@ package object migration {
implicit def future2OldFuture[T](future: Future[T]): OldFuture[T] = new OldFuture[T](future)
implicit def askTimeout: Timeout = GlobalActorSystem.settings.ActorTimeout
implicit def askTimeout: Timeout = GlobalActorSystem.AwaitTimeout
implicit def defaultDispatcher: MessageDispatcher = GlobalActorSystem.dispatcher

View file

@ -40,10 +40,10 @@ public class JavaExtension {
static final ExtensionKey<OtherExtension> key = new ExtensionKey<OtherExtension>(OtherExtension.class) {
};
public final ActorSystemImpl system;
public final ExtendedActorSystem system;
public OtherExtension(ActorSystemImpl i) {
system = i;
public OtherExtension(ExtendedActorSystem system) {
this.system = system;
}
}

View file

@ -5,6 +5,7 @@ import akka.actor.ActorSystem;
import akka.japi.*;
import akka.util.Duration;
import akka.testkit.TestKitExtension;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@ -28,7 +29,7 @@ public class JavaFutureTests {
@BeforeClass
public static void beforeAll() {
system = ActorSystem.create("JavaFutureTests", AkkaSpec.testConf());
t = system.settings().ActorTimeout();
t = TestKitExtension.get(system).DefaultTimeout();
}
@AfterClass
@ -61,10 +62,10 @@ public class JavaFutureTests {
Promise<String> cf = Futures.promise(system.dispatcher());
Future<String> f = cf;
f.onSuccess(new OnSuccess<String>() {
public void onSuccess(String result) {
if (result.equals("foo"))
latch.countDown();
}
public void onSuccess(String result) {
if (result.equals("foo"))
latch.countDown();
}
});
cf.success("foo");
@ -78,10 +79,10 @@ public class JavaFutureTests {
Promise<String> cf = Futures.promise(system.dispatcher());
Future<String> f = cf;
f.onFailure(new OnFailure() {
public void onFailure(Throwable t) {
if (t instanceof NullPointerException)
latch.countDown();
}
public void onFailure(Throwable t) {
if (t instanceof NullPointerException)
latch.countDown();
}
});
Throwable exception = new NullPointerException();
@ -296,8 +297,10 @@ public class JavaFutureTests {
Promise<Object> p = Futures.promise(system.dispatcher());
Future<Object> f = p.future().recover(new Recover<Object>() {
public Object recover(Throwable t) throws Throwable {
if (t == fail) return "foo";
else throw t;
if (t == fail)
return "foo";
else
throw t;
}
});
Duration d = Duration.create(1, TimeUnit.SECONDS);
@ -311,8 +314,10 @@ public class JavaFutureTests {
Promise<Object> p = Futures.promise(system.dispatcher());
Future<Object> f = p.future().recoverWith(new Recover<Future<Object>>() {
public Future<Object> recover(Throwable t) throws Throwable {
if (t == fail) return Futures.<Object>successful("foo", system.dispatcher()).future();
else throw t;
if (t == fail)
return Futures.<Object> successful("foo", system.dispatcher()).future();
else
throw t;
}
});
Duration d = Duration.create(1, TimeUnit.SECONDS);

View file

@ -305,6 +305,7 @@ class ActorLookupSpec extends AkkaSpec with DefaultTimeout {
intercept[MalformedURLException] { ActorPath.fromString("://hallo") }
intercept[MalformedURLException] { ActorPath.fromString("s://dd@:12") }
intercept[MalformedURLException] { ActorPath.fromString("s://dd@h:hd") }
intercept[MalformedURLException] { ActorPath.fromString("a://l:1/b") }
}
}

View file

@ -11,11 +11,10 @@ import akka.testkit._
import akka.util.Timeout
import akka.util.duration._
import java.lang.IllegalStateException
import akka.util.ReflectiveAccess
import akka.serialization.Serialization
import java.util.concurrent.{ CountDownLatch, TimeUnit }
import akka.dispatch.{ Await, DefaultPromise, Promise, Future }
import akka.pattern.ask
import akka.serialization.JavaSerializer
object ActorRefSpec {
@ -240,6 +239,7 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout {
"be serializable using Java Serialization on local node" in {
val a = system.actorOf(Props[InnerActor])
val esys = system.asInstanceOf[ExtendedActorSystem]
import java.io._
@ -251,14 +251,21 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout {
out.flush
out.close
Serialization.currentSystem.withValue(system.asInstanceOf[ActorSystemImpl]) {
val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray))
val bytes = baos.toByteArray
JavaSerializer.currentSystem.withValue(esys) {
val in = new ObjectInputStream(new ByteArrayInputStream(bytes))
val readA = in.readObject
a.isInstanceOf[LocalActorRef] must be === true
readA.isInstanceOf[LocalActorRef] must be === true
(readA eq a) must be === true
}
val ser = new JavaSerializer(esys)
val readA = ser.fromBinary(bytes, None)
readA.isInstanceOf[LocalActorRef] must be === true
(readA eq a) must be === true
}
"throw an exception on deserialize if no system in scope" in {
@ -297,7 +304,7 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout {
out.flush
out.close
Serialization.currentSystem.withValue(sysImpl) {
JavaSerializer.currentSystem.withValue(sysImpl) {
val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray))
in.readObject must be === new EmptyLocalActorRef(sysImpl.provider, system.actorFor("/").path / "non-existing", system.eventStream)
}

View file

@ -8,7 +8,6 @@ import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach }
import akka.util.Duration
import akka.util.Timeout
import akka.util.duration._
import akka.serialization.Serialization
import java.util.concurrent.atomic.AtomicReference
import annotation.tailrec
import akka.testkit.{ EventFilter, filterEvents, AkkaSpec }
@ -19,6 +18,7 @@ import akka.japi.{ Creator, Option ⇒ JOption }
import akka.testkit.DefaultTimeout
import akka.dispatch.{ Await, Dispatchers, Future, Promise }
import akka.pattern.ask
import akka.serialization.JavaSerializer
object TypedActorSpec {
@ -113,7 +113,7 @@ object TypedActorSpec {
}
def futureComposePigdogFrom(foo: Foo): Future[String] = {
implicit val timeout = TypedActor.context.system.settings.ActorTimeout
implicit val timeout = TypedActor(TypedActor.context.system).DefaultReturnTimeout
foo.futurePigdog(500).map(_.toUpperCase)
}
@ -367,7 +367,7 @@ class TypedActorSpec extends AkkaSpec(TypedActorSpec.config)
"be able to serialize and deserialize invocations" in {
import java.io._
Serialization.currentSystem.withValue(system.asInstanceOf[ActorSystemImpl]) {
JavaSerializer.currentSystem.withValue(system.asInstanceOf[ExtendedActorSystem]) {
val m = TypedActor.MethodCall(classOf[Foo].getDeclaredMethod("pigdog"), Array[AnyRef]())
val baos = new ByteArrayOutputStream(8192 * 4)
val out = new ObjectOutputStream(baos)
@ -386,7 +386,7 @@ class TypedActorSpec extends AkkaSpec(TypedActorSpec.config)
"be able to serialize and deserialize invocations' parameters" in {
import java.io._
val someFoo: Foo = new Bar
Serialization.currentSystem.withValue(system.asInstanceOf[ActorSystemImpl]) {
JavaSerializer.currentSystem.withValue(system.asInstanceOf[ExtendedActorSystem]) {
val m = TypedActor.MethodCall(classOf[Foo].getDeclaredMethod("testMethodCallSerialization", Array[Class[_]](classOf[Foo], classOf[String], classOf[Int]): _*), Array[AnyRef](someFoo, null, 1.asInstanceOf[AnyRef]))
val baos = new ByteArrayOutputStream(8192 * 4)
val out = new ObjectOutputStream(baos)

View file

@ -50,6 +50,8 @@ object ActorModelSpec {
case object Restart extends ActorModelMessage
case object DoubleStop extends ActorModelMessage
case class ThrowException(e: Throwable) extends ActorModelMessage
val Ping = "Ping"
@ -86,6 +88,7 @@ object ActorModelSpec {
case Restart ack; busy.switchOff(); throw new Exception("Restart requested")
case Interrupt ack; sender ! Status.Failure(new ActorInterruptedException(new InterruptedException("Ping!"))); busy.switchOff(); throw new InterruptedException("Ping!")
case ThrowException(e: Throwable) ack; busy.switchOff(); throw e
case DoubleStop ack; context.stop(self); context.stop(self); busy.switchOff
}
}
@ -107,8 +110,9 @@ object ActorModelSpec {
val stops = new AtomicLong(0)
def getStats(actorRef: ActorRef) = {
stats.putIfAbsent(actorRef, new InterceptorStats) match {
case null stats.get(actorRef)
val is = new InterceptorStats
stats.putIfAbsent(actorRef, is) match {
case null is
case other other
}
}
@ -124,12 +128,12 @@ object ActorModelSpec {
}
protected[akka] abstract override def register(actor: ActorCell) {
getStats(actor.self).registers.incrementAndGet()
assert(getStats(actor.self).registers.incrementAndGet() == 1)
super.register(actor)
}
protected[akka] abstract override def unregister(actor: ActorCell) {
getStats(actor.self).unregisters.incrementAndGet()
assert(getStats(actor.self).unregisters.incrementAndGet() == 1)
super.unregister(actor)
}
@ -190,13 +194,13 @@ object ActorModelSpec {
}
def assertRef(actorRef: ActorRef, dispatcher: MessageDispatcher = null)(
suspensions: Long = statsFor(actorRef).suspensions.get(),
resumes: Long = statsFor(actorRef).resumes.get(),
registers: Long = statsFor(actorRef).registers.get(),
unregisters: Long = statsFor(actorRef).unregisters.get(),
msgsReceived: Long = statsFor(actorRef).msgsReceived.get(),
msgsProcessed: Long = statsFor(actorRef).msgsProcessed.get(),
restarts: Long = statsFor(actorRef).restarts.get())(implicit system: ActorSystem) {
suspensions: Long = statsFor(actorRef, dispatcher).suspensions.get(),
resumes: Long = statsFor(actorRef, dispatcher).resumes.get(),
registers: Long = statsFor(actorRef, dispatcher).registers.get(),
unregisters: Long = statsFor(actorRef, dispatcher).unregisters.get(),
msgsReceived: Long = statsFor(actorRef, dispatcher).msgsReceived.get(),
msgsProcessed: Long = statsFor(actorRef, dispatcher).msgsProcessed.get(),
restarts: Long = statsFor(actorRef, dispatcher).restarts.get())(implicit system: ActorSystem) {
val stats = statsFor(actorRef, Option(dispatcher).getOrElse(actorRef.asInstanceOf[LocalActorRef].underlying.dispatcher))
val deadline = System.currentTimeMillis + 1000
try {
@ -362,16 +366,19 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa
case e
dispatcher match {
case dispatcher: BalancingDispatcher
val buddies = dispatcher.buddies
val team = dispatcher.team
val mq = dispatcher.messageQueue
System.err.println("Buddies left: ")
buddies.toArray foreach {
System.err.println("Teammates left: " + team.size + " stopLatch: " + stopLatch.getCount + " inhab:" + dispatcher.inhabitants)
team.toArray sorted new Ordering[AnyRef] {
def compare(l: AnyRef, r: AnyRef) = (l, r) match { case (ll: ActorCell, rr: ActorCell) ll.self.path compareTo rr.self.path }
} foreach {
case cell: ActorCell
System.err.println(" - " + cell.self.path + " " + cell.isTerminated + " " + cell.mailbox.status + " " + cell.mailbox.numberOfMessages + " " + SystemMessage.size(cell.mailbox.systemDrain()))
}
System.err.println("Mailbox: " + mq.numberOfMessages + " " + mq.hasMessages + " ")
System.err.println("Mailbox: " + mq.numberOfMessages + " " + mq.hasMessages)
Iterator.continually(mq.dequeue) takeWhile (_ ne null) foreach System.err.println
case _
}
@ -426,6 +433,14 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa
assert(f5.value.isEmpty)
}
}
"not double-deregister" in {
implicit val dispatcher = interceptedDispatcher()
val a = newTestActor(dispatcher.id)
a ! DoubleStop
awaitCond(statsFor(a, dispatcher).registers.get == 1)
awaitCond(statsFor(a, dispatcher).unregisters.get == 1)
}
}
}
@ -529,7 +544,8 @@ object BalancingDispatcherModelSpec {
Duration(config.getNanoseconds("throughput-deadline-time"), TimeUnit.NANOSECONDS),
mailboxType,
configureExecutor(),
Duration(config.getMilliseconds("shutdown-timeout"), TimeUnit.MILLISECONDS)) with MessageDispatcherInterceptor
Duration(config.getMilliseconds("shutdown-timeout"), TimeUnit.MILLISECONDS),
config.getBoolean("attempt-teamwork")) with MessageDispatcherInterceptor
override def dispatcher(): MessageDispatcher = instance
}

View file

@ -53,6 +53,7 @@ class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference) {
c.getMilliseconds("shutdown-timeout") must equal(1 * 1000)
c.getInt("throughput") must equal(5)
c.getMilliseconds("throughput-deadline-time") must equal(0)
c.getBoolean("attempt-teamwork") must equal(true)
}
//Fork join executor config

View file

@ -860,7 +860,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa
val l1, l2 = new TestLatch
val complex = Future() map { _
Future.blocking(system.dispatcher)
Future.blocking()
val nested = Future(())
nested foreach (_ l1.open())
Await.ready(l1, TestLatch.DefaultTimeout) // make sure nested is completed

View file

@ -5,8 +5,9 @@ package akka.pattern
import akka.testkit.AkkaSpec
import akka.util.duration._
import akka.testkit.DefaultTimeout
class AskSpec extends AkkaSpec {
class AskSpec extends AkkaSpec with DefaultTimeout {
"The “ask” pattern" must {
@ -22,7 +23,6 @@ class AskSpec extends AkkaSpec {
"return broken promises on EmptyLocalActorRefs" in {
val empty = system.actorFor("unknown")
implicit val timeout = system.settings.ActorTimeout
val f = empty ? 3.14
f.isCompleted must be(true)
f.value.get match {

View file

@ -11,6 +11,7 @@ import akka.util.duration._
import akka.actor.ActorRef
import java.util.concurrent.atomic.AtomicInteger
import akka.pattern.ask
import akka.util.Duration
object ResizerSpec {
@ -24,6 +25,9 @@ object ResizerSpec {
}
}
}
bal-disp {
type = BalancingDispatcher
}
"""
class TestActor extends Actor {
@ -133,7 +137,7 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with
pressureThreshold = 0,
messagesPerResize = 1)
val router = system.actorOf(Props[BusyActor].withRouter(RoundRobinRouter(resizer = Some(resizer))))
val router = system.actorOf(Props[BusyActor].withRouter(RoundRobinRouter(resizer = Some(resizer))).withDispatcher("bal-disp"))
val latch1 = new TestLatch(1)
router ! (latch1, busy)
@ -157,53 +161,48 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with
// as influenced by the backlog of blocking pooled actors
val resizer = DefaultResizer(
lowerBound = 2,
upperBound = 4,
lowerBound = 3,
upperBound = 5,
rampupRate = 0.1,
backoffRate = 0.0,
pressureThreshold = 1,
messagesPerResize = 1,
backoffThreshold = 0.0)
val router = system.actorOf(Props(new Actor {
def receive = {
case (n: Int, latch: TestLatch, count: AtomicInteger)
(n millis).dilated.sleep
count.incrementAndGet
latch.countDown()
case d: Duration d.dilated.sleep; sender ! "done"
case "echo" sender ! "reply"
}
}).withRouter(RoundRobinRouter(resizer = Some(resizer))))
// first message should create the minimum number of routees
router ! 1
router ! "echo"
expectMsg("reply")
Await.result(router ? CurrentRoutees, 5 seconds).asInstanceOf[RouterRoutees].routees.size must be(2)
def routees(r: ActorRef): Int = {
r ! CurrentRoutees
expectMsgType[RouterRoutees].routees.size
}
def loop(loops: Int, t: Int, latch: TestLatch, count: AtomicInteger) = {
(10 millis).dilated.sleep
for (m 0 until loops) {
router.!((t, latch, count))
(10 millis).dilated.sleep
}
routees(router) must be(3)
def loop(loops: Int, d: Duration) = {
for (m 0 until loops) router ! d
for (m 0 until loops) expectMsg(d * 3, "done")
}
// 2 more should go thru without triggering more
val count1 = new AtomicInteger
val latch1 = TestLatch(2)
loop(2, 200, latch1, count1)
Await.ready(latch1, TestLatch.DefaultTimeout)
count1.get must be(2)
loop(2, 200 millis)
Await.result(router ? CurrentRoutees, 5 seconds).asInstanceOf[RouterRoutees].routees.size must be(2)
routees(router) must be(3)
// a whole bunch should max it out
val count2 = new AtomicInteger
val latch2 = TestLatch(10)
loop(10, 200, latch2, count2)
Await.ready(latch2, TestLatch.DefaultTimeout)
count2.get must be(10)
Await.result(router ? CurrentRoutees, 5 seconds).asInstanceOf[RouterRoutees].routees.size must be(4)
loop(10, 500 millis)
awaitCond(routees(router) > 3)
loop(10, 500 millis)
awaitCond(routees(router) == 5)
}
"backoff" in {
@ -236,9 +235,9 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with
(300 millis).dilated.sleep
// let it cool down
for (m 0 to 3) {
for (m 0 to 5) {
router ! 1
(200 millis).dilated.sleep
(500 millis).dilated.sleep
}
Await.result(router ? CurrentRoutees, 5 seconds).asInstanceOf[RouterRoutees].routees.size must be < (z)

View file

@ -15,6 +15,7 @@ import com.typesafe.config.ConfigFactory
import akka.pattern.ask
import java.util.concurrent.ConcurrentHashMap
import com.typesafe.config.Config
import akka.dispatch.Dispatchers
object RoutingSpec {
@ -51,6 +52,7 @@ object RoutingSpec {
case (sender, message) Nil
}
}
def routerDispatcher: String = Dispatchers.DefaultDispatcherId
}
}
@ -539,6 +541,8 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
//#crRouter
case class VoteCountRouter() extends RouterConfig {
def routerDispatcher: String = Dispatchers.DefaultDispatcherId
//#crRoute
def createRoute(routeeProps: Props, routeeProvider: RouteeProvider): Route = {
val democratActor = routeeProvider.context.actorOf(Props(new DemocratActor()), "d")

View file

@ -89,7 +89,7 @@ class SerializeSpec extends AkkaSpec(SerializeSpec.config) {
case Left(exception) fail(exception)
case Right(bytes) bytes
}
deserialize(b.asInstanceOf[Array[Byte]], classOf[Address], None) match {
deserialize(b.asInstanceOf[Array[Byte]], classOf[Address]) match {
case Left(exception) fail(exception)
case Right(add) assert(add === addr)
}
@ -101,7 +101,7 @@ class SerializeSpec extends AkkaSpec(SerializeSpec.config) {
case Left(exception) fail(exception)
case Right(bytes) bytes
}
deserialize(b.asInstanceOf[Array[Byte]], classOf[Person], None) match {
deserialize(b.asInstanceOf[Array[Byte]], classOf[Person]) match {
case Left(exception) fail(exception)
case Right(p) assert(p === person)
}
@ -114,7 +114,7 @@ class SerializeSpec extends AkkaSpec(SerializeSpec.config) {
case Left(exception) fail(exception)
case Right(bytes) bytes
}
deserialize(b.asInstanceOf[Array[Byte]], classOf[Record], None) match {
deserialize(b.asInstanceOf[Array[Byte]], classOf[Record]) match {
case Left(exception) fail(exception)
case Right(p) assert(p === r)
}
@ -146,7 +146,7 @@ class SerializeSpec extends AkkaSpec(SerializeSpec.config) {
out.close()
val in = new ObjectInputStream(new ByteArrayInputStream(outbuf.toByteArray))
Serialization.currentSystem.withValue(a.asInstanceOf[ActorSystemImpl]) {
JavaSerializer.currentSystem.withValue(a.asInstanceOf[ActorSystemImpl]) {
val deadLetters = in.readObject().asInstanceOf[DeadLetterActorRef]
(deadLetters eq a.deadLetters) must be(true)
}
@ -285,8 +285,5 @@ class TestSerializer extends Serializer {
Array.empty[Byte]
}
def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]] = None,
classLoader: Option[ClassLoader] = None): AnyRef = {
null
}
def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = null
}

View file

@ -378,10 +378,11 @@ public interface Config extends ConfigMergeable {
Object getAnyRef(String path);
/**
* Gets the value at the given path, unless the value is a null value or
* missing, in which case it throws just like the other getters. Use
* {@code get()} from the {@link java.util.Map Map} interface if you want an
* unprocessed value.
* Gets the value at the given path, unless the value is a
* null value or missing, in which case it throws just like
* the other getters. Use {@code get()} on the {@link
* Config#root()} object (or other object in the tree) if you
* want an unprocessed value.
*
* @param path
* path expression

View file

@ -161,8 +161,8 @@ public final class ConfigFactory {
* used rather than the default
* <code>application.{conf,json,properties}</code> classpath resources.
* These system properties should not be set in code (after all, you can
* just parse whatever you want manually and then use {@link #load(Config)
* if you don't want to use <code>application.conf</code>}). The properties
* just parse whatever you want manually and then use {@link #load(Config)}
* if you don't want to use <code>application.conf</code>). The properties
* are intended for use by the person or script launching the application.
* For example someone might have a <code>production.conf</code> that
* include <code>application.conf</code> but then change a couple of values.

View file

@ -23,6 +23,9 @@ import com.typesafe.config.ConfigValueType;
abstract class AbstractConfigObject extends AbstractConfigValue implements
ConfigObject {
private static final long serialVersionUID = 1L;
final private SimpleConfig config;
protected AbstractConfigObject(ConfigOrigin origin) {

View file

@ -20,6 +20,8 @@ import com.typesafe.config.ConfigValue;
*/
abstract class AbstractConfigValue implements ConfigValue, MergeableValue, Serializable {
private static final long serialVersionUID = 1L;
final private SimpleConfigOrigin origin;
AbstractConfigValue(ConfigOrigin origin) {

View file

@ -8,6 +8,8 @@ import com.typesafe.config.ConfigValueType;
final class ConfigBoolean extends AbstractConfigValue {
private static final long serialVersionUID = 1L;
final private boolean value;
ConfigBoolean(ConfigOrigin origin, boolean value) {

View file

@ -8,6 +8,8 @@ import com.typesafe.config.ConfigValueType;
final class ConfigDouble extends ConfigNumber {
private static final long serialVersionUID = 1L;
final private double value;
ConfigDouble(ConfigOrigin origin, double value, String originalText) {

View file

@ -8,6 +8,8 @@ import com.typesafe.config.ConfigValueType;
final class ConfigInt extends ConfigNumber {
private static final long serialVersionUID = 1L;
final private int value;
ConfigInt(ConfigOrigin origin, int value, String originalText) {

View file

@ -8,6 +8,8 @@ import com.typesafe.config.ConfigValueType;
final class ConfigLong extends ConfigNumber {
private static final long serialVersionUID = 1L;
final private long value;
ConfigLong(ConfigOrigin origin, long value, String originalText) {

View file

@ -16,6 +16,8 @@ import com.typesafe.config.ConfigValueType;
*/
final class ConfigNull extends AbstractConfigValue {
private static final long serialVersionUID = 1L;
ConfigNull(ConfigOrigin origin) {
super(origin);
}

View file

@ -7,6 +7,9 @@ import com.typesafe.config.ConfigException;
import com.typesafe.config.ConfigOrigin;
abstract class ConfigNumber extends AbstractConfigValue {
private static final long serialVersionUID = 1L;
// This is so when we concatenate a number into a string (say it appears in
// a sentence) we always have it exactly as the person typed it into the
// config file. It's purely cosmetic; equals/hashCode don't consider this

View file

@ -8,6 +8,8 @@ import com.typesafe.config.ConfigValueType;
final class ConfigString extends AbstractConfigValue {
private static final long serialVersionUID = 1L;
final private String value;
ConfigString(ConfigOrigin origin, String value) {

View file

@ -11,6 +11,8 @@ import com.typesafe.config.ConfigException;
final class Path implements Serializable {
private static final long serialVersionUID = 1L;
final private String first;
final private Path remainder;

View file

@ -31,6 +31,8 @@ import com.typesafe.config.ConfigValueType;
*/
final class SimpleConfig implements Config, MergeableValue, Serializable {
private static final long serialVersionUID = 1L;
final private AbstractConfigObject object;
SimpleConfig(AbstractConfigObject object) {

View file

@ -17,6 +17,8 @@ import com.typesafe.config.ConfigValue;
final class SimpleConfigObject extends AbstractConfigObject {
private static final long serialVersionUID = 1L;
// this map should never be modified - assume immutable
final private Map<String, AbstractConfigValue> value;
final private boolean resolved;

View file

@ -19,6 +19,9 @@ import com.typesafe.config.ConfigOrigin;
// it would be cleaner to have a class hierarchy for various origin types,
// but was hoping this would be enough simpler to be a little messy. eh.
final class SimpleConfigOrigin implements ConfigOrigin, Serializable {
private static final long serialVersionUID = 1L;
final private String description;
final private int lineNumber;
final private int endLineNumber;

View file

@ -4,6 +4,8 @@ import java.io.Serializable;
final class SubstitutionExpression implements Serializable {
private static final long serialVersionUID = 1L;
final private Path path;
final private boolean optional;

View file

@ -1,6 +1,6 @@
##############################
# Akka Reference Config File #
##############################
####################################
# Akka Actor Reference Config File #
####################################
# This the reference config file has all the default settings.
# Make your edits/overrides in your application.conf.
@ -50,12 +50,6 @@ akka {
# removed from their parents
reaper-interval = 5s
# Default timeout for Future based invocations
# - Actor: ask && ?
# - UntypedActor: ask
# - TypedActor: methods with non-void return type
timeout = 5s
# Serializes and deserializes (non-primitive) messages to ensure immutability,
# this is only intended for testing.
serialize-messages = off
@ -64,6 +58,11 @@ akka {
# this is only intended for testing.
serialize-creators = off
typed {
# Default timeout for typed actor methods with non-void return type
timeout = 5s
}
deployment {
# deployment id pattern - on the format: /parent/child etc.
@ -246,6 +245,11 @@ akka {
# mailbox is used. The Class of the FQCN must have a constructor with a
# com.typesafe.config.Config parameter.
mailbox-type = ""
# For BalancingDispatcher: If the balancing dispatcher should attempt to
# schedule idle actors using the same dispatcher when a message comes in,
# and the dispatchers ExecutorService is not fully busy already.
attempt-teamwork = on
}
debug {

View file

@ -5,10 +5,8 @@
package akka
import akka.actor.newUuid
import java.net.{ InetAddress, UnknownHostException }
object AkkaException {
val hostname = try InetAddress.getLocalHost.getHostAddress catch { case e: UnknownHostException "unknown host" }
def toStringWithStackTrace(throwable: Throwable): String = throwable match {
case null "Unknown Throwable: was 'null'"
@ -36,7 +34,7 @@ object AkkaException {
*/
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
class AkkaException(message: String = "", cause: Throwable = null) extends RuntimeException(message, cause) with Serializable {
val uuid = "%s_%s".format(AkkaException.hostname, newUuid)
lazy val uuid = newUuid.toString
override lazy val toString =
"%s:%s\n[%s]".format(getClass.getName, message, uuid)

View file

@ -220,7 +220,7 @@ private[akka] class ActorCell(
val ser = SerializationExtension(system)
ser.serialize(props.creator) match {
case Left(t) throw t
case Right(bytes) ser.deserialize(bytes, props.creator.getClass, None) match {
case Right(bytes) ser.deserialize(bytes, props.creator.getClass) match {
case Left(t) throw t
case _ //All good
}
@ -285,14 +285,18 @@ private[akka] class ActorCell(
final def isTerminated: Boolean = mailbox.isClosed
final def start(): Unit = {
/*
* Create the mailbox and enqueue the Create() message to ensure that
* this is processed before anything else.
*/
mailbox = dispatcher.createMailbox(this)
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
mailbox.systemEnqueue(self, Create())
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
parent.sendSystemMessage(akka.dispatch.Supervise(self))
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
dispatcher.systemDispatch(this, Create())
// This call is expected to start off the actor by scheduling its mailbox.
dispatcher.attach(this)
}
@ -384,8 +388,6 @@ private[akka] class ActorCell(
failedActor.preRestart(cause, if (c ne null) Some(c.message) else None)
} finally {
clearActorFields()
currentMessage = null
actor = null
}
}
actor = freshActor // assign it here so if preStart fails, we can null out the sef-refs next call
@ -493,7 +495,7 @@ private[akka] class ActorCell(
dispatcher.reportFailure(new LogEventException(Error(e, self.path.toString, clazz(actor), e.getMessage), e))
// prevent any further messages to be processed until the actor has been restarted
dispatcher.suspend(this)
actor.supervisorStrategy.handleSupervisorFailing(self, children)
if (actor ne null) actor.supervisorStrategy.handleSupervisorFailing(self, children)
parent.tell(Failed(e), self)
} finally {
checkReceiveTimeout // Reschedule receive timeout
@ -555,9 +557,8 @@ private[akka] class ActorCell(
if (system.settings.DebugLifecycle)
system.eventStream.publish(Debug(self.path.toString, clazz(actor), "stopped")) // FIXME: can actor be null?
} finally {
currentMessage = null
clearActorFields()
if (a ne null) a.clearBehaviorStack()
clearActorFields()
}
}
}
@ -597,7 +598,11 @@ private[akka] class ActorCell(
}
}
final def clearActorFields(): Unit = setActorFields(context = null, self = system.deadLetters)
final def clearActorFields(): Unit = {
setActorFields(context = null, self = system.deadLetters)
currentMessage = null
actor = null
}
final def setActorFields(context: ActorContext, self: ActorRef) {
@tailrec
@ -625,8 +630,5 @@ private[akka] class ActorCell(
}
}
private def clazz(o: AnyRef): Class[_] = {
if (o eq null) this.getClass
else o.getClass
}
private final def clazz(o: AnyRef): Class[_] = if (o eq null) this.getClass else o.getClass
}

View file

@ -8,7 +8,7 @@ import akka.dispatch._
import akka.util._
import scala.collection.immutable.Stack
import java.lang.{ UnsupportedOperationException, IllegalStateException }
import akka.serialization.Serialization
import akka.serialization.{ Serialization, JavaSerializer }
import akka.event.EventStream
import scala.annotation.tailrec
import java.util.concurrent.{ ConcurrentHashMap }
@ -335,7 +335,7 @@ private[akka] class LocalActorRef private[akka] (
*/
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
case class SerializedActorRef private (path: String) {
import akka.serialization.Serialization.currentSystem
import akka.serialization.JavaSerializer.currentSystem
@throws(classOf[java.io.ObjectStreamException])
def readResolve(): AnyRef = currentSystem.value match {
@ -401,7 +401,7 @@ private[akka] object DeadLetterActorRef {
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
class SerializedDeadLetterActorRef extends Serializable { //TODO implement as Protobuf for performance?
@throws(classOf[java.io.ObjectStreamException])
private def readResolve(): AnyRef = Serialization.currentSystem.value.deadLetters
private def readResolve(): AnyRef = JavaSerializer.currentSystem.value.deadLetters
}
val serialized = new SerializedDeadLetterActorRef

View file

@ -318,12 +318,12 @@ class LocalActorRefProvider(
settings: ActorSystem.Settings,
eventStream: EventStream,
scheduler: Scheduler,
classloader: ClassLoader) =
dynamicAccess: DynamicAccess) =
this(_systemName,
settings,
eventStream,
scheduler,
new Deployer(settings, classloader))
new Deployer(settings, dynamicAccess))
val rootPath: ActorPath = RootActorPath(Address("akka", _systemName))

View file

@ -70,7 +70,6 @@ object ActorSystem {
final val CreationTimeout = Timeout(Duration(getMilliseconds("akka.actor.creation-timeout"), MILLISECONDS))
final val ReaperInterval = Duration(getMilliseconds("akka.actor.reaper-interval"), MILLISECONDS)
final val ActorTimeout = Timeout(Duration(getMilliseconds("akka.actor.timeout"), MILLISECONDS))
final val SerializeAllMessages = getBoolean("akka.actor.serialize-messages")
final val SerializeAllCreators = getBoolean("akka.actor.serialize-creators")
@ -325,16 +324,16 @@ abstract class ExtendedActorSystem extends ActorSystem {
def deathWatch: DeathWatch
/**
* ClassLoader which is used for reflective accesses internally. This is set
* to the context class loader, if one is set, or the class loader which
* ClassLoader wrapper which is used for reflective accesses internally. This is set
* to use the context class loader, if one is set, or the class loader which
* loaded the ActorSystem implementation. The context class loader is also
* set on all threads created by the ActorSystem, if one was set during
* creation.
*/
def internalClassLoader: ClassLoader
def dynamicAccess: DynamicAccess
}
class ActorSystemImpl(val name: String, applicationConfig: Config) extends ExtendedActorSystem {
class ActorSystemImpl protected[akka] (val name: String, applicationConfig: Config) extends ExtendedActorSystem {
if (!name.matches("""^\w+$"""))
throw new IllegalArgumentException("invalid ActorSystem name [" + name + "], must contain only word characters (i.e. [a-zA-Z_0-9])")
@ -358,6 +357,35 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten
final val threadFactory: MonitorableThreadFactory =
MonitorableThreadFactory(name, settings.Daemonicity, Option(Thread.currentThread.getContextClassLoader), uncaughtExceptionHandler)
/**
* This is an extension point: by overriding this method, subclasses can
* control all reflection activities of an actor system.
*/
protected def createDynamicAccess(): DynamicAccess = new ReflectiveDynamicAccess(findClassLoader)
protected def findClassLoader: ClassLoader = {
def findCaller(get: Int Class[_]): ClassLoader = {
val frames = Iterator.from(2).map(get)
frames dropWhile { c
c != null &&
(c.getName.startsWith("akka.actor.ActorSystem") ||
c.getName.startsWith("scala.Option") ||
c.getName.startsWith("scala.collection.Iterator") ||
c.getName.startsWith("akka.util.Reflect"))
} next () match {
case null getClass.getClassLoader
case c c.getClassLoader
}
}
Option(Thread.currentThread.getContextClassLoader) orElse
(Reflect.getCallerClass map findCaller) getOrElse
getClass.getClassLoader
}
private val _pm: DynamicAccess = createDynamicAccess()
def dynamicAccess: DynamicAccess = _pm
def logConfiguration(): Unit = log.info(settings.toString)
protected def systemImpl = this
@ -408,17 +436,15 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten
val scheduler: Scheduler = createScheduler()
val internalClassLoader = Option(Thread.currentThread.getContextClassLoader) getOrElse getClass.getClassLoader
val provider: ActorRefProvider = {
val arguments = Seq(
classOf[String] -> name,
classOf[Settings] -> settings,
classOf[EventStream] -> eventStream,
classOf[Scheduler] -> scheduler,
classOf[ClassLoader] -> internalClassLoader)
classOf[DynamicAccess] -> dynamicAccess)
ReflectiveAccess.createInstance[ActorRefProvider](ProviderClass, arguments, internalClassLoader) match {
dynamicAccess.createInstanceFor[ActorRefProvider](ProviderClass, arguments) match {
case Left(e) throw e
case Right(p) p
}
@ -440,7 +466,7 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten
def locker: Locker = provider.locker
val dispatchers: Dispatchers = new Dispatchers(settings, DefaultDispatcherPrerequisites(
threadFactory, eventStream, deadLetterMailbox, scheduler, internalClassLoader))
threadFactory, eventStream, deadLetterMailbox, scheduler, dynamicAccess))
val dispatcher: MessageDispatcher = dispatchers.defaultGlobalDispatcher
@ -559,8 +585,7 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten
private def loadExtensions() {
import scala.collection.JavaConversions._
settings.config.getStringList("akka.extensions") foreach { fqcn
import ReflectiveAccess.{ getObjectFor, createInstance, noParams, noArgs }
getObjectFor[AnyRef](fqcn, internalClassLoader).fold(_ createInstance[AnyRef](fqcn, noParams, noArgs), Right(_)) match {
dynamicAccess.getObjectFor[AnyRef](fqcn).fold(_ dynamicAccess.createInstanceFor[AnyRef](fqcn, Seq()), Right(_)) match {
case Right(p: ExtensionIdProvider) registerExtension(p.lookup());
case Right(p: ExtensionId[_]) registerExtension(p);
case Right(other) log.error("[{}] is not an 'ExtensionIdProvider' or 'ExtensionId', skipping...", fqcn)

View file

@ -15,7 +15,7 @@ import java.net.MalformedURLException
* for example a remote transport would want to associate additional
* information with an address, then this must be done externally.
*/
final case class Address(protocol: String, system: String, host: Option[String], port: Option[Int]) {
final case class Address private (protocol: String, system: String, host: Option[String], port: Option[Int]) {
def this(protocol: String, system: String) = this(protocol, system, None, None)
def this(protocol: String, system: String, host: String, port: Int) = this(protocol, system, Option(host), Some(port))
@ -62,20 +62,26 @@ object RelativeActorPath {
* This object serves as extractor for Scala and as address parser for Java.
*/
object AddressExtractor {
def unapply(addr: String): Option[Address] = {
def unapply(addr: String): Option[Address] =
try {
val uri = new URI(addr)
if (uri.getScheme == null || (uri.getUserInfo == null && uri.getHost == null)) None
else {
val addr = Address(uri.getScheme, if (uri.getUserInfo != null) uri.getUserInfo else uri.getHost,
if (uri.getUserInfo == null || uri.getHost == null) None else Some(uri.getHost),
if (uri.getPort < 0) None else Some(uri.getPort))
Some(addr)
}
unapply(uri)
} catch {
case _: URISyntaxException None
}
}
def unapply(uri: URI): Option[Address] =
if (uri eq null) None
else if (uri.getScheme == null || (uri.getUserInfo == null && uri.getHost == null)) None
else if (uri.getUserInfo == null) { // case 1: akka://system
if (uri.getPort != -1) None
else Some(Address(uri.getScheme, uri.getHost))
} else { // case 2: akka://system@host:port
if (uri.getHost == null || uri.getPort == -1) None
else Some(
if (uri.getUserInfo == null) Address(uri.getScheme, uri.getHost)
else Address(uri.getScheme, uri.getUserInfo, uri.getHost, uri.getPort))
}
/**
* Try to construct an Address from the given String or throw a java.net.MalformedURLException.
@ -92,18 +98,15 @@ object AddressExtractor {
}
object ActorPathExtractor {
def unapply(addr: String): Option[(Address, Iterable[String])] = {
def unapply(addr: String): Option[(Address, Iterable[String])] =
try {
val uri = new URI(addr)
if (uri.getScheme == null || (uri.getUserInfo == null && uri.getHost == null) || uri.getPath == null) None
else {
val addr = Address(uri.getScheme, if (uri.getUserInfo != null) uri.getUserInfo else uri.getHost,
if (uri.getUserInfo == null || uri.getHost == null) None else Some(uri.getHost),
if (uri.getPort < 0) None else Some(uri.getPort))
Some((addr, ActorPath.split(uri.getPath).drop(1)))
if (uri.getPath == null) None
else AddressExtractor.unapply(uri) match {
case None None
case Some(addr) Some((addr, ActorPath.split(uri.getPath).drop(1)))
}
} catch {
case _: URISyntaxException None
}
}
}

View file

@ -8,7 +8,6 @@ import akka.util.Duration
import com.typesafe.config._
import akka.routing._
import java.util.concurrent.{ TimeUnit, ConcurrentHashMap }
import akka.util.ReflectiveAccess
/**
* This class represents deployment configuration for a given actor path. It is
@ -86,7 +85,7 @@ case object NoScopeGiven extends Scope {
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class Deployer(val settings: ActorSystem.Settings, val classloader: ClassLoader) {
class Deployer(val settings: ActorSystem.Settings, val dynamicAccess: DynamicAccess) {
import scala.collection.JavaConverters._
@ -128,7 +127,7 @@ class Deployer(val settings: ActorSystem.Settings, val classloader: ClassLoader)
case "broadcast" BroadcastRouter(nrOfInstances, routees, resizer)
case fqn
val args = Seq(classOf[Config] -> deployment)
ReflectiveAccess.createInstance[RouterConfig](fqn, args, classloader) match {
dynamicAccess.createInstanceFor[RouterConfig](fqn, args) match {
case Right(router) router
case Left(exception)
throw new IllegalArgumentException(

View file

@ -0,0 +1,129 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.actor
import akka.util.NonFatal
import java.lang.reflect.InvocationTargetException
/**
* The DynamicAccess implementation is the class which is used for
* loading all configurable parts of an actor system (the
* [[akka.actor.ReflectiveDynamicAccess]] is the default implementation).
*
* This is an internal facility and users are not expected to encounter it
* unless they are extending Akka in ways which go beyond simple Extensions.
*/
trait DynamicAccess {
/**
* Convenience method which given a `Class[_]` object and a constructor description
* will create a new instance of that class.
*
* {{{
* val obj = DynamicAccess.createInstanceFor(clazz, Seq(classOf[Config] -> config, classOf[String] -> name))
* }}}
*/
def createInstanceFor[T: ClassManifest](clazz: Class[_], args: Seq[(Class[_], AnyRef)]): Either[Throwable, T] = {
val types = args.map(_._1).toArray
val values = args.map(_._2).toArray
withErrorHandling {
val constructor = clazz.getDeclaredConstructor(types: _*)
constructor.setAccessible(true)
val obj = constructor.newInstance(values: _*).asInstanceOf[T]
val t = classManifest[T].erasure
if (t.isInstance(obj)) Right(obj) else Left(new ClassCastException(clazz + " is not a subtype of " + t))
}
}
/**
* Obtain a `Class[_]` object loaded with the right class loader (i.e. the one
* returned by `classLoader`).
*/
def getClassFor[T: ClassManifest](fqcn: String): Either[Throwable, Class[_ <: T]]
/**
* Obtain an object conforming to the type T, which is expected to be
* instantiated from a class designated by the fully-qualified class name
* given, where the constructor is selected and invoked according to the
* `args` argument. The exact usage of args depends on which type is requested,
* see the relevant requesting code for details.
*/
def createInstanceFor[T: ClassManifest](fqcn: String, args: Seq[(Class[_], AnyRef)]): Either[Throwable, T]
/**
* Obtain the Scala object instance for the given fully-qualified class name, if there is one.
*/
def getObjectFor[T: ClassManifest](fqcn: String): Either[Throwable, T]
/**
* This is the class loader to be used in those special cases where the
* other factory method are not applicable (e.g. when constructing a ClassLoaderBinaryInputStream).
*/
def classLoader: ClassLoader
/**
* Caught exception is returned as Left(exception).
* Unwraps `InvocationTargetException` if its getTargetException is an `Exception`.
* Other `Throwable`, such as `Error` is thrown.
*/
@inline
final def withErrorHandling[T](body: Either[Throwable, T]): Either[Throwable, T] =
try body catch {
case e: InvocationTargetException
e.getTargetException match {
case NonFatal(t) Left(t)
case t throw t
}
case NonFatal(e) Left(e)
}
}
/**
* This is the default [[akka.actor.DynamicAccess]] implementation used by [[akka.actor.ActorSystemImpl]]
* unless overridden. It uses reflection to turn fully-qualified class names into `Class[_]` objects
* and creates instances from there using `getDeclaredConstructor()` and invoking that. The class loader
* to be used for all this is determined by the [[akka.actor.ActorSystemImpl]]s `findClassLoader` method
* by default.
*/
class ReflectiveDynamicAccess(val classLoader: ClassLoader) extends DynamicAccess {
override def getClassFor[T: ClassManifest](fqcn: String): Either[Throwable, Class[_ <: T]] =
try {
val c = classLoader.loadClass(fqcn).asInstanceOf[Class[_ <: T]]
val t = classManifest[T].erasure
if (t.isAssignableFrom(c)) Right(c) else Left(new ClassCastException(t + " is not assignable from " + c))
} catch {
case NonFatal(e) Left(e)
}
override def createInstanceFor[T: ClassManifest](fqcn: String, args: Seq[(Class[_], AnyRef)]): Either[Throwable, T] =
getClassFor(fqcn).fold(Left(_), { c
val types = args.map(_._1).toArray
val values = args.map(_._2).toArray
withErrorHandling {
val constructor = c.getDeclaredConstructor(types: _*)
constructor.setAccessible(true)
val obj = constructor.newInstance(values: _*)
val t = classManifest[T].erasure
if (t.isInstance(obj)) Right(obj) else Left(new ClassCastException(fqcn + " is not a subtype of " + t))
}
})
override def getObjectFor[T: ClassManifest](fqcn: String): Either[Throwable, T] = {
getClassFor(fqcn).fold(Left(_), { c
withErrorHandling {
val module = c.getDeclaredField("MODULE$")
module.setAccessible(true)
val t = classManifest[T].erasure
module.get(null) match {
case null Left(new NullPointerException)
case x if !t.isInstance(x) Left(new ClassCastException(fqcn + " is not a subtype of " + t))
case x Right(x.asInstanceOf[T])
}
}
})
}
}

View file

@ -3,8 +3,6 @@
*/
package akka.actor
import akka.util.ReflectiveAccess
/**
* The basic ActorSystem covers all that is needed for locally running actors,
* using futures and so on. In addition, more features can hook into it and
@ -73,12 +71,12 @@ trait ExtensionIdProvider {
/**
* This is a one-stop-shop if all you want is an extension which is
* constructed with the ActorSystemImpl as its only constructor argument:
* constructed with the ExtendedActorSystem as its only constructor argument:
*
* {{{
* object MyExt extends ExtensionKey[Ext]
*
* class Ext(system: ActorSystemImpl) extends MyExt {
* class Ext(system: ExtendedActorSystem) extends MyExt {
* ...
* }
* }}}
@ -89,7 +87,7 @@ trait ExtensionIdProvider {
* public class MyExt extends Extension {
* static final ExtensionKey<MyExt> key = new ExtensionKey<MyExt>(MyExt.class);
*
* public MyExt(ActorSystemImpl system) {
* public MyExt(ExtendedActorSystem system) {
* ...
* }
* }}}
@ -99,7 +97,7 @@ abstract class ExtensionKey[T <: Extension](implicit m: ClassManifest[T]) extend
override def lookup(): ExtensionId[T] = this
def createExtension(system: ExtendedActorSystem): T =
ReflectiveAccess.createInstance[T](m.erasure, Array[Class[_]](classOf[ActorSystemImpl]), Array[AnyRef](system)) match {
system.dynamicAccess.createInstanceFor[T](m.erasure, Seq(classOf[ExtendedActorSystem] -> system)) match {
case Left(ex) throw ex
case Right(r) r
}

View file

@ -11,7 +11,9 @@ import java.util.concurrent.atomic.{ AtomicReference ⇒ AtomVar }
import akka.serialization.{ Serialization, SerializationExtension }
import akka.dispatch._
import java.util.concurrent.TimeoutException
import java.util.concurrent.TimeUnit.MILLISECONDS
import java.lang.IllegalStateException
import akka.util.Duration
trait TypedActorFactory {
@ -129,7 +131,8 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi
val serializedParameters = Array.ofDim[(Int, Class[_], Array[Byte])](ps.length)
for (i 0 until ps.length) {
val p = ps(i)
val s = SerializationExtension(Serialization.currentSystem.value).findSerializerFor(p)
val system = akka.serialization.JavaSerializer.currentSystem.value
val s = SerializationExtension(system).findSerializerFor(p)
val m = if (s.includeManifest) p.getClass else null
serializedParameters(i) = (s.identifier, m, s toBinary parameters(i)) //Mutable for the sake of sanity
}
@ -146,7 +149,7 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi
//TODO implement writeObject and readObject to serialize
//TODO Possible optimization is to special encode the parameter-types to conserve space
private def readResolve(): AnyRef = {
val system = akka.serialization.Serialization.currentSystem.value
val system = akka.serialization.JavaSerializer.currentSystem.value
if (system eq null) throw new IllegalStateException(
"Trying to deserialize a SerializedMethodCall without an ActorSystem in scope." +
" Use akka.serialization.Serialization.currentSystem.withValue(system) { ... }")
@ -158,7 +161,8 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi
val deserializedParameters: Array[AnyRef] = Array.ofDim[AnyRef](a.length) //Mutable for the sake of sanity
for (i 0 until a.length) {
val (sId, manifest, bytes) = a(i)
deserializedParameters(i) = serialization.serializerByIdentity(sId).fromBinary(bytes, Option(manifest))
deserializedParameters(i) =
serialization.serializerByIdentity(sId).fromBinary(bytes, Option(manifest))
}
deserializedParameters
@ -500,7 +504,7 @@ case class TypedProps[T <: AnyRef] protected[TypedProps] (
/**
* @return a new TypedProps that will use the specified Timeout for its non-void-returning methods,
* if null is specified, it will use the default ActorTimeout as specified in the configuration.
* if null is specified, it will use the default timeout as specified in the configuration.
*
* Java API
*/
@ -508,7 +512,7 @@ case class TypedProps[T <: AnyRef] protected[TypedProps] (
/**
* @return a new TypedProps that will use the specified Timeout for its non-void-returning methods,
* if None is specified, it will use the default ActorTimeout as specified in the configuration.
* if None is specified, it will use the default timeout as specified in the configuration.
*
* Scala API
*/
@ -548,6 +552,11 @@ class TypedActorExtension(system: ExtendedActorSystem) extends TypedActorFactory
val serialization = SerializationExtension(system)
val settings = system.settings
/**
* Default timeout for typed actor methods with non-void return type
*/
final val DefaultReturnTimeout = Timeout(Duration(settings.config.getMilliseconds("akka.actor.typed.timeout"), MILLISECONDS))
/**
* Retrieves the underlying ActorRef for the supplied TypedActor proxy, or null if none found
*/
@ -573,7 +582,7 @@ class TypedActorExtension(system: ExtendedActorSystem) extends TypedActorFactory
new TypedActorInvocationHandler(
this,
actorVar,
if (props.timeout.isDefined) props.timeout.get else this.settings.ActorTimeout)).asInstanceOf[R]
if (props.timeout.isDefined) props.timeout.get else DefaultReturnTimeout)).asInstanceOf[R]
proxyVar match {
case null

View file

@ -12,7 +12,6 @@ import akka.actor.ActorSystem
import scala.annotation.tailrec
import akka.event.EventStream
import com.typesafe.config.Config
import akka.util.ReflectiveAccess
import akka.serialization.SerializationExtension
import akka.util.NonFatal
import akka.event.Logging.LogEventException
@ -26,7 +25,7 @@ final case class Envelope(val message: Any, val sender: ActorRef)(system: ActorS
val ser = SerializationExtension(system)
ser.serialize(msg) match { //Verify serializability
case Left(t) throw t
case Right(bytes) ser.deserialize(bytes, msg.getClass, None) match { //Verify deserializability
case Right(bytes) ser.deserialize(bytes, msg.getClass) match { //Verify deserializability
case Left(t) throw t
case _ //All good
}
@ -157,7 +156,10 @@ trait ExecutionContext {
* log the problem or whatever is appropriate for the implementation.
*/
def reportFailure(t: Throwable): Unit
}
private[akka] trait LoadMetrics { self: Executor
def atFullThrottle(): Boolean
}
object MessageDispatcher {
@ -186,9 +188,14 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
def id: String
/**
* Attaches the specified actor instance to this dispatcher
* Attaches the specified actor instance to this dispatcher, which includes
* scheduling it to run for the first time (Create() is expected to have
* been enqueued by the ActorCell upon mailbox creation).
*/
final def attach(actor: ActorCell): Unit = register(actor)
final def attach(actor: ActorCell): Unit = {
register(actor)
registerForExecution(actor.mailbox, false, true)
}
/**
* Detaches the specified actor instance from this dispatcher
@ -244,7 +251,7 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
() if (inhabitantsUpdater.decrementAndGet(this) == 0) ifSensibleToDoSoThenScheduleShutdown()
/**
* If you override it, you must call it. But only ever once. See "attach" for only invocation
* If you override it, you must call it. But only ever once. See "attach" for only invocation.
*/
protected[akka] def register(actor: ActorCell) {
inhabitantsUpdater.incrementAndGet(this)
@ -261,6 +268,8 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
mailBox.cleanUp()
}
def inhabitants: Long = inhabitantsUpdater.get(this)
private val shutdownAction = new Runnable {
@tailrec
final def run() {
@ -369,7 +378,7 @@ abstract class MessageDispatcherConfigurator(val config: Config, val prerequisit
}
case fqcn
val args = Seq(classOf[Config] -> config)
ReflectiveAccess.createInstance[MailboxType](fqcn, args, prerequisites.classloader) match {
prerequisites.dynamicAccess.createInstanceFor[MailboxType](fqcn, args) match {
case Right(instance) instance
case Left(exception)
throw new IllegalArgumentException(
@ -385,8 +394,10 @@ abstract class MessageDispatcherConfigurator(val config: Config, val prerequisit
case null | "" | "fork-join-executor" new ForkJoinExecutorConfigurator(config.getConfig("fork-join-executor"), prerequisites)
case "thread-pool-executor" new ThreadPoolExecutorConfigurator(config.getConfig("thread-pool-executor"), prerequisites)
case fqcn
val constructorSignature = Array[Class[_]](classOf[Config], classOf[DispatcherPrerequisites])
ReflectiveAccess.createInstance[ExecutorServiceConfigurator](fqcn, constructorSignature, Array[AnyRef](config, prerequisites), prerequisites.classloader) match {
val args = Seq(
classOf[Config] -> config,
classOf[DispatcherPrerequisites] -> prerequisites)
prerequisites.dynamicAccess.createInstanceFor[ExecutorServiceConfigurator](fqcn, args) match {
case Right(instance) instance
case Left(exception) throw new IllegalArgumentException(
("""Cannot instantiate ExecutorServiceConfigurator ("executor = [%s]"), defined in [%s],
@ -439,11 +450,13 @@ object ForkJoinExecutorConfigurator {
final class AkkaForkJoinPool(parallelism: Int,
threadFactory: ForkJoinPool.ForkJoinWorkerThreadFactory,
unhandledExceptionHandler: Thread.UncaughtExceptionHandler)
extends ForkJoinPool(parallelism, threadFactory, unhandledExceptionHandler, true) {
extends ForkJoinPool(parallelism, threadFactory, unhandledExceptionHandler, true) with LoadMetrics {
override def execute(r: Runnable): Unit = r match {
case m: Mailbox super.execute(new MailboxExecutionTask(m))
case other super.execute(other)
}
def atFullThrottle(): Boolean = this.getActiveThreadCount() >= this.getParallelism()
}
/**

View file

@ -4,12 +4,11 @@
package akka.dispatch
import util.DynamicVariable
import akka.actor.{ ActorCell, ActorRef }
import java.util.concurrent.{ LinkedBlockingQueue, ConcurrentLinkedQueue, ConcurrentSkipListSet }
import annotation.tailrec
import java.util.concurrent.atomic.AtomicBoolean
import akka.util.Duration
import akka.util.{ Duration, Helpers }
import java.util.{ Comparator, Iterator }
import java.util.concurrent.{ Executor, LinkedBlockingQueue, ConcurrentLinkedQueue, ConcurrentSkipListSet }
/**
* An executor based event driven dispatcher which will try to redistribute work from busy actors to idle actors. It is assumed
@ -32,20 +31,27 @@ class BalancingDispatcher(
throughputDeadlineTime: Duration,
mailboxType: MailboxType,
_executorServiceFactoryProvider: ExecutorServiceFactoryProvider,
_shutdownTimeout: Duration)
_shutdownTimeout: Duration,
attemptTeamWork: Boolean)
extends Dispatcher(_prerequisites, _id, throughput, throughputDeadlineTime, mailboxType, _executorServiceFactoryProvider, _shutdownTimeout) {
val buddies = new ConcurrentSkipListSet[ActorCell](akka.util.Helpers.IdentityHashComparator)
val rebalance = new AtomicBoolean(false)
val team = new ConcurrentSkipListSet[ActorCell](
Helpers.identityHashComparator(new Comparator[ActorCell] {
def compare(l: ActorCell, r: ActorCell) = l.self.path compareTo r.self.path
}))
val messageQueue: MessageQueue = mailboxType match {
case u: UnboundedMailbox new QueueBasedMessageQueue with UnboundedMessageQueueSemantics {
final val queue = new ConcurrentLinkedQueue[Envelope]
}
case BoundedMailbox(cap, timeout) new QueueBasedMessageQueue with BoundedMessageQueueSemantics {
final val queue = new LinkedBlockingQueue[Envelope](cap)
final val pushTimeOut = timeout
}
case UnboundedMailbox()
new QueueBasedMessageQueue with UnboundedMessageQueueSemantics {
final val queue = new ConcurrentLinkedQueue[Envelope]
}
case BoundedMailbox(cap, timeout)
new QueueBasedMessageQueue with BoundedMessageQueueSemantics {
final val queue = new LinkedBlockingQueue[Envelope](cap)
final val pushTimeOut = timeout
}
case other throw new IllegalArgumentException("Only handles BoundedMailbox and UnboundedMailbox, but you specified [" + other + "]")
}
@ -76,38 +82,33 @@ class BalancingDispatcher(
}
}
protected[akka] override def register(actor: ActorCell) = {
protected[akka] override def register(actor: ActorCell): Unit = {
super.register(actor)
buddies.add(actor)
team.add(actor)
}
protected[akka] override def unregister(actor: ActorCell) = {
buddies.remove(actor)
protected[akka] override def unregister(actor: ActorCell): Unit = {
team.remove(actor)
super.unregister(actor)
intoTheFray(except = actor) //When someone leaves, he tosses a friend into the fray
teamWork()
}
def intoTheFray(except: ActorCell): Unit =
if (rebalance.compareAndSet(false, true)) {
try {
val i = buddies.iterator()
@tailrec
def throwIn(): Unit = {
val n = if (i.hasNext) i.next() else null
if (n eq null) ()
else if ((n ne except) && registerForExecution(n.mailbox, false, false)) ()
else throwIn()
}
throwIn()
} finally {
rebalance.set(false)
}
}
override protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope) = {
messageQueue.enqueue(receiver.self, invocation)
registerForExecution(receiver.mailbox, false, false)
intoTheFray(except = receiver)
if (!registerForExecution(receiver.mailbox, false, false)) teamWork()
}
protected def teamWork(): Unit = if (attemptTeamWork) {
@tailrec def scheduleOne(i: Iterator[ActorCell] = team.iterator): Unit =
if (messageQueue.hasMessages
&& i.hasNext
&& (executorService.get().executor match {
case lm: LoadMetrics lm.atFullThrottle == false
case other true
})
&& !registerForExecution(i.next.mailbox, false, false))
scheduleOne(i)
scheduleOne()
}
}

View file

@ -32,12 +32,11 @@ class Dispatcher(
val shutdownTimeout: Duration)
extends MessageDispatcher(_prerequisites) {
protected[akka] val executorServiceFactory: ExecutorServiceFactory =
protected val executorServiceFactory: ExecutorServiceFactory =
executorServiceFactoryProvider.createExecutorServiceFactory(id, prerequisites.threadFactory)
protected[akka] val executorService = new AtomicReference[ExecutorService](new ExecutorServiceDelegate {
lazy val executor = executorServiceFactory.createExecutorService
})
protected val executorService = new AtomicReference[ExecutorServiceDelegate](
new ExecutorServiceDelegate { lazy val executor = executorServiceFactory.createExecutorService })
protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope) = {
val mbox = receiver.mailbox

View file

@ -4,22 +4,24 @@
package akka.dispatch
import akka.actor.newUuid
import akka.util.{ Duration, ReflectiveAccess }
import akka.actor.ActorSystem
import akka.event.EventStream
import akka.actor.Scheduler
import com.typesafe.config.Config
import com.typesafe.config.ConfigFactory
import java.util.concurrent.{ ConcurrentHashMap, TimeUnit, ThreadFactory }
import scala.collection.JavaConverters.mapAsJavaMapConverter
import com.typesafe.config.{ ConfigFactory, Config }
import Dispatchers.DefaultDispatcherId
import akka.actor.{ Scheduler, DynamicAccess, ActorSystem }
import akka.event.Logging.Warning
import java.util.concurrent.{ ThreadFactory, TimeUnit, ConcurrentHashMap }
import akka.event.EventStream
import akka.util.Duration
trait DispatcherPrerequisites {
def threadFactory: ThreadFactory
def eventStream: EventStream
def deadLetterMailbox: Mailbox
def scheduler: Scheduler
def classloader: ClassLoader
def dynamicAccess: DynamicAccess
}
case class DefaultDispatcherPrerequisites(
@ -27,7 +29,7 @@ case class DefaultDispatcherPrerequisites(
val eventStream: EventStream,
val deadLetterMailbox: Mailbox,
val scheduler: Scheduler,
val classloader: ClassLoader) extends DispatcherPrerequisites
val dynamicAccess: DynamicAccess) extends DispatcherPrerequisites
object Dispatchers {
/**
@ -137,7 +139,7 @@ class Dispatchers(val settings: ActorSystem.Settings, val prerequisites: Dispatc
case "PinnedDispatcher" new PinnedDispatcherConfigurator(cfg, prerequisites)
case fqn
val args = Seq(classOf[Config] -> cfg, classOf[DispatcherPrerequisites] -> prerequisites)
ReflectiveAccess.createInstance[MessageDispatcherConfigurator](fqn, args, prerequisites.classloader) match {
prerequisites.dynamicAccess.createInstanceFor[MessageDispatcherConfigurator](fqn, args) match {
case Right(configurator) configurator
case Left(exception)
throw new IllegalArgumentException(
@ -187,7 +189,8 @@ class BalancingDispatcherConfigurator(config: Config, prerequisites: DispatcherP
config.getInt("throughput"),
Duration(config.getNanoseconds("throughput-deadline-time"), TimeUnit.NANOSECONDS),
mailboxType, configureExecutor(),
Duration(config.getMilliseconds("shutdown-timeout"), TimeUnit.MILLISECONDS))
Duration(config.getMilliseconds("shutdown-timeout"), TimeUnit.MILLISECONDS),
config.getBoolean("attempt-teamwork"))
/**
* Returns the same dispatcher instance for each invocation

View file

@ -20,6 +20,7 @@ import akka.event.Logging.Debug
import java.util.concurrent.TimeUnit.NANOSECONDS
import java.util.concurrent.{ ExecutionException, Callable, TimeoutException }
import java.util.concurrent.atomic.{ AtomicInteger, AtomicReferenceFieldUpdater }
import akka.pattern.AskTimeoutException
object Await {
@ -151,6 +152,26 @@ object Futures {
for (r fr; b fb) yield { r add b; r }
}
}
/**
* Signals that the current thread of execution will potentially engage
* in blocking calls after the call to this method, giving the system a
* chance to spawn new threads, reuse old threads or otherwise, to prevent
* starvation and/or unfairness.
*
* Assures that any Future tasks initiated in the current thread will be
* executed asynchronously, including any tasks currently queued to be
* executed in the current thread. This is needed if the current task may
* block, causing delays in executing the remaining tasks which in some
* cases may cause a deadlock.
*
* Usage: Call this method in a callback (map, flatMap etc also count) to a Future,
* if you will be doing blocking in the callback.
*
* Note: Calling 'Await.result(future)' or 'Await.ready(future)' will automatically trigger this method.
*
*/
def blocking(): Unit = Future.blocking()
}
object Future {
@ -317,17 +338,22 @@ object Future {
* }
* </pre>
*/
def blocking(implicit executor: ExecutionContext): Unit =
def blocking(): Unit =
_taskStack.get match {
case stack if (stack ne null) && stack.nonEmpty
val executionContext = _executionContext.get match {
case null throw new IllegalStateException("'blocking' needs to be invoked inside a Future callback.")
case some some
}
val tasks = stack.elems
stack.clear()
_taskStack.remove()
dispatchTask(() _taskStack.get.elems = tasks, true)
dispatchTask(() _taskStack.get.elems = tasks, true)(executionContext)
case _ _taskStack.remove()
}
private val _taskStack = new ThreadLocal[Stack[() Unit]]()
private val _executionContext = new ThreadLocal[ExecutionContext]()
/**
* Internal API, do not call
@ -339,7 +365,7 @@ object Future {
new Runnable {
def run =
try {
_executionContext set executor
val taskStack = Stack.empty[() Unit]
taskStack push task
_taskStack set taskStack
@ -352,7 +378,10 @@ object Future {
case NonFatal(e) executor.reportFailure(e)
}
}
} finally { _taskStack.remove() }
} finally {
_executionContext.remove()
_taskStack.remove()
}
})
}
@ -795,8 +824,9 @@ class DefaultPromise[T](implicit val executor: ExecutionContext) extends Abstrac
def result(atMost: Duration)(implicit permit: CanAwait): T =
ready(atMost).value.get match {
case Left(e) throw e
case Right(r) r
case Left(e: AskTimeoutException) throw new AskTimeoutException(e.getMessage, e) // to get meaningful stack trace
case Left(e) throw e
case Right(r) r
}
def value: Option[Either[Throwable, T]] = getState match {

View file

@ -189,8 +189,11 @@ private[akka] abstract class Mailbox(val actor: ActorCell) extends MessageQueue
final def processAllSystemMessages() {
var nextMessage = systemDrain()
try {
while (nextMessage ne null) {
if (debug) println(actor.self + " processing system message " + nextMessage + " with children " + actor.childrenRefs)
while ((nextMessage ne null) && !isClosed) {
if (debug) println(actor.self + " processing system message " + nextMessage + " with " +
(if (actor.childrenRefs.isEmpty) "no children"
else if (actor.childrenRefs.size > 20) actor.childrenRefs.size + " children"
else actor.childrenRefs.mkString("children:\n ", "\n ", "")))
actor systemInvoke nextMessage
nextMessage = nextMessage.next
// dont ever execute normal message when system message present!
@ -236,15 +239,26 @@ private[akka] abstract class Mailbox(val actor: ActorCell) extends MessageQueue
}
trait MessageQueue {
/*
* These method need to be implemented in subclasses; they should not rely on the internal stuff above.
/**
* Try to enqueue the message to this queue, or throw an exception.
*/
def enqueue(receiver: ActorRef, handle: Envelope)
def enqueue(receiver: ActorRef, handle: Envelope): Unit // NOTE: receiver is used only in two places, but cannot be removed
/**
* Try to dequeue the next message from this queue, return null failing that.
*/
def dequeue(): Envelope
/**
* Should return the current number of messages held in this queue; may
* always return 0 if no other value is available efficiently. Do not use
* this for testing for presence of messages, use `hasMessages` instead.
*/
def numberOfMessages: Int
/**
* Indicates whether this queue is non-empty.
*/
def hasMessages: Boolean
}
@ -292,15 +306,15 @@ trait DefaultSystemMessageQueue { self: Mailbox ⇒
}
trait UnboundedMessageQueueSemantics extends QueueBasedMessageQueue {
final def enqueue(receiver: ActorRef, handle: Envelope): Unit = queue add handle
final def dequeue(): Envelope = queue.poll()
def enqueue(receiver: ActorRef, handle: Envelope): Unit = queue add handle
def dequeue(): Envelope = queue.poll()
}
trait BoundedMessageQueueSemantics extends QueueBasedMessageQueue {
def pushTimeOut: Duration
override def queue: BlockingQueue[Envelope]
final def enqueue(receiver: ActorRef, handle: Envelope) {
def enqueue(receiver: ActorRef, handle: Envelope) {
if (pushTimeOut.length > 0) {
queue.offer(handle, pushTimeOut.length, pushTimeOut.unit) || {
throw new MessageQueueAppendFailedException("Couldn't enqueue message " + handle + " to " + receiver)
@ -308,13 +322,13 @@ trait BoundedMessageQueueSemantics extends QueueBasedMessageQueue {
} else queue put handle
}
final def dequeue(): Envelope = queue.poll()
def dequeue(): Envelope = queue.poll()
}
trait QueueBasedMessageQueue extends MessageQueue {
def queue: Queue[Envelope]
final def numberOfMessages = queue.size
final def hasMessages = !queue.isEmpty
def numberOfMessages = queue.size
def hasMessages = !queue.isEmpty
}
/**

View file

@ -81,14 +81,16 @@ case class ThreadPoolConfig(allowCorePoolTimeout: Boolean = ThreadPoolConfig.def
extends ExecutorServiceFactoryProvider {
class ThreadPoolExecutorServiceFactory(val threadFactory: ThreadFactory) extends ExecutorServiceFactory {
def createExecutorService: ExecutorService = {
val service = new ThreadPoolExecutor(
val service: ThreadPoolExecutor = new ThreadPoolExecutor(
corePoolSize,
maxPoolSize,
threadTimeout.length,
threadTimeout.unit,
queueFactory(),
threadFactory,
rejectionPolicy)
rejectionPolicy) with LoadMetrics {
def atFullThrottle(): Boolean = this.getActiveCount >= this.getPoolSize
}
service.allowCoreThreadTimeOut(allowCorePoolTimeout)
service
}
@ -182,7 +184,7 @@ case class MonitorableThreadFactory(name: String,
protected def wire[T <: Thread](t: T): T = {
t.setUncaughtExceptionHandler(exceptionHandler)
t.setDaemon(daemonic)
contextClassLoader foreach (t.setContextClassLoader(_))
contextClassLoader foreach t.setContextClassLoader
t
}
}

View file

@ -6,7 +6,6 @@ package akka.event
import akka.actor._
import akka.AkkaException
import akka.actor.ActorSystem.Settings
import akka.util.ReflectiveAccess
import akka.config.ConfigurationException
import akka.util.ReentrantGuard
import akka.util.duration._
@ -101,7 +100,7 @@ trait LoggingBus extends ActorEventBus {
if loggerName != StandardOutLoggerName
} yield {
try {
ReflectiveAccess.getClassFor[Actor](loggerName, system.internalClassLoader) match {
system.dynamicAccess.getClassFor[Actor](loggerName) match {
case Right(actorClass) addLogger(system, actorClass, level, logName)
case Left(exception) throw exception
}
@ -350,7 +349,7 @@ object Logging {
object Extension extends ExtensionKey[LogExt]
class LogExt(system: ActorSystemImpl) extends Extension {
class LogExt(system: ExtendedActorSystem) extends Extension {
private val loggerId = new AtomicInteger
def id() = loggerId.incrementAndGet()
}

View file

@ -9,15 +9,14 @@ import akka.util.duration._
import akka.config.ConfigurationException
import akka.pattern.pipe
import akka.pattern.AskSupport
import com.typesafe.config.Config
import scala.collection.JavaConversions.iterableAsScalaIterable
import java.util.concurrent.atomic.{ AtomicLong, AtomicBoolean }
import java.util.concurrent.TimeUnit
import java.util.concurrent.locks.ReentrantLock
import akka.jsr166y.ThreadLocalRandom
import akka.util.Unsafe
import akka.dispatch.Dispatchers
/**
* A RoutedActorRef is an ActorRef that has a set of connected ActorRef and it uses a Router to
@ -26,25 +25,88 @@ import akka.jsr166y.ThreadLocalRandom
private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _supervisor: InternalActorRef, _path: ActorPath)
extends LocalActorRef(
_system,
_props.copy(creator = () _props.routerConfig.createActor()),
_props.copy(creator = () _props.routerConfig.createActor(), dispatcher = _props.routerConfig.routerDispatcher),
_supervisor,
_path) {
private val routeeProps = _props.copy(routerConfig = NoRouter)
private val resizeProgress = new AtomicBoolean
/*
* CAUTION: RoutedActorRef is PROBLEMATIC
* ======================================
*
* We are constructing/assembling the children outside of the scope of the
* Router actor, inserting them in its childrenRef list, which is not at all
* synchronized. This is done exactly once at start-up, all other accesses
* are done from the Router actor. This means that the only thing which is
* really hairy is making sure that the Router does not touch its childrenRefs
* before we are done with them: lock the monitor of the actor cell (hence the
* override of newActorCell) and use that to block the Router constructor for
* as long as it takes to setup the RoutedActorRef itself.
*/
override def newActorCell(
system: ActorSystemImpl,
ref: InternalActorRef,
props: Props,
supervisor: InternalActorRef,
receiveTimeout: Option[Duration]): ActorCell =
{
val cell = super.newActorCell(system, ref, props, supervisor, receiveTimeout)
Unsafe.instance.monitorEnter(cell)
cell
}
private[akka] val routerConfig = _props.routerConfig
private[akka] val routeeProps = _props.copy(routerConfig = NoRouter)
private[akka] val resizeInProgress = new AtomicBoolean
private val resizeCounter = new AtomicLong
@volatile
private var _routees: IndexedSeq[ActorRef] = IndexedSeq.empty[ActorRef] // this MUST be initialized during createRoute
def routees = _routees
@volatile
private var _routeeProvider: RouteeProvider = _
def routeeProvider = _routeeProvider
val route =
try {
_routeeProvider = routerConfig.createRouteeProvider(actorContext)
val r = routerConfig.createRoute(routeeProps, routeeProvider)
// initial resize, before message send
routerConfig.resizer foreach { r
if (r.isTimeForResize(resizeCounter.getAndIncrement()))
r.resize(routeeProps, routeeProvider)
}
r
} finally {
assert(Thread.holdsLock(actorContext))
Unsafe.instance.monitorExit(actorContext) // unblock Routers constructor
}
if (routerConfig.resizer.isEmpty && _routees.isEmpty)
throw new ActorInitializationException("router " + routerConfig + " did not register routees!")
/*
* end of construction
*/
def applyRoute(sender: ActorRef, message: Any): Iterable[Destination] = message match {
case _: AutoReceivedMessage Destination(this, this) :: Nil
case Terminated(_) Destination(this, this) :: Nil
case CurrentRoutees
sender ! RouterRoutees(_routees)
Nil
case _
if (route.isDefinedAt(sender, message)) route(sender, message)
else Nil
}
/**
* Adds the routees to existing routees.
* Adds death watch of the routees so that they are removed when terminated.
* Not thread safe, but intended to be called from protected points, such as
* `RouterConfig.createRoute` and `Resizer.resize`
*/
private[akka] def addRoutees(newRoutees: IndexedSeq[ActorRef]) {
private[akka] def addRoutees(newRoutees: IndexedSeq[ActorRef]): Unit = {
_routees = _routees ++ newRoutees
// subscribe to Terminated messages for all route destinations, to be handled by Router actor
newRoutees foreach underlying.watch
@ -56,34 +118,11 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup
* Not thread safe, but intended to be called from protected points, such as
* `Resizer.resize`
*/
private[akka] def removeRoutees(abandonedRoutees: IndexedSeq[ActorRef]) {
private[akka] def removeRoutees(abandonedRoutees: IndexedSeq[ActorRef]): Unit = {
_routees = _routees diff abandonedRoutees
abandonedRoutees foreach underlying.unwatch
}
private val routeeProvider = _props.routerConfig.createRouteeProvider(actorContext)
val route = _props.routerConfig.createRoute(routeeProps, routeeProvider)
// initial resize, before message send
resize()
def applyRoute(sender: ActorRef, message: Any): Iterable[Destination] = message match {
case _: AutoReceivedMessage Nil
case Terminated(_) Nil
case CurrentRoutees
sender ! RouterRoutees(_routees)
Nil
case _
if (route.isDefinedAt(sender, message)) route(sender, message)
else Nil
}
if (_props.routerConfig.resizer.isEmpty && _routees.isEmpty)
throw new ActorInitializationException("router " + _props.routerConfig + " did not register routees!")
_routees match {
case x _routees = x // volatile write to publish the route before sending messages
}
override def !(message: Any)(implicit sender: ActorRef = null): Unit = {
resize()
@ -95,20 +134,15 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup
}
applyRoute(s, message) match {
case Nil super.!(message)(s)
case refs refs foreach (p p.recipient.!(msg)(p.sender))
case Destination(_, x) :: Nil if x eq this super.!(message)(s)
case refs refs foreach (p p.recipient.!(msg)(p.sender))
}
}
def resize() {
for (r _props.routerConfig.resizer) {
if (r.isTimeForResize(resizeCounter.getAndIncrement()) && resizeProgress.compareAndSet(false, true)) {
try {
r.resize(routeeProps, routeeProvider)
} finally {
resizeProgress.set(false)
}
}
def resize(): Unit = {
for (r routerConfig.resizer) {
if (r.isTimeForResize(resizeCounter.getAndIncrement()) && resizeInProgress.compareAndSet(false, true))
super.!(Router.Resize)
}
}
}
@ -139,6 +173,11 @@ trait RouterConfig {
def createActor(): Router = new Router {}
/**
* Dispatcher ID to use for running the head actor, i.e. the [[akka.routing.Router]].
*/
def routerDispatcher: String
/**
* Overridable merge strategy, by default completely prefers this (i.e. no merge).
*/
@ -246,13 +285,20 @@ trait CustomRoute {
*/
trait Router extends Actor {
val ref = self match {
case x: RoutedActorRef x
case _ throw new ActorInitializationException("Router actor can only be used in RoutedActorRef")
// make sure that we synchronize properly to get the childrenRefs into our CPU cache
val ref = context.synchronized {
self match {
case x: RoutedActorRef x
case _ throw new ActorInitializationException("Router actor can only be used in RoutedActorRef")
}
}
final def receive = ({
case Router.Resize
try ref.routerConfig.resizer foreach (_.resize(ref.routeeProps, ref.routeeProvider))
finally assert(ref.resizeInProgress.getAndSet(false))
case Terminated(child)
ref.removeRoutees(IndexedSeq(child))
if (ref.routees.isEmpty) context.stop(self)
@ -264,6 +310,10 @@ trait Router extends Actor {
}
}
private object Router {
case object Resize
}
/**
* Used to broadcast a message to all connections in a router; only the
* contained message will be forwarded, i.e. the `Broadcast(...)`
@ -302,6 +352,7 @@ case class Destination(sender: ActorRef, recipient: ActorRef)
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
case object NoRouter extends RouterConfig {
def createRoute(props: Props, routeeProvider: RouteeProvider): Route = null
def routerDispatcher: String = ""
override def withFallback(other: RouterConfig): RouterConfig = other
}
@ -311,13 +362,20 @@ case object NoRouter extends RouterConfig {
case object FromConfig extends RouterConfig {
def createRoute(props: Props, routeeProvider: RouteeProvider): Route =
throw new ConfigurationException("router " + routeeProvider.context.self + " needs external configuration from file (e.g. application.conf)")
def routerDispatcher: String = Dispatchers.DefaultDispatcherId
}
/**
* Java API: Router configuration which has no default, i.e. external configuration is required.
*
* This can be used when the dispatcher to be used for the head Router needs to be configured
* (defaults to default-dispatcher).
*/
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
case class FromConfig() extends RouterConfig {
case class FromConfig(val routerDispatcher: String = Dispatchers.DefaultDispatcherId) extends RouterConfig {
def this() = this(Dispatchers.DefaultDispatcherId)
def createRoute(props: Props, routeeProvider: RouteeProvider): Route =
throw new ConfigurationException("router " + routeeProvider.context.self + " needs external configuration from file (e.g. application.conf)")
}
@ -348,7 +406,8 @@ object RoundRobinRouter {
* using `actorFor` in [[akka.actor.ActorRefProvider]]
*/
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None)
case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
extends RouterConfig with RoundRobinLike {
/**
@ -374,6 +433,11 @@ case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] =
* Java API
*/
def this(resizer: Resizer) = this(resizer = Some(resizer))
/**
* Java API for setting routerDispatcher
*/
def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId)
}
trait RoundRobinLike { this: RouterConfig
@ -428,7 +492,8 @@ object RandomRouter {
* using `actorFor` in [[akka.actor.ActorRefProvider]]
*/
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None)
case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
extends RouterConfig with RandomLike {
/**
@ -454,6 +519,11 @@ case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil,
* Java API
*/
def this(resizer: Resizer) = this(resizer = Some(resizer))
/**
* Java API for setting routerDispatcher
*/
def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId)
}
trait RandomLike { this: RouterConfig
@ -514,7 +584,8 @@ object SmallestMailboxRouter {
* using `actorFor` in [[akka.actor.ActorRefProvider]]
*/
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None)
case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
extends RouterConfig with SmallestMailboxLike {
/**
@ -540,6 +611,11 @@ case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[Strin
* Java API
*/
def this(resizer: Resizer) = this(resizer = Some(resizer))
/**
* Java API for setting routerDispatcher
*/
def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId)
}
trait SmallestMailboxLike { this: RouterConfig
@ -659,7 +735,8 @@ object BroadcastRouter {
* using `actorFor` in [[akka.actor.ActorRefProvider]]
*/
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
case class BroadcastRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None)
case class BroadcastRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
extends RouterConfig with BroadcastLike {
/**
@ -686,6 +763,10 @@ case class BroadcastRouter(nrOfInstances: Int = 0, routees: Iterable[String] = N
*/
def this(resizer: Resizer) = this(resizer = Some(resizer))
/**
* Java API for setting routerDispatcher
*/
def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId)
}
trait BroadcastLike { this: RouterConfig
@ -732,7 +813,8 @@ object ScatterGatherFirstCompletedRouter {
*/
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, within: Duration,
override val resizer: Option[Resizer] = None)
override val resizer: Option[Resizer] = None,
val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
extends RouterConfig with ScatterGatherFirstCompletedLike {
if (within <= Duration.Zero) throw new IllegalArgumentException(
@ -761,6 +843,11 @@ case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: It
* Java API
*/
def this(resizer: Resizer, w: Duration) = this(resizer = Some(resizer), within = w)
/**
* Java API for setting routerDispatcher
*/
def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId)
}
trait ScatterGatherFirstCompletedLike { this: RouterConfig
@ -795,14 +882,22 @@ trait Resizer {
* for the initial resize and continues with 1 for the first message. Make sure to perform
* initial resize before first message (messageCounter == 0), because there is no guarantee
* that resize will be done when concurrent messages are in play.
*
* CAUTION: this method is invoked from the thread which tries to send a
* message to the pool, i.e. the ActorRef.!() method, hence it may be called
* concurrently.
*/
def isTimeForResize(messageCounter: Long): Boolean
/**
* Decide if the capacity of the router need to be changed. Will be invoked when `isTimeForResize`
* returns true and no other resize is in progress.
* Create and register more routees with `routeeProvider.registerRoutees(newRoutees)
* or remove routees with `routeeProvider.unregisterRoutees(abandonedRoutees)` and
* sending [[akka.actor.PoisonPill]] to them.
*
* This method is invoked only in the context of the Router actor in order to safely
* create/stop children.
*/
def resize(props: Props, routeeProvider: RouteeProvider)
}
@ -933,7 +1028,8 @@ case class DefaultResizer(
*/
def capacity(routees: IndexedSeq[ActorRef]): Int = {
val currentSize = routees.size
val delta = filter(pressure(routees), currentSize)
val press = pressure(routees)
val delta = filter(press, currentSize)
val proposed = currentSize + delta
if (proposed < lowerBound) delta + (lowerBound - proposed)
@ -963,7 +1059,7 @@ case class DefaultResizer(
case a: LocalActorRef
val cell = a.underlying
pressureThreshold match {
case 1 cell.mailbox.isScheduled && cell.currentMessage != null
case 1 cell.mailbox.isScheduled && cell.mailbox.hasMessages
case i if i < 1 cell.mailbox.isScheduled && cell.currentMessage != null
case threshold cell.mailbox.numberOfMessages >= threshold
}

View file

@ -1,103 +0,0 @@
package akka.serialization
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
import akka.actor.Actor
/**
* trait Serializer {
* @volatile
* var classLoader: Option[ClassLoader] = None
* def deepClone(obj: AnyRef): AnyRef = fromBinary(toBinary(obj), Some(obj.getClass))
*
* def toBinary(obj: AnyRef): Array[Byte]
* def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef
* }
*/
/**
*
* object Format {
* implicit object Default extends Serializer {
* import java.io.{ ObjectOutputStream, ByteArrayOutputStream, ObjectInputStream, ByteArrayInputStream }
* //import org.apache.commons.io.input.ClassLoaderObjectInputStream
*
* def toBinary(obj: AnyRef): Array[Byte] = {
* val bos = new ByteArrayOutputStream
* val out = new ObjectOutputStream(bos)
* out.writeObject(obj)
* out.close()
* bos.toByteArray
* }
*
* def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]], classLoader: Option[ClassLoader] = None): AnyRef = {
* val in =
* //if (classLoader.isDefined) new ClassLoaderObjectInputStream(classLoader.get, new ByteArrayInputStream(bytes)) else
* new ObjectInputStream(new ByteArrayInputStream(bytes))
* val obj = in.readObject
* in.close()
* obj
* }
*
* def identifier: Byte = 111 //Pick a number and hope no one has chosen the same :-) 0 - 16 is reserved for Akka internals
*
* }
*
* val defaultSerializerName = Default.getClass.getName
* }
*/
trait FromBinary[T <: Actor] {
def fromBinary(bytes: Array[Byte], act: T): T
}
trait ToBinary[T <: Actor] {
def toBinary(t: T): Array[Byte]
}
/**
* Type class definition for Actor Serialization.
* Client needs to implement Format[] for the respective actor.
*/
trait Format[T <: Actor] extends FromBinary[T] with ToBinary[T]
/**
* A default implementation for a stateless actor
*
* Create a Format object with the client actor as the implementation of the type class
*
* <pre>
* object BinaryFormatMyStatelessActor {
* implicit object MyStatelessActorFormat extends StatelessActorFormat[MyStatelessActor]
* }
* </pre>
*/
trait StatelessActorFormat[T <: Actor] extends Format[T] {
def fromBinary(bytes: Array[Byte], act: T) = act
def toBinary(ac: T) = Array.empty[Byte]
}
/**
* A default implementation of the type class for a Format that specifies a serializer
*
* Create a Format object with the client actor as the implementation of the type class and
* a serializer object
*
* <pre>
* object BinaryFormatMyJavaSerializableActor {
* implicit object MyJavaSerializableActorFormat extends SerializerBasedActorFormat[MyJavaSerializableActor] {
* val serializer = Serializers.Java
* }
* }
* </pre>
*/
trait SerializerBasedActorFormat[T <: Actor] extends Format[T] {
val serializer: Serializer
def fromBinary(bytes: Array[Byte], act: T) = serializer.fromBinary(bytes, Some(act.getClass)).asInstanceOf[T]
def toBinary(ac: T) = serializer.toBinary(ac)
}

View file

@ -5,12 +5,12 @@
package akka.serialization
import akka.AkkaException
import akka.util.ReflectiveAccess
import scala.util.DynamicVariable
import com.typesafe.config.Config
import akka.actor.{ Extension, ActorSystem, ExtendedActorSystem, Address }
import java.util.concurrent.ConcurrentHashMap
import akka.actor.{ Extension, ExtendedActorSystem, Address, DynamicAccess }
import akka.event.Logging
import java.util.concurrent.ConcurrentHashMap
import akka.util.NonFatal
import scala.collection.mutable.ArrayBuffer
import java.io.NotSerializableException
@ -23,19 +23,6 @@ object Serialization {
*/
type ClassSerializer = (Class[_], Serializer)
/**
* This holds a reference to the current ActorSystem (the surrounding context)
* during serialization and deserialization.
*
* If you are using Serializers yourself, outside of SerializationExtension,
* you'll need to surround the serialization/deserialization with:
*
* currentSystem.withValue(system) {
* ...code...
* }
*/
val currentSystem = new DynamicVariable[ActorSystem](null)
/**
* This holds a reference to the current transport address to be inserted
* into local actor refs during serialization.
@ -71,8 +58,9 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
* Serializes the given AnyRef/java.lang.Object according to the Serialization configuration
* to either an Array of Bytes or an Exception if one was thrown.
*/
def serialize(o: AnyRef): Either[Exception, Array[Byte]] =
try { Right(findSerializerFor(o).toBinary(o)) } catch { case e: Exception Left(e) }
def serialize(o: AnyRef): Either[Throwable, Array[Byte]] =
try Right(findSerializerFor(o).toBinary(o))
catch { case NonFatal(e) Left(e) }
/**
* Deserializes the given array of bytes using the specified serializer id,
@ -81,26 +69,18 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
*/
def deserialize(bytes: Array[Byte],
serializerId: Int,
clazz: Option[Class[_]],
classLoader: ClassLoader): Either[Exception, AnyRef] =
try {
currentSystem.withValue(system) {
Right(serializerByIdentity(serializerId).fromBinary(bytes, clazz, Some(classLoader)))
}
} catch { case e: Exception Left(e) }
clazz: Option[Class[_]]): Either[Throwable, AnyRef] =
try Right(serializerByIdentity(serializerId).fromBinary(bytes, clazz))
catch { case NonFatal(e) Left(e) }
/**
* Deserializes the given array of bytes using the specified type to look up what Serializer should be used.
* You can specify an optional ClassLoader to load the object into.
* Returns either the resulting object or an Exception if one was thrown.
*/
def deserialize(
bytes: Array[Byte],
clazz: Class[_],
classLoader: Option[ClassLoader]): Either[Exception, AnyRef] =
try {
currentSystem.withValue(system) { Right(serializerFor(clazz).fromBinary(bytes, Some(clazz), classLoader)) }
} catch { case e: Exception Left(e) }
def deserialize(bytes: Array[Byte], clazz: Class[_]): Either[Throwable, AnyRef] =
try Right(serializerFor(clazz).fromBinary(bytes, Some(clazz)))
catch { case NonFatal(e) Left(e) }
/**
* Returns the Serializer configured for the given object, returns the NullSerializer if it's null.
@ -149,10 +129,12 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
}
/**
* Tries to instantiate the specified Serializer by the FQN
* Tries to load the specified Serializer by the fully-qualified name; the actual
* loading is performed by the systems [[akka.actor.DynamicAccess]].
*/
def serializerOf(serializerFQN: String): Either[Exception, Serializer] =
ReflectiveAccess.createInstance(serializerFQN, ReflectiveAccess.noParams, ReflectiveAccess.noArgs, system.internalClassLoader)
def serializerOf(serializerFQN: String): Either[Throwable, Serializer] =
system.dynamicAccess.createInstanceFor[Serializer](serializerFQN, Seq(classOf[ExtendedActorSystem] -> system)).fold(_
system.dynamicAccess.createInstanceFor[Serializer](serializerFQN, Seq()), Right(_))
/**
* A Map of serializer from alias to implementation (class implementing akka.serialization.Serializer)
@ -169,7 +151,7 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
*/
private[akka] val bindings: Seq[ClassSerializer] = {
val configuredBindings = for ((k: String, v: String) settings.SerializationBindings if v != "none") yield {
val c = ReflectiveAccess.getClassFor(k, system.internalClassLoader).fold(throw _, identity[Class[_]])
val c = system.dynamicAccess.getClassFor(k).fold(throw _, identity[Class[_]])
(c, serializers(v))
}
sort(configuredBindings)

View file

@ -6,11 +6,31 @@ package akka.serialization
import java.io.{ ObjectOutputStream, ByteArrayOutputStream, ObjectInputStream, ByteArrayInputStream }
import akka.util.ClassLoaderObjectInputStream
import akka.actor.DynamicAccess
import akka.actor.ExtendedActorSystem
import scala.util.DynamicVariable
/**
* A Serializer represents a bimap between an object and an array of bytes representing that object
* A Serializer represents a bimap between an object and an array of bytes representing that object.
*
* Serializers are loaded using reflection during [[akka.actor.ActorSystem]]
* start-up, where two constructors are tried in order:
*
* <ul>
* <li>taking exactly one argument of type [[akka.actor.ExtendedActorSystem]];
* this should be the preferred one because all reflective loading of classes
* during deserialization should use ExtendedActorSystem.dynamicAccess (see
* [[akka.actor.DynamicAccess]]), and</li>
* <li>without arguments, which is only an option if the serializer does not
* load classes using reflection.</li>
* </ul>
*
* <b>Be sure to always use the PropertyManager for loading classes!</b> This is necessary to
* avoid strange match errors and inequalities which arise from different class loaders loading
* the same class.
*/
trait Serializer {
/**
* Completely unique value to identify this implementation of Serializer, used to optimize network traffic
* Values from 0 to 16 is reserved for Akka internal usage
@ -28,42 +48,61 @@ trait Serializer {
def includeManifest: Boolean
/**
* Deserializes the given Array of Bytes into an AnyRef
* Produces an object from an array of bytes, with an optional type-hint;
* the class should be loaded using ActorSystem.dynamicAccess.
*/
def fromBinary(bytes: Array[Byte]): AnyRef = fromBinary(bytes, None, None)
def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]]): AnyRef
/**
* Deserializes the given Array of Bytes into an AnyRef with an optional type hint
* Java API: deserialize without type hint
*/
def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]]): AnyRef = fromBinary(bytes, manifest, None)
final def fromBinary(bytes: Array[Byte]): AnyRef = fromBinary(bytes, None)
/**
* Produces an object from an array of bytes, with an optional type-hint and a classloader to load the class into
* Java API: deserialize with type hint
*/
def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]], classLoader: Option[ClassLoader]): AnyRef
final def fromBinary(bytes: Array[Byte], clazz: Class[_]): AnyRef = fromBinary(bytes, Option(clazz))
}
/**
* Java API for creating a Serializer
* Java API for creating a Serializer: make sure to include a constructor which
* takes exactly one argument of type [[akka.actor.ExtendedActorSystem]], because
* that is the preferred constructor which will be invoked when reflectively instantiating
* the JSerializer (also possible with empty constructor).
*/
abstract class JSerializer extends Serializer {
def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]] = None, classLoader: Option[ClassLoader] = None): AnyRef =
fromBinary(bytes, manifest.orNull, classLoader.orNull)
final def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]]): AnyRef =
fromBinaryJava(bytes, manifest.orNull)
/**
* This method should be overridden,
* manifest and classLoader may be null.
* This method must be implemented, manifest may be null.
*/
def fromBinary(bytes: Array[Byte], manifest: Class[_], classLoader: ClassLoader): AnyRef
protected def fromBinaryJava(bytes: Array[Byte], manifest: Class[_]): AnyRef
}
object JavaSerializer extends JavaSerializer
object NullSerializer extends NullSerializer
object JavaSerializer {
/**
* This holds a reference to the current ActorSystem (the surrounding context)
* during serialization and deserialization.
*
* If you are using Serializers yourself, outside of SerializationExtension,
* you'll need to surround the serialization/deserialization with:
*
* currentSystem.withValue(system) {
* ...code...
* }
*/
val currentSystem = new DynamicVariable[ExtendedActorSystem](null)
}
/**
* This Serializer uses standard Java Serialization
*/
class JavaSerializer extends Serializer {
class JavaSerializer(val system: ExtendedActorSystem) extends Serializer {
def includeManifest: Boolean = false
@ -77,12 +116,11 @@ class JavaSerializer extends Serializer {
bos.toByteArray
}
def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]] = None,
classLoader: Option[ClassLoader] = None): AnyRef = {
val in =
if (classLoader.isDefined) new ClassLoaderObjectInputStream(classLoader.get, new ByteArrayInputStream(bytes)) else
new ObjectInputStream(new ByteArrayInputStream(bytes))
val obj = in.readObject
def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = {
val in = new ClassLoaderObjectInputStream(system.dynamicAccess.classLoader, new ByteArrayInputStream(bytes))
val obj = JavaSerializer.currentSystem.withValue(system) {
in.readObject
}
in.close()
obj
}
@ -96,5 +134,5 @@ class NullSerializer extends Serializer {
def includeManifest: Boolean = false
def identifier = 0
def toBinary(o: AnyRef) = nullAsBytes
def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]] = None, classLoader: Option[ClassLoader] = None): AnyRef = null
def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = null
}

View file

@ -21,8 +21,18 @@ object Helpers {
if (diff > 0) 1 else if (diff < 0) -1 else 0
}
val IdentityHashComparator = new Comparator[AnyRef] {
def compare(a: AnyRef, b: AnyRef): Int = compareIdentityHash(a, b)
/**
* Create a comparator which will efficiently use `System.identityHashCode`,
* unless that happens to be the same for two non-equals objects, in which
* case the supplied real comparator is used; the comparator must be
* consistent with equals, otherwise it would not be an enhancement over
* the identityHashCode.
*/
def identityHashComparator[T <: AnyRef](comp: Comparator[T]): Comparator[T] = new Comparator[T] {
def compare(a: T, b: T): Int = compareIdentityHash(a, b) match {
case 0 if a != b comp.compare(a, b)
case x x
}
}
final val base64chars = "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789+~"

View file

@ -0,0 +1,30 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.util
/**
* Collection of internal reflection utilities which may or may not be
* available (most services specific to HotSpot, but fails gracefully).
*/
object Reflect {
/**
* This optionally holds a function which looks N levels above itself
* on the call stack and returns the `Class[_]` object for the code
* executing in that stack frame. Implemented using
* `sun.reflect.Reflection.getCallerClass` if available, None otherwise.
*
* Hint: when comparing to Thread.currentThread.getStackTrace, add two levels.
*/
val getCallerClass: Option[Int Class[_]] = {
try {
val c = Class.forName("sun.reflect.Reflection");
val m = c.getMethod("getCallerClass", Array(classOf[Int]): _*)
Some((i: Int) m.invoke(null, Array[AnyRef](i.asInstanceOf[java.lang.Integer]): _*).asInstanceOf[Class[_]])
} catch {
case NonFatal(e) None
}
}
}

View file

@ -1,126 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.util
import java.lang.reflect.InvocationTargetException
object ReflectiveAccess {
val loader = getClass.getClassLoader
val noParams: Array[Class[_]] = Array()
val noArgs: Array[AnyRef] = Array()
def createInstance[T](clazz: Class[_],
params: Array[Class[_]],
args: Array[AnyRef]): Either[Exception, T] = withErrorHandling {
assert(clazz ne null)
assert(params ne null)
assert(args ne null)
val ctor = clazz.getDeclaredConstructor(params: _*)
ctor.setAccessible(true)
Right(ctor.newInstance(args: _*).asInstanceOf[T])
}
def createInstance[T](fqn: String,
params: Array[Class[_]],
args: Array[AnyRef],
classloader: ClassLoader = loader): Either[Exception, T] = withErrorHandling {
assert(params ne null)
assert(args ne null)
getClassFor(fqn, classloader) match {
case Right(value)
val ctor = value.getDeclaredConstructor(params: _*)
ctor.setAccessible(true)
Right(ctor.newInstance(args: _*).asInstanceOf[T])
case Left(exception) Left(exception) //We could just cast this to Either[Exception, T] but it's ugly
}
}
def createInstance[T](clazz: Class[_], args: Seq[(Class[_], AnyRef)]): Either[Exception, T] =
createInstance(clazz, args.map(_._1).toArray, args.map(_._2).toArray)
def createInstance[T](fqcn: String, args: Seq[(Class[_], AnyRef)], classloader: ClassLoader): Either[Exception, T] =
createInstance(fqcn, args.map(_._1).toArray, args.map(_._2).toArray, classloader)
def createInstance[T](fqcn: String, args: Seq[(Class[_], AnyRef)]): Either[Exception, T] =
createInstance(fqcn, args.map(_._1).toArray, args.map(_._2).toArray, loader)
//Obtains a reference to fqn.MODULE$
def getObjectFor[T](fqn: String, classloader: ClassLoader = loader): Either[Exception, T] = try {
getClassFor(fqn, classloader) match {
case Right(value)
val instance = value.getDeclaredField("MODULE$")
instance.setAccessible(true)
val obj = instance.get(null)
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 {
case e: Exception
Left(e)
}
def getClassFor[T](fqn: String, classloader: ClassLoader = loader): Either[Exception, Class[T]] = try {
assert(fqn ne null)
// First, use the specified CL
val first = try {
Right(classloader.loadClass(fqn).asInstanceOf[Class[T]])
} catch {
case c: ClassNotFoundException Left(c)
}
if (first.isRight) first
else {
// Second option is to use the ContextClassLoader
val second = try {
Right(Thread.currentThread.getContextClassLoader.loadClass(fqn).asInstanceOf[Class[T]])
} catch {
case c: ClassNotFoundException Left(c)
}
if (second.isRight) second
else {
val third = try {
if (classloader ne loader) Right(loader.loadClass(fqn).asInstanceOf[Class[T]]) else Left(null) //Horrid
} catch {
case c: ClassNotFoundException Left(c)
}
if (third.isRight) third
else {
try {
Right(Class.forName(fqn).asInstanceOf[Class[T]]) // Last option is Class.forName
} catch {
case c: ClassNotFoundException Left(c)
}
}
}
}
} catch {
case e: Exception Left(e)
}
/**
* Caught exception is returned as Left(exception).
* Unwraps `InvocationTargetException` if its getTargetException is an `Exception`.
* Other `Throwable`, such as `Error` is thrown.
*/
@inline
private final def withErrorHandling[T](body: Either[Exception, T]): Either[Exception, T] = {
try {
body
} catch {
case e: InvocationTargetException e.getTargetException match {
case t: Exception Left(t)
case t throw t
}
case e: Exception
Left(e)
}
}
}

View file

@ -9,7 +9,7 @@ class AccrualFailureDetectorSpec extends AkkaSpec("""
""") {
"An AccrualFailureDetector" must {
val conn = Address("akka", "", Some("localhost"), Some(2552))
val conn = Address("akka", "", "localhost", 2552)
"mark node as available after a series of successful heartbeats" in {
val fd = new AccrualFailureDetector(system)

View file

@ -18,28 +18,46 @@ The multi-JVM testing is an sbt plugin that you can find here:
http://github.com/typesafehub/sbt-multi-jvm
You can add it as a plugin by adding the following to your plugins/build.sbt::
You can add it as a plugin by adding the following to your project/plugins.sbt::
resolvers += Classpaths.typesafeResolver
addSbtPlugin("com.typesafe.sbtmultijvm" % "sbt-multi-jvm" % "0.1.9")
You can then add multi-JVM testing to a project by including the ``MultiJvm``
You can then add multi-JVM testing to ``project/Build.scala`` by including the ``MultiJvm``
settings and config. For example, here is how the akka-remote project adds
multi-JVM testing::
import MultiJvmPlugin.{ MultiJvm, extraOptions }
import sbt._
import Keys._
import com.typesafe.sbtmultijvm.MultiJvmPlugin
import com.typesafe.sbtmultijvm.MultiJvmPlugin.{ MultiJvm, extraOptions }
lazy val cluster = Project(
id = "akka-remote",
base = file("akka-remote"),
settings = defaultSettings ++ MultiJvmPlugin.settings ++ Seq(
extraOptions in MultiJvm <<= (sourceDirectory in MultiJvm) { src =>
(name: String) => (src ** (name + ".conf")).get.headOption.map("-Dconfig.file=" + _.absolutePath).toSeq
},
test in Test <<= (test in Test) dependsOn (test in MultiJvm)
object AkkaBuild extends Build {
lazy val remote = Project(
id = "akka-remote",
base = file("akka-remote"),
settings = defaultSettings ++ MultiJvmPlugin.settings ++ Seq(
extraOptions in MultiJvm <<= (sourceDirectory in MultiJvm) { src =>
(name: String) => (src ** (name + ".conf")).get.headOption.map("-Dconfig.file=" + _.absolutePath).toSeq
},
test in Test <<= (test in Test) dependsOn (test in MultiJvm)
)
) configs (MultiJvm)
lazy val buildSettings = Defaults.defaultSettings ++ Seq(
organization := "com.typesafe.akka",
version := "2.0-SNAPSHOT",
scalaVersion := "2.9.1",
crossPaths := false
)
) configs (MultiJvm)
lazy val defaultSettings = buildSettings ++ Seq(
resolvers += "Typesafe Repo" at "http://repo.typesafe.com/typesafe/releases/"
)
}
You can specify JVM options for the forked JVMs::
@ -87,8 +105,8 @@ options after the test names and ``--``. For example:
Creating application tests
==========================
The tests are discovered, and combined, through a naming convention. A test is
named with the following pattern:
The tests are discovered, and combined, through a naming convention. MultiJvm tests are
located in ``src/multi-jvm/scala`` directory. A test is named with the following pattern:
.. code-block:: none

View file

@ -75,6 +75,24 @@ akka-testkit
.. literalinclude:: ../../akka-testkit/src/main/resources/reference.conf
:language: none
akka-transactor
~~~~~~~~~~~~~~~
.. literalinclude:: ../../akka-transactor/src/main/resources/reference.conf
:language: none
akka-agent
~~~~~~~~~~
.. literalinclude:: ../../akka-agent/src/main/resources/reference.conf
:language: none
akka-zeromq
~~~~~~~~~~~
.. literalinclude:: ../../akka-zeromq/src/main/resources/reference.conf
:language: none
akka-beanstalk-mailbox
~~~~~~~~~~~~~~~~~~~~~~

View file

@ -45,6 +45,18 @@ To prevent visibility and reordering problems on actors, Akka guarantees the fol
Both rules only apply for the same actor instance and are not valid if different actors are used.
Futures and the Java Memory Model
---------------------------------
The completion of a Future "happens before" the invocation of any callbacks registered to it are executed.
We recommend not to close over non-final fields (final in Java and val in Scala), and if you *do* choose to close over
non-final fields, they must be marked *volatile* in order for the current value of the field to be visible to the callback.
If you close over a reference, you must also ensure that the instance that is referred to is thread safe.
We highly recommend staying away from objects that use locking, since it can introduce performance problems and in the worst case, deadlocks.
Such are the perils of synchronized.
STM and the Java Memory Model
-----------------------------
Akka's Software Transactional Memory (STM) also provides a "happens before" rule:

View file

@ -78,7 +78,7 @@ public class FutureDocTestBase {
ActorRef actor = system.actorOf(new Props(MyActor.class));
String msg = "hello";
//#ask-blocking
Timeout timeout = system.settings().ActorTimeout();
Timeout timeout = new Timeout(Duration.parse("5 seconds"));
Future<Object> future = Patterns.ask(actor, msg, timeout);
String result = (String) Await.result(future, timeout.duration());
//#ask-blocking
@ -196,19 +196,17 @@ public class FutureDocTestBase {
Iterable<Future<Integer>> listOfFutureInts = source;
// now we have a Future[Iterable[Integer]]
Future<Iterable<Integer>> futureListOfInts =
sequence(listOfFutureInts, system.dispatcher());
Future<Iterable<Integer>> futureListOfInts = sequence(listOfFutureInts, system.dispatcher());
// Find the sum of the odd numbers
Future<Long> futureSum = futureListOfInts.map(
new Mapper<Iterable<Integer>, Long>() {
public Long apply(Iterable<Integer> ints) {
long sum = 0;
for (Integer i : ints)
sum += i;
return sum;
}
});
Future<Long> futureSum = futureListOfInts.map(new Mapper<Iterable<Integer>, Long>() {
public Long apply(Iterable<Integer> ints) {
long sum = 0;
for (Integer i : ints)
sum += i;
return sum;
}
});
long result = Await.result(futureSum, Duration.create(1, SECONDS));
//#sequence
@ -221,20 +219,18 @@ public class FutureDocTestBase {
//Just a sequence of Strings
Iterable<String> listStrings = Arrays.asList("a", "b", "c");
Future<Iterable<String>> futureResult = 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();
}
}, system.dispatcher());
}
}, system.dispatcher());
Future<Iterable<String>> futureResult = 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();
}
}, system.dispatcher());
}
}, system.dispatcher());
//Returns the sequence of strings as upper case
Iterable<String> result =
Await.result(futureResult, Duration.create(1, SECONDS));
Iterable<String> result = Await.result(futureResult, Duration.create(1, SECONDS));
assertEquals(Arrays.asList("A", "B", "C"), result);
//#traverse
}
@ -250,12 +246,11 @@ public class FutureDocTestBase {
Iterable<Future<String>> futures = source;
//Start value is the empty string
Future<String> resultFuture = fold("", futures,
new Function2<String, String, String>() {
public String apply(String r, String t) {
return r + t; //Just concatenate
}
}, system.dispatcher());
Future<String> resultFuture = fold("", futures, new Function2<String, String, String>() {
public String apply(String r, String t) {
return r + t; //Just concatenate
}
}, system.dispatcher());
String result = Await.result(resultFuture, Duration.create(1, SECONDS));
//#fold
@ -272,12 +267,11 @@ public class FutureDocTestBase {
//A sequence of Futures, in this case Strings
Iterable<Future<String>> futures = source;
Future<Object> resultFuture = reduce(futures,
new Function2<Object, String, Object>() {
public Object apply(Object r, String t) {
return r + t; //Just concatenate
}
}, system.dispatcher());
Future<Object> resultFuture = reduce(futures, new Function2<Object, String, Object>() {
public Object apply(Object r, String t) {
return r + t; //Just concatenate
}
}, system.dispatcher());
Object result = Await.result(resultFuture, Duration.create(1, SECONDS));
//#reduce
@ -285,32 +279,35 @@ public class FutureDocTestBase {
assertEquals("ab", result);
}
@Test public void useSuccessfulAndFailed() {
@Test
public void useSuccessfulAndFailed() {
//#successful
Future<String> future = Futures.successful("Yay!", system.dispatcher());
//#successful
//#failed
Future<String> otherFuture =
Futures.failed(new IllegalArgumentException("Bang!"), system.dispatcher());
Future<String> otherFuture = Futures.failed(new IllegalArgumentException("Bang!"), system.dispatcher());
//#failed
Object result = Await.result(future, Duration.create(1, SECONDS));
assertEquals("Yay!",result);
assertEquals("Yay!", result);
Throwable result2 = Await.result(otherFuture.failed(), Duration.create(1, SECONDS));
assertEquals("Bang!",result2.getMessage());
assertEquals("Bang!", result2.getMessage());
}
@Test public void useFilter() {
//#filter
@Test
public void useFilter() {
//#filter
Future<Integer> future1 = Futures.successful(4, system.dispatcher());
Future<Integer> successfulFilter =
future1.filter(new Filter<Integer>() {
public boolean filter(Integer i) { return i % 2 == 0; }
});
Future<Integer> successfulFilter = future1.filter(new Filter<Integer>() {
public boolean filter(Integer i) {
return i % 2 == 0;
}
});
Future<Integer> failedFilter =
future1.filter(new Filter<Integer>() {
public boolean filter(Integer i) { return i % 2 != 0; }
});
Future<Integer> failedFilter = future1.filter(new Filter<Integer>() {
public boolean filter(Integer i) {
return i % 2 != 0;
}
});
//When filter fails, the returned Future will be failed with a scala.MatchError
//#filter
}
@ -323,137 +320,140 @@ public class FutureDocTestBase {
}
@Test public void useAndThen() {
@Test
public void useAndThen() {
//#and-then
Future<String> future1 = Futures.successful("value", system.dispatcher()).
andThen(new OnComplete<String>() {
public void onComplete(Throwable failure, String result) {
if (failure != null) sendToIssueTracker(failure);
}
Future<String> future1 = Futures.successful("value", system.dispatcher()).andThen(new OnComplete<String>() {
public void onComplete(Throwable failure, String result) {
if (failure != null)
sendToIssueTracker(failure);
}
}).andThen(new OnComplete<String>() {
public void onComplete(Throwable failure, String result) {
if (result != null) sendToTheInternetz(result);
}
public void onComplete(Throwable failure, String result) {
if (result != null)
sendToTheInternetz(result);
}
});
//#and-then
}
@Test public void useRecover() {
@Test
public void useRecover() {
//#recover
Future<Integer> future = future(new Callable<Integer>() {
public Integer call() {
return 1 / 0;
}
}, system.dispatcher()).recover(new Recover<Integer>() {
public Integer recover(Throwable problem) throws Throwable {
if (problem instanceof ArithmeticException) return 0;
else throw problem;
}
public Integer recover(Throwable problem) throws Throwable {
if (problem instanceof ArithmeticException)
return 0;
else
throw problem;
}
});
int result = Await.result(future, Duration.create(1, SECONDS));
assertEquals(result, 0);
//#recover
}
@Test public void useTryRecover() {
@Test
public void useTryRecover() {
//#try-recover
Future<Integer> future = future(new Callable<Integer>() {
public Integer call() {
return 1 / 0;
}
}, system.dispatcher()).recoverWith(new Recover<Future<Integer>>() {
public Future<Integer> recover(Throwable problem) throws Throwable {
if (problem instanceof ArithmeticException) {
return future(new Callable<Integer>() {
public Integer call() {
return 0;
}
}, system.dispatcher());
public Future<Integer> recover(Throwable problem) throws Throwable {
if (problem instanceof ArithmeticException) {
return future(new Callable<Integer>() {
public Integer call() {
return 0;
}
else throw problem;
}
}, system.dispatcher());
} else
throw problem;
}
});
int result = Await.result(future, Duration.create(1, SECONDS));
assertEquals(result, 0);
//#try-recover
}
@Test public void useOnSuccessOnFailureAndOnComplete() {
{
@Test
public void useOnSuccessOnFailureAndOnComplete() {
{
Future<String> future = Futures.successful("foo", system.dispatcher());
//#onSuccess
future.onSuccess(new OnSuccess<String>() {
public void onSuccess(String result) {
if ("bar" == result) {
//Do something if it resulted in "bar"
} else {
//Do something if it was some other String
}
public void onSuccess(String result) {
if ("bar" == result) {
//Do something if it resulted in "bar"
} else {
//Do something if it was some other String
}
}
});
//#onSuccess
}
{
Future<String> future =
Futures.failed(new IllegalStateException("OHNOES"), system.dispatcher());
//#onFailure
future.onFailure( new OnFailure() {
}
{
Future<String> future = Futures.failed(new IllegalStateException("OHNOES"), system.dispatcher());
//#onFailure
future.onFailure(new OnFailure() {
public void onFailure(Throwable failure) {
if (failure instanceof IllegalStateException) {
//Do something if it was this particular failure
} else {
//Do something if it was some other failure
}
if (failure instanceof IllegalStateException) {
//Do something if it was this particular failure
} else {
//Do something if it was some other failure
}
}
});
//#onFailure
}
{
Future<String> future = Futures.successful("foo", system.dispatcher());
//#onComplete
future.onComplete(new OnComplete<String>() {
public void onComplete(Throwable failure, String result) {
if (failure != null) {
//We got a failure, handle it here
} else {
// We got a result, do something with it
}
}
});
//#onComplete
}
}
{
Future<String> future = Futures.successful("foo", system.dispatcher());
//#onComplete
future.onComplete(new OnComplete<String>() {
public void onComplete(Throwable failure, String result) {
if (failure != null) {
//We got a failure, handle it here
} else {
// We got a result, do something with it
}
}
});
//#onComplete
}
}
@Test public void useOrAndZip(){
@Test
public void useOrAndZip() {
{
//#zip
Future<String> future1 = Futures.successful("foo", system.dispatcher());
Future<String> future2 = Futures.successful("bar", system.dispatcher());
Future<String> future3 =
future1.zip(future2).map(new Mapper<scala.Tuple2<String,String>, String>() {
public String apply(scala.Tuple2<String,String> zipped) {
return zipped._1() + " " + zipped._2();
//#zip
Future<String> future1 = Futures.successful("foo", system.dispatcher());
Future<String> future2 = Futures.successful("bar", system.dispatcher());
Future<String> future3 = future1.zip(future2).map(new Mapper<scala.Tuple2<String, String>, String>() {
public String apply(scala.Tuple2<String, String> zipped) {
return zipped._1() + " " + zipped._2();
}
});
});
String result = Await.result(future3, Duration.create(1, SECONDS));
assertEquals("foo bar", result);
//#zip
String result = Await.result(future3, Duration.create(1, SECONDS));
assertEquals("foo bar", result);
//#zip
}
{
//#fallback-to
Future<String> future1 =
Futures.failed(new IllegalStateException("OHNOES1"), system.dispatcher());
Future<String> future2 =
Futures.failed(new IllegalStateException("OHNOES2"), system.dispatcher());
Future<String> future3 =
Futures.successful("bar", system.dispatcher());
Future<String> future4 =
future1.fallbackTo(future2).fallbackTo(future3); // Will have "bar" in this case
String result = Await.result(future4, Duration.create(1, SECONDS));
assertEquals("bar", result);
//#fallback-to
//#fallback-to
Future<String> future1 = Futures.failed(new IllegalStateException("OHNOES1"), system.dispatcher());
Future<String> future2 = Futures.failed(new IllegalStateException("OHNOES2"), system.dispatcher());
Future<String> future3 = Futures.successful("bar", system.dispatcher());
Future<String> future4 = future1.fallbackTo(future2).fallbackTo(future3); // Will have "bar" in this case
String result = Await.result(future4, Duration.create(1, SECONDS));
assertEquals("bar", result);
//#fallback-to
}
}

View file

@ -17,6 +17,7 @@ import akka.util.Duration;
import akka.util.Timeout;
import akka.dispatch.Await;
import akka.dispatch.Future;
import akka.dispatch.Dispatchers;
import akka.testkit.AkkaSpec;
import com.typesafe.config.ConfigFactory;
import static akka.pattern.Patterns.ask;
@ -39,6 +40,19 @@ public class CustomRouterDocTestBase {
system.shutdown();
}
public static class MyActor extends UntypedActor {
@Override public void onReceive(Object o) {}
}
@Test
public void demonstrateDispatchers() {
//#dispatchers
final ActorRef router = system.actorOf(new Props(MyActor.class)
.withRouter(new RoundRobinRouter(5).withDispatcher("head")) // head router runs on "head" dispatcher
.withDispatcher("workers")); // MyActor workers run on "workers" dispatcher
//#dispatchers
}
//#crTest
@Test
public void countVotesAsIntendedNotAsInFlorida() {
@ -106,6 +120,10 @@ public class CustomRouterDocTestBase {
//#crRouter
public static class VoteCountRouter extends CustomRouterConfig {
@Override public String routerDispatcher() {
return Dispatchers.DefaultDispatcherId();
}
//#crRoute
@Override
public CustomRoute createCustomRoute(Props props, RouteeProvider routeeProvider) {

View file

@ -54,9 +54,9 @@ public class ParentActor extends UntypedActor {
ActorRef scatterGatherFirstCompletedRouter = getContext().actorOf(
new Props(FibonacciActor.class).withRouter(new ScatterGatherFirstCompletedRouter(5, Duration
.parse("2 seconds"))), "router");
Timeout timeout = getContext().system().settings().ActorTimeout();
Future<Object> futureResult = akka.pattern.Patterns.ask(
scatterGatherFirstCompletedRouter, new FibonacciActor.FibonacciNumber(10), timeout);
Timeout timeout = new Timeout(Duration.parse("5 seconds"));
Future<Object> futureResult = akka.pattern.Patterns.ask(scatterGatherFirstCompletedRouter,
new FibonacciActor.FibonacciNumber(10), timeout);
int result = (Integer) Await.result(futureResult, timeout.duration());
//#scatterGatherFirstCompletedRouter
System.out.println(String.format("The result of calculating Fibonacci for 10 is %d", result));

View file

@ -3,6 +3,7 @@
*/
package akka.docs.serialization;
import akka.japi.Option;
import akka.serialization.JSerializer;
import akka.serialization.Serialization;
import akka.serialization.SerializationExtension;
@ -43,10 +44,8 @@ public class SerializationDocTestBase {
// "fromBinary" deserializes the given array,
// using the type hint (if any, see "includeManifest" above)
// into the optionally provided classLoader.
@Override public Object fromBinary(byte[] bytes,
Class clazz,
ClassLoader classLoader) {
@Override public Object fromBinaryJava(byte[] bytes,
Class<?> clazz) {
// Put your code that deserializes here
//#...
return null;

View file

@ -0,0 +1,8 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.docs.zeromq
import org.scalatest.junit.JUnitSuite
class ZeromqDocTest extends ZeromqDocTestBase with JUnitSuite

View file

@ -0,0 +1,286 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.docs.zeromq;
//#pub-socket
import akka.zeromq.Bind;
import akka.zeromq.ZeroMQExtension;
//#pub-socket
//#sub-socket
import akka.zeromq.Connect;
import akka.zeromq.Listener;
import akka.zeromq.Subscribe;
//#sub-socket
//#unsub-topic-socket
import akka.zeromq.Unsubscribe;
//#unsub-topic-socket
//#pub-topic
import akka.zeromq.Frame;
import akka.zeromq.ZMQMessage;
//#pub-topic
import akka.zeromq.HighWatermark;
import akka.zeromq.SocketOption;
import akka.zeromq.ZeroMQVersion;
//#health
import akka.actor.ActorRef;
import akka.actor.UntypedActor;
import akka.actor.Props;
import akka.event.Logging;
import akka.event.LoggingAdapter;
import akka.util.Duration;
import akka.serialization.SerializationExtension;
import akka.serialization.Serialization;
import java.io.Serializable;
import java.lang.management.ManagementFactory;
//#health
import com.typesafe.config.ConfigFactory;
import java.lang.management.MemoryMXBean;
import java.lang.management.MemoryUsage;
import java.lang.management.OperatingSystemMXBean;
import java.util.concurrent.TimeUnit;
import java.util.Date;
import java.text.SimpleDateFormat;
import akka.actor.ActorSystem;
import akka.testkit.AkkaSpec;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.Assume;
import akka.zeromq.SocketType;
public class ZeromqDocTestBase {
ActorSystem system;
@Before
public void setUp() {
system = ActorSystem.create("ZeromqDocTest",
ConfigFactory.parseString("akka.loglevel=INFO").withFallback(AkkaSpec.testConf()));
}
@After
public void tearDown() {
system.shutdown();
}
@Test
public void demonstrateCreateSocket() {
Assume.assumeTrue(checkZeroMQInstallation());
//#pub-socket
ActorRef pubSocket = ZeroMQExtension.get(system).newPubSocket(new Bind("tcp://127.0.0.1:1233"));
//#pub-socket
//#sub-socket
ActorRef listener = system.actorOf(new Props(ListenerActor.class));
ActorRef subSocket = ZeroMQExtension.get(system).newSubSocket(new Connect("tcp://127.0.0.1:1233"),
new Listener(listener), Subscribe.all());
//#sub-socket
//#sub-topic-socket
ActorRef subTopicSocket = ZeroMQExtension.get(system).newSubSocket(new Connect("tcp://127.0.0.1:1233"),
new Listener(listener), new Subscribe("foo.bar"));
//#sub-topic-socket
//#unsub-topic-socket
subTopicSocket.tell(new Unsubscribe("foo.bar"));
//#unsub-topic-socket
byte[] payload = new byte[0];
//#pub-topic
pubSocket.tell(new ZMQMessage(new Frame("foo.bar"), new Frame(payload)));
//#pub-topic
//#high-watermark
ActorRef highWatermarkSocket = ZeroMQExtension.get(system).newRouterSocket(
new SocketOption[] { new Listener(listener), new Bind("tcp://127.0.0.1:1233"), new HighWatermark(50000) });
//#high-watermark
}
@Test
public void demonstratePubSub() throws Exception {
Assume.assumeTrue(checkZeroMQInstallation());
//#health2
system.actorOf(new Props(HealthProbe.class), "health");
//#health2
//#logger2
system.actorOf(new Props(Logger.class), "logger");
//#logger2
//#alerter2
system.actorOf(new Props(HeapAlerter.class), "alerter");
//#alerter2
// Let it run for a while to see some output.
// Don't do like this in real tests, this is only doc demonstration.
Thread.sleep(3000L);
}
private boolean checkZeroMQInstallation() {
try {
ZeroMQVersion v = ZeroMQExtension.get(system).version();
return (v.major() == 2 && v.minor() == 1);
} catch (LinkageError e) {
return false;
}
}
//#listener-actor
public static class ListenerActor extends UntypedActor {
public void onReceive(Object message) throws Exception {
//...
}
}
//#listener-actor
//#health
public static final Object TICK = "TICK";
public static class Heap implements Serializable {
public final long timestamp;
public final long used;
public final long max;
public Heap(long timestamp, long used, long max) {
this.timestamp = timestamp;
this.used = used;
this.max = max;
}
}
public static class Load implements Serializable {
public final long timestamp;
public final double loadAverage;
public Load(long timestamp, double loadAverage) {
this.timestamp = timestamp;
this.loadAverage = loadAverage;
}
}
public static class HealthProbe extends UntypedActor {
ActorRef pubSocket = ZeroMQExtension.get(getContext().system()).newPubSocket(new Bind("tcp://127.0.0.1:1237"));
MemoryMXBean memory = ManagementFactory.getMemoryMXBean();
OperatingSystemMXBean os = ManagementFactory.getOperatingSystemMXBean();
Serialization ser = SerializationExtension.get(getContext().system());
@Override
public void preStart() {
getContext().system().scheduler()
.schedule(Duration.parse("1 second"), Duration.parse("1 second"), getSelf(), TICK);
}
@Override
public void postRestart(Throwable reason) {
// don't call preStart, only schedule once
}
@Override
public void onReceive(Object message) {
if (message.equals(TICK)) {
MemoryUsage currentHeap = memory.getHeapMemoryUsage();
long timestamp = System.currentTimeMillis();
// use akka SerializationExtension to convert to bytes
byte[] heapPayload = ser.serializerFor(Heap.class).toBinary(
new Heap(timestamp, currentHeap.getUsed(), currentHeap.getMax()));
// the first frame is the topic, second is the message
pubSocket.tell(new ZMQMessage(new Frame("health.heap"), new Frame(heapPayload)));
// use akka SerializationExtension to convert to bytes
byte[] loadPayload = ser.serializerFor(Load.class).toBinary(new Load(timestamp, os.getSystemLoadAverage()));
// the first frame is the topic, second is the message
pubSocket.tell(new ZMQMessage(new Frame("health.load"), new Frame(loadPayload)));
} else {
unhandled(message);
}
}
}
//#health
//#logger
public static class Logger extends UntypedActor {
ActorRef subSocket = ZeroMQExtension.get(getContext().system()).newSubSocket(new Connect("tcp://127.0.0.1:1237"),
new Listener(getSelf()), new Subscribe("health"));
Serialization ser = SerializationExtension.get(getContext().system());
SimpleDateFormat timestampFormat = new SimpleDateFormat("HH:mm:ss.SSS");
LoggingAdapter log = Logging.getLogger(getContext().system(), this);
@Override
public void onReceive(Object message) {
if (message instanceof ZMQMessage) {
ZMQMessage m = (ZMQMessage) message;
// the first frame is the topic, second is the message
if (m.firstFrameAsString().equals("health.heap")) {
Heap heap = (Heap) ser.serializerFor(Heap.class).fromBinary(m.payload(1));
log.info("Used heap {} bytes, at {}", heap.used, timestampFormat.format(new Date(heap.timestamp)));
} else if (m.firstFrameAsString().equals("health.load")) {
Load load = (Load) ser.serializerFor(Load.class).fromBinary(m.payload(1));
log.info("Load average {}, at {}", load.loadAverage, timestampFormat.format(new Date(load.timestamp)));
}
} else {
unhandled(message);
}
}
}
//#logger
//#alerter
public static class HeapAlerter extends UntypedActor {
ActorRef subSocket = ZeroMQExtension.get(getContext().system()).newSubSocket(new Connect("tcp://127.0.0.1:1237"),
new Listener(getSelf()), new Subscribe("health.heap"));
Serialization ser = SerializationExtension.get(getContext().system());
LoggingAdapter log = Logging.getLogger(getContext().system(), this);
int count = 0;
@Override
public void onReceive(Object message) {
if (message instanceof ZMQMessage) {
ZMQMessage m = (ZMQMessage) message;
// the first frame is the topic, second is the message
if (m.firstFrameAsString().equals("health.heap")) {
Heap heap = (Heap) ser.serializerFor(Heap.class).fromBinary(m.payload(1));
if (((double) heap.used / heap.max) > 0.9) {
count += 1;
} else {
count = 0;
}
if (count > 10) {
log.warning("Need more memory, using {} %", (100.0 * heap.used / heap.max));
}
}
} else {
unhandled(message);
}
}
}
//#alerter
}

View file

@ -23,3 +23,4 @@ Java API
transactors
fsm
extending-akka
zeromq

View file

@ -8,11 +8,6 @@ Routing (Java)
.. contents:: :local:
Akka-core includes some building blocks to build more complex message flow handlers, they are listed and explained below:
Router
------
A Router is an actor that routes incoming messages to outbound actors.
The router routes the messages sent to it to its underlying actors called 'routees'.
@ -249,6 +244,16 @@ This is an example of how to programatically create a resizable router:
*It is also worth pointing out that if you define the ``router`` in the configuration file then this value
will be used instead of any programmatically sent parameters.*
.. note::
Resizing is triggered by sending messages to the actor pool, but it is not
completed synchronously; instead a message is sent to the “head”
:class:`Router` to perform the size change. Thus you cannot rely on resizing
to instantaneously create new workers when all others are busy, because the
message just sent will be queued to the mailbox of a busy actor. To remedy
this, configure the pool to use a balancing dispatcher, see `Configuring
Dispatchers`_ for more information.
Custom Router
^^^^^^^^^^^^^
@ -312,3 +317,23 @@ A router with dynamically resizable number of routees is implemented by providin
in ``resizer`` method of the ``RouterConfig``. See ``akka.routing.DefaultResizer`` for inspiration
of how to write your own resize strategy.
Configuring Dispatchers
^^^^^^^^^^^^^^^^^^^^^^^
The dispatcher for created children of the router will be taken from
:class:`Props` as described in :ref:`dispatchers-java`. For a dynamic pool it
makes sense to configure the :class:`BalancingDispatcher` if the precise
routing is not so important (i.e. no consistent hashing or round-robin is
required); this enables newly created routees to pick up work immediately by
stealing it from their siblings.
The “head” router, of couse, cannot run on the same balancing dispatcher,
because it does not process the same messages, hence this special actor does
not use the dispatcher configured in :class:`Props`, but takes the
``routerDispatcher`` from the :class:`RouterConfig` instead, which defaults to
the actor systems default dispatcher. All standard routers allow setting this
property in their constructor or factory method, custom routers have to
implement the method in a suitable way.
.. includecode:: code/akka/docs/jrouting/CustomRouterDocTestBase.java#dispatchers

View file

@ -103,3 +103,15 @@ which is done by extending ``akka.serialization.JSerializer``, like this:
Then you only need to fill in the blanks, bind it to a name in your :ref:`configuration` and then
list which classes that should be serialized using it.
A Word About Java Serialization
===============================
When using Java serialization without employing the :class:`JavaSerializer` for
the task, you must make sure to supply a valid :class:`ExtendedActorSystem` in
the dynamic variable ``JavaSerializer.currentSystem``. This is used when
reading in the representation of an :class:`ActorRef` for turning the string
representation into a real reference. :class:`DynamicVariable` is a
thread-local variable, so be sure to have it set while deserializing anything
which might contain actor references.

98
akka-docs/java/zeromq.rst Normal file
View file

@ -0,0 +1,98 @@
.. _zeromq-java:
###############
ZeroMQ (Java)
###############
.. sidebar:: Contents
.. contents:: :local:
Akka provides a ZeroMQ module which abstracts a ZeroMQ connection and therefore allows interaction between Akka actors to take place over ZeroMQ connections. The messages can be of a proprietary format or they can be defined using Protobuf. The socket actor is fault-tolerant by default and when you use the newSocket method to create new sockets it will properly reinitialize the socket.
ZeroMQ is very opinionated when it comes to multi-threading so configuration option `akka.zeromq.socket-dispatcher` always needs to be configured to a PinnedDispatcher, because the actual ZeroMQ socket can only be accessed by the thread that created it.
The ZeroMQ module for Akka is written against an API introduced in JZMQ, which uses JNI to interact with the native ZeroMQ library. Instead of using JZMQ, the module uses ZeroMQ binding for Scala that uses the native ZeroMQ library through JNA. In other words, the only native library that this module requires is the native ZeroMQ library.
The benefit of the scala library is that you don't need to compile and manage native dependencies at the cost of some runtime performance. The scala-bindings are compatible with the JNI bindings so they are a drop-in replacement, in case you really need to get that extra bit of performance out.
Connection
==========
ZeroMQ supports multiple connectivity patterns, each aimed to meet a different set of requirements. Currently, this module supports publisher-subscriber connections and connections based on dealers and routers. For connecting or accepting connections, a socket must be created.
Sockets are always created using the ``akka.zeromq.ZeroMQExtension``, for example:
.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#pub-socket
Above examples will create a ZeroMQ Publisher socket that is Bound to the port 1233 on localhost.
Similarly you can create a subscription socket, with a listener, that subscribes to all messages from the publisher using:
.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#sub-socket
.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#listener-actor
The following sub-sections describe the supported connection patterns and how they can be used in an Akka environment. However, for a comprehensive discussion of connection patterns, please refer to `ZeroMQ -- The Guide <http://zguide.zeromq.org/page:all>`_.
Publisher-subscriber connection
-------------------------------
In a publisher-subscriber (pub-sub) connection, the publisher accepts one or more subscribers. Each subscriber shall
subscribe to one or more topics, whereas the publisher publishes messages to a set of topics. Also, a subscriber can
subscribe to all available topics. In an Akka environment, pub-sub connections shall be used when an actor sends messages
to one or more actors that do not interact with the actor that sent the message.
When you're using zeromq pub/sub you should be aware that it needs multicast - check your cloud - to work properly and that the filtering of events for topics happens client side, so all events are always broadcasted to every subscriber.
An actor is subscribed to a topic as follows:
.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#sub-topic-socket
It is a prefix match so it is subscribed to all topics starting with ``foo.bar``. Note that if the given string is empty or
``Subscribe.all()`` is used, the actor is subscribed to all topics.
To unsubscribe from a topic you do the following:
.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#unsub-topic-socket
To publish messages to a topic you must use two Frames with the topic in the first frame.
.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#pub-topic
Pub-Sub in Action
^^^^^^^^^^^^^^^^^
The following example illustrates one publisher with two subscribers.
The publisher monitors current heap usage and system load and periodically publishes ``Heap`` events on the ``"health.heap"`` topic
and ``Load`` events on the ``"health.load"`` topic.
.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#health
.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#health2
Let's add one subscriber that logs the information. It subscribes to all topics starting with ``"health"``, i.e. both ``Heap`` and
``Load`` events.
.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#logger
.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#logger2
Another subscriber keep track of used heap and warns if too much heap is used. It only subscribes to ``Heap`` events.
.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#alerter
.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#alerter2
Router-Dealer connection
------------------------
While Pub/Sub is nice the real advantage of zeromq is that it is a "lego-box" for reliable messaging. And because there are so many integrations the multi-language support is fantastic.
When you're using ZeroMQ to integrate many systems you'll probably need to build your own ZeroMQ devices. This is where the router and dealer socket types come in handy.
With those socket types you can build your own reliable pub sub broker that uses TCP/IP and does publisher side filtering of events.
To create a Router socket that has a high watermark configured, you would do:
.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#high-watermark
The akka-zeromq module accepts most if not all the available configuration options for a zeromq socket.

View file

@ -0,0 +1,6 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
//#global
object Global extends com.typesafe.play.mini.Setup(akka.docs.http.PlayMiniApplication)
//#global

View file

@ -0,0 +1,128 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.docs.http
//#imports
import com.typesafe.play.mini.{ POST, GET, Path, Application }
import play.api.mvc.{ Action, AsyncResult }
import play.api.mvc.Results._
import play.api.libs.concurrent._
import play.api.data._
import play.api.data.Forms._
import akka.pattern.ask
import akka.util.Timeout
import akka.util.duration._
import akka.actor.{ ActorSystem, Props, Actor }
import scala.collection.mutable.{ Map MutableMap }
//#imports
//#playMiniDefinition
object PlayMiniApplication extends Application {
//#playMiniDefinition
private val system = ActorSystem("sample")
//#regexURI
private final val StatementPattern = """/account/statement/(\w+)""".r
//#regexURI
private lazy val accountActor = system.actorOf(Props[AccountActor])
implicit val timeout = Timeout(1000 milliseconds)
//#route
def route = {
//#routeLogic
//#simpleGET
case GET(Path("/ping")) Action {
Ok("Pong @ " + System.currentTimeMillis)
}
//#simpleGET
//#regexGET
case GET(Path(StatementPattern(accountId))) Action {
AsyncResult {
//#innerRegexGET
(accountActor ask Status(accountId)).mapTo[Int].asPromise.map { r
if (r >= 0) Ok("Account total: " + r)
else BadRequest("Unknown account: " + accountId)
}
//#innerRegexGET
}
}
//#regexGET
//#asyncDepositPOST
case POST(Path("/account/deposit")) Action { implicit request
//#formAsyncDepositPOST
val (accountId, amount) = commonForm.bindFromRequest.get
//#formAsyncDepositPOST
AsyncResult {
(accountActor ask Deposit(accountId, amount)).mapTo[Int].asPromise.map { r Ok("Updated account total: " + r) }
}
}
//#asyncDepositPOST
//#asyncWithdrawPOST
case POST(Path("/account/withdraw")) Action { implicit request
val (accountId, amount) = commonForm.bindFromRequest.get
AsyncResult {
(accountActor ask Withdraw(accountId, amount)).mapTo[Int].asPromise.map { r
if (r >= 0) Ok("Updated account total: " + r)
else BadRequest("Unknown account or insufficient funds. Get your act together.")
}
}
}
//#asyncWithdrawPOST
//#routeLogic
}
//#route
//#form
val commonForm = Form(
tuple(
"accountId" -> nonEmptyText,
"amount" -> number(min = 1)))
//#form
}
//#cases
case class Status(accountId: String)
case class Deposit(accountId: String, amount: Int)
case class Withdraw(accountId: String, amount: Int)
//#cases
//#actor
class AccountActor extends Actor {
var accounts = MutableMap[String, Int]()
//#receive
def receive = {
//#senderBang
case Status(accountId) sender ! accounts.getOrElse(accountId, -1)
//#senderBang
case Deposit(accountId, amount) sender ! deposit(accountId, amount)
case Withdraw(accountId, amount) sender ! withdraw(accountId, amount)
}
//#receive
private def deposit(accountId: String, amount: Int): Int = {
accounts.get(accountId) match {
case Some(value)
val newValue = value + amount
accounts += accountId -> newValue
newValue
case None
accounts += accountId -> amount
amount
}
}
private def withdraw(accountId: String, amount: Int): Int = {
accounts.get(accountId) match {
case Some(value)
if (value < amount) -1
else {
val newValue = value - amount
accounts += accountId -> newValue
newValue
}
case None -1
}
}
//#actor
}

View file

@ -7,8 +7,194 @@ HTTP
.. contents:: :local:
Play!
-----
Play2-mini
----------
The Akka team recommends the `Play2-mini <https://github.com/typesafehub/play2-mini>`_ framework when building RESTful
service applications that integrates with Akka. It provides a REST API on top of `Play2 <https://github.com/playframework/Play20/>`_.
Akka will recommend using `Play! Mini <https://github.com/typesafehub/play2-mini>`_
Getting started
---------------
First you must make your application aware of play-mini.
In SBT you just have to add the following to your _libraryDependencies_::
libraryDependencies += "com.typesafe" %% "play-mini" % "2.0-RC1-SNAPSHOT"
Sample Application
------------------
To illustrate how easy it is to wire a RESTful service with Akka we will use a sample application.
The aim of the application is to show how to use play-mini and Akka in combination. Do not put too much
attention on the actual business logic itself, which is a extremely simple bank application, as building a bank
application is a little more complex than what's shown in the sample...
The application should support the following URL commands:
- GET /ping - returns a Pong message with the time of the server (used to see if the application is up and running)
- GET /account/statement/{accountId} - returns the account statement
- POST /account/deposit - deposits money to an account (and creates a new one if it's not already existing)
- POST /account/withdraw - withdraws money from an account
Error messages will be returned in case of any misuse of the application, e.g. withdrawing more money than an
account has etc.
Getting started
---------------
To build a play-mini application you first have to make your object extend com.typesafe.play.mini.Application:
.. includecode:: code/akka/docs/http/PlayMiniApplication.scala
:include: playMiniDefinition
The next step is to implement the mandatory method ``route``:
.. includecode:: code/akka/docs/http/PlayMiniApplication.scala
:include: route
:exclude: routeLogic
It is inside the ``route`` method that all the magic happens.
In the sections below we will show how to set up play-mini to handle both GET and POST HTTP calls.
Simple GET
----------
We start off by creating the simplest method we can - a "ping" method:
.. includecode:: code/akka/docs/http/PlayMiniApplication.scala
:include: simpleGET
As you can see in the section above play-mini uses Scala's wonderful pattern matching.
In the snippet we instruct play-mini to reply to all HTTP GET calls with the URI "/ping".
The ``Action`` returned comes from Play! and you can find more information about it `here <https://github.com/playframework/Play20/wiki/ScalaActions>`_.
.. _Advanced-GET:
Advanced GET
------------
Let's try something more advanced, retrieving parameters from the URI and also make an asynchronous call to an actor:
.. includecode:: code/akka/docs/http/PlayMiniApplication.scala
:include: regexGET
The regular expression looks like this:
.. includecode:: code/akka/docs/http/PlayMiniApplication.scala
:include: regexURI
In the snippets above we extract a URI parameter with the help of a simple regular expression and then we pass this
parameter on to the underlying actor system. As you can see ``AsyncResult`` is being used. This means that the call to
the actor will be performed asynchronously, i.e. no blocking.
The asynchronous call to the actor is being done with a ``ask``, e.g.::
(accountActor ask Status(accountId))
The actor that receives the message returns the result by using a standard *sender !*
as can be seen here:
.. includecode:: code/akka/docs/http/PlayMiniApplication.scala
:include: senderBang
When the result is returned to the calling code we use some mapping code in Play to convert a Akka future to a Play future.
This is shown in this code:
.. includecode:: code/akka/docs/http/PlayMiniApplication.scala
:include: innerRegexGET
In this snippet we check the result to decide what type of response we want to send to the calling client.
Using HTTP POST
---------------
Okay, in the sections above we have shown you how to use play-mini for HTTP GET calls. Let's move on to when the user
posts values to the application.
.. includecode:: code/akka/docs/http/PlayMiniApplication.scala
:include: asyncDepositPOST
As you can see the structure is almost the same as for the :ref:`Advanced-GET`. The difference is that we make the
``request`` parameter ``implicit`` and also that the following line of code is used to extract parameters from the POST.
.. includecode:: code/akka/docs/http/PlayMiniApplication.scala
:include: formAsyncDepositPOST
The code snippet used to map the call to parameters looks like this:
.. includecode:: code/akka/docs/http/PlayMiniApplication.scala
:include: form
Apart from the mapping of parameters the call to the actor looks is done the same as in :ref:`Advanced-GET`.
The Complete Code Sample
------------------------
Below is the complete application in all its beauty.
Global.scala (<yourApp>/src/main/scala/Global.scala):
.. includecode:: code/Global.scala
PlayMiniApplication.scala (<yourApp>/src/main/scala/akka/docs/http/PlayMiniApplication.scala):
.. includecode:: code/akka/docs/http/PlayMiniApplication.scala
Build.scala (<yourApp>/project/Build.scala):
.. code-block:: scala
import sbt._
import Keys._
object PlayMiniApplicationBuild extends Build {
lazy val root = Project(id = "play-mini-application", base = file("."), settings = Project.defaultSettings).settings(
libraryDependencies += "com.typesafe" %% "play-mini" % "2.0-RC1-SNAPSHOT",
mainClass in (Compile, run) := Some("play.core.server.NettyServer"))
}
Running the Application
-----------------------
Firstly, start up the application by opening a command terminal and type::
> sbt
> run
Now you should see something similar to this in your terminal window::
[info] Running play.core.server.NettyServer
Play server process ID is 2523
[info] play - Application started (Prod)
[info] play - Listening for HTTP on port 9000...
In this example we will use the awesome `cURL <http://en.wikipedia.org/wiki/CURL>`_ command to interact with the application.
Fire up a command terminal and try the application out::
First we check the status of a couple of accounts:
> curl http://localhost:9000/account/statement/TheDudesAccount
Unknown account: TheDudesAccount
> curl http://localhost:9000/account/statement/MrLebowskisAccount
Unknown account: MrLebowskisAccount
Now deposit some money to the accounts:
> curl -d "accountId=TheDudesAccount&amount=1000" http://localhost:9000/account/deposit
Updated account total: 1000
> curl -d "accountId=MrLebowskisAccount&amount=500" http://localhost:9000/account/deposit
Updated account total: 500
Next thing is to check the status of the account:
> curl http://localhost:9000/account/statement/TheDudesAccount
Account total: 1000
> curl http://localhost:9000/account/statement/MrLebowskisAccount
Account total: 500
Fair enough, let's try to withdraw some cash shall we:
> curl -d "accountId=TheDudesAccount&amount=999" http://localhost:9000/account/withdraw
Updated account total: 1
> curl -d "accountId=MrLebowskisAccount&amount=999" http://localhost:9000/account/withdraw
Unknown account or insufficient funds. Get your act together.
> curl -d "accountId=MrLebowskisAccount&amount=500" http://localhost:9000/account/withdraw
Updated account total: 0
Yeah, it works!
Now we leave it to the astute reader of this document to take advantage of the power of play-mini and Akka.

View file

@ -11,6 +11,7 @@ import akka.actor.Props
import akka.actor.Status.Failure
import akka.dispatch.Future
import akka.dispatch.Await
import akka.util.Timeout
import akka.util.duration._
import akka.dispatch.Promise
import java.lang.IllegalStateException
@ -46,8 +47,10 @@ class FutureDocSpec extends AkkaSpec {
//#ask-blocking
import akka.dispatch.Await
import akka.pattern.ask
import akka.util.Timeout
import akka.util.duration._
implicit val timeout = system.settings.ActorTimeout
implicit val timeout = Timeout(5 seconds)
val future = actor ? msg // enabled by the ask import
val result = Await.result(future, timeout.duration).asInstanceOf[String]
//#ask-blocking
@ -57,7 +60,7 @@ class FutureDocSpec extends AkkaSpec {
"demonstrate usage of mapTo" in {
val actor = system.actorOf(Props[MyActor])
val msg = "hello"
implicit val timeout = system.settings.ActorTimeout
implicit val timeout = Timeout(5 seconds)
//#map-to
import akka.dispatch.Future
import akka.pattern.ask
@ -164,7 +167,7 @@ class FutureDocSpec extends AkkaSpec {
val actor3 = system.actorOf(Props[MyActor])
val msg1 = 1
val msg2 = 2
implicit val timeout = system.settings.ActorTimeout
implicit val timeout = Timeout(5 seconds)
import akka.dispatch.Await
import akka.pattern.ask
//#composing-wrong
@ -188,7 +191,7 @@ class FutureDocSpec extends AkkaSpec {
val actor3 = system.actorOf(Props[MyActor])
val msg1 = 1
val msg2 = 2
implicit val timeout = system.settings.ActorTimeout
implicit val timeout = Timeout(5 seconds)
import akka.dispatch.Await
import akka.pattern.ask
//#composing
@ -208,7 +211,7 @@ class FutureDocSpec extends AkkaSpec {
}
"demonstrate usage of sequence with actors" in {
implicit val timeout = system.settings.ActorTimeout
implicit val timeout = Timeout(5 seconds)
val oddActor = system.actorOf(Props[OddActor])
//#sequence-ask
// oddActor returns odd numbers sequentially from 1 as a List[Future[Int]]
@ -256,7 +259,7 @@ class FutureDocSpec extends AkkaSpec {
}
"demonstrate usage of recover" in {
implicit val timeout = system.settings.ActorTimeout
implicit val timeout = Timeout(5 seconds)
val actor = system.actorOf(Props[MyActor])
val msg1 = -1
//#recover
@ -268,7 +271,7 @@ class FutureDocSpec extends AkkaSpec {
}
"demonstrate usage of recoverWith" in {
implicit val timeout = system.settings.ActorTimeout
implicit val timeout = Timeout(5 seconds)
val actor = system.actorOf(Props[MyActor])
val msg1 = -1
//#try-recover

View file

@ -28,7 +28,7 @@ class RemoteDeploymentDocSpec extends AkkaSpec("""
import RemoteDeploymentDocSpec._
val other = ActorSystem("remote", system.settings.config)
val address = other.asInstanceOf[ExtendedActorSystem].provider.getExternalAddressFor(Address("akka", "s", Some("host"), Some(1))).get
val address = other.asInstanceOf[ExtendedActorSystem].provider.getExternalAddressFor(Address("akka", "s", "host", 1)).get
override def atTermination() { other.shutdown() }

View file

@ -0,0 +1,29 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.docs.routing
import RouterDocSpec.MyActor
import akka.actor.{ Props, Actor }
import akka.testkit.AkkaSpec
import akka.routing.RoundRobinRouter
object RouterDocSpec {
class MyActor extends Actor {
def receive = {
case _
}
}
}
class RouterDocSpec extends AkkaSpec {
import RouterDocSpec._
//#dispatchers
val router = system.actorOf(Props[MyActor]
.withRouter(RoundRobinRouter(5, routerDispatcher = "router")) // head will run on "router" dispatcher
.withDispatcher("workers")) // MyActor workers will run on "workers" dispatcher
//#dispatchers
}

View file

@ -7,6 +7,7 @@ import akka.routing.{ ScatterGatherFirstCompletedRouter, BroadcastRouter, Random
import annotation.tailrec
import akka.actor.{ Props, Actor }
import akka.util.duration._
import akka.util.Timeout
import akka.dispatch.Await
import akka.pattern.ask
import akka.routing.SmallestMailboxRouter
@ -80,7 +81,7 @@ class ParentActor extends Actor {
val scatterGatherFirstCompletedRouter = context.actorOf(
Props[FibonacciActor].withRouter(ScatterGatherFirstCompletedRouter(
nrOfInstances = 5, within = 2 seconds)), "router")
implicit val timeout = context.system.settings.ActorTimeout
implicit val timeout = Timeout(5 seconds)
val futureResult = scatterGatherFirstCompletedRouter ? FibonacciNumber(10)
val result = Await.result(futureResult, timeout.duration)
//#scatterGatherFirstCompletedRouter

View file

@ -35,8 +35,7 @@ class MyOwnSerializer extends Serializer {
// using the type hint (if any, see "includeManifest" above)
// into the optionally provided classLoader.
def fromBinary(bytes: Array[Byte],
clazz: Option[Class[_]],
classLoader: Option[ClassLoader] = None): AnyRef = {
clazz: Option[Class[_]]): AnyRef = {
// Put your code that deserializes here
//#...
null
@ -143,9 +142,7 @@ class SerializationDocSpec extends AkkaSpec {
val bytes = serializer.toBinary(original)
// Turn it back into an object
val back = serializer.fromBinary(bytes,
manifest = None,
classLoader = None)
val back = serializer.fromBinary(bytes, manifest = None)
// Voilá!
back must equal(original)

View file

@ -0,0 +1,195 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.docs.zeromq
import akka.actor.Actor
import akka.actor.Props
import akka.util.duration._
import akka.testkit._
import akka.zeromq.ZeroMQVersion
import akka.zeromq.ZeroMQExtension
import java.text.SimpleDateFormat
import java.util.Date
import akka.zeromq.SocketType
import akka.zeromq.Bind
object ZeromqDocSpec {
//#health
import akka.zeromq._
import akka.actor.Actor
import akka.actor.Props
import akka.actor.ActorLogging
import akka.serialization.SerializationExtension
import java.lang.management.ManagementFactory
case object Tick
case class Heap(timestamp: Long, used: Long, max: Long)
case class Load(timestamp: Long, loadAverage: Double)
class HealthProbe extends Actor {
val pubSocket = context.system.newSocket(SocketType.Pub, Bind("tcp://127.0.0.1:1235"))
val memory = ManagementFactory.getMemoryMXBean
val os = ManagementFactory.getOperatingSystemMXBean
val ser = SerializationExtension(context.system)
override def preStart() {
context.system.scheduler.schedule(1 second, 1 second, self, Tick)
}
override def postRestart(reason: Throwable) {
// don't call preStart, only schedule once
}
def receive: Receive = {
case Tick
val currentHeap = memory.getHeapMemoryUsage
val timestamp = System.currentTimeMillis
// use akka SerializationExtension to convert to bytes
val heapPayload = ser.serialize(Heap(timestamp, currentHeap.getUsed, currentHeap.getMax)).fold(throw _, identity)
// the first frame is the topic, second is the message
pubSocket ! ZMQMessage(Seq(Frame("health.heap"), Frame(heapPayload)))
// use akka SerializationExtension to convert to bytes
val loadPayload = ser.serialize(Load(timestamp, os.getSystemLoadAverage)).fold(throw _, identity)
// the first frame is the topic, second is the message
pubSocket ! ZMQMessage(Seq(Frame("health.load"), Frame(loadPayload)))
}
}
//#health
//#logger
class Logger extends Actor with ActorLogging {
context.system.newSocket(SocketType.Sub, Listener(self), Connect("tcp://127.0.0.1:1235"), Subscribe("health"))
val ser = SerializationExtension(context.system)
val timestampFormat = new SimpleDateFormat("HH:mm:ss.SSS")
def receive = {
// the first frame is the topic, second is the message
case m: ZMQMessage if m.firstFrameAsString == "health.heap"
ser.deserialize(m.payload(1), classOf[Heap]) match {
case Right(Heap(timestamp, used, max))
log.info("Used heap {} bytes, at {}", used, timestampFormat.format(new Date(timestamp)))
case Left(e) throw e
}
case m: ZMQMessage if m.firstFrameAsString == "health.load"
ser.deserialize(m.payload(1), classOf[Load]) match {
case Right(Load(timestamp, loadAverage))
log.info("Load average {}, at {}", loadAverage, timestampFormat.format(new Date(timestamp)))
case Left(e) throw e
}
}
}
//#logger
//#alerter
class HeapAlerter extends Actor with ActorLogging {
context.system.newSocket(SocketType.Sub, Listener(self), Connect("tcp://127.0.0.1:1235"), Subscribe("health.heap"))
val ser = SerializationExtension(context.system)
var count = 0
def receive = {
// the first frame is the topic, second is the message
case m: ZMQMessage if m.firstFrameAsString == "health.heap"
ser.deserialize(m.payload(1), classOf[Heap]) match {
case Right(Heap(timestamp, used, max))
if ((used.toDouble / max) > 0.9) count += 1
else count = 0
if (count > 10) log.warning("Need more memory, using {} %", (100.0 * used / max))
case Left(e) throw e
}
}
}
//#alerter
}
class ZeromqDocSpec extends AkkaSpec("akka.loglevel=INFO") {
import ZeromqDocSpec._
"demonstrate how to create socket" in {
checkZeroMQInstallation()
//#pub-socket
import akka.zeromq.ZeroMQExtension
val pubSocket = ZeroMQExtension(system).newSocket(SocketType.Pub, Bind("tcp://127.0.0.1:1234"))
//#pub-socket
//#pub-socket2
import akka.zeromq._
val pubSocket2 = system.newSocket(SocketType.Pub, Bind("tcp://127.0.0.1:1234"))
//#pub-socket2
//#sub-socket
import akka.zeromq._
val listener = system.actorOf(Props(new Actor {
def receive: Receive = {
case Connecting //...
case m: ZMQMessage //...
case _ //...
}
}))
val subSocket = system.newSocket(SocketType.Sub, Listener(listener), Connect("tcp://127.0.0.1:1234"), SubscribeAll)
//#sub-socket
//#sub-topic-socket
val subTopicSocket = system.newSocket(SocketType.Sub, Listener(listener), Connect("tcp://127.0.0.1:1234"), Subscribe("foo.bar"))
//#sub-topic-socket
//#unsub-topic-socket
subTopicSocket ! Unsubscribe("foo.bar")
//#unsub-topic-socket
val payload = Array.empty[Byte]
//#pub-topic
pubSocket ! ZMQMessage(Seq(Frame("foo.bar"), Frame(payload)))
//#pub-topic
//#high-watermark
val highWatermarkSocket = system.newSocket(
SocketType.Router,
Listener(listener),
Bind("tcp://127.0.0.1:1234"),
HighWatermark(50000))
//#high-watermark
}
"demonstrate pub-sub" in {
checkZeroMQInstallation()
//#health
system.actorOf(Props[HealthProbe], name = "health")
//#health
//#logger
system.actorOf(Props[Logger], name = "logger")
//#logger
//#alerter
system.actorOf(Props[HeapAlerter], name = "alerter")
//#alerter
// Let it run for a while to see some output.
// Don't do like this in real tests, this is only doc demonstration.
3.seconds.sleep()
}
def checkZeroMQInstallation() = try {
ZeroMQExtension(system).version match {
case ZeroMQVersion(2, 1, _) Unit
case version pending
}
} catch {
case e: LinkageError pending
}
}

View file

@ -8,11 +8,6 @@ Routing (Scala)
.. contents:: :local:
Akka-core includes some building blocks to build more complex message flow handlers, they are listed and explained below:
Router
------
A Router is an actor that routes incoming messages to outbound actors.
The router routes the messages sent to it to its underlying actors called 'routees'.
@ -250,6 +245,16 @@ This is an example of how to programatically create a resizable router:
*It is also worth pointing out that if you define the ``router`` in the configuration file then this value
will be used instead of any programmatically sent parameters.*
.. note::
Resizing is triggered by sending messages to the actor pool, but it is not
completed synchronously; instead a message is sent to the “head”
:class:`Router` to perform the size change. Thus you cannot rely on resizing
to instantaneously create new workers when all others are busy, because the
message just sent will be queued to the mailbox of a busy actor. To remedy
this, configure the pool to use a balancing dispatcher, see `Configuring
Dispatchers`_ for more information.
Custom Router
^^^^^^^^^^^^^
@ -311,3 +316,23 @@ A router with dynamically resizable number of routees is implemented by providin
in ``resizer`` method of the ``RouterConfig``. See ``akka.routing.DefaultResizer`` for inspiration
of how to write your own resize strategy.
Configuring Dispatchers
^^^^^^^^^^^^^^^^^^^^^^^
The dispatcher for created children of the router will be taken from
:class:`Props` as described in :ref:`dispatchers-scala`. For a dynamic pool it
makes sense to configure the :class:`BalancingDispatcher` if the precise
routing is not so important (i.e. no consistent hashing or round-robin is
required); this enables newly created routees to pick up work immediately by
stealing it from their siblings.
The “head” router, of couse, cannot run on the same balancing dispatcher,
because it does not process the same messages, hence this special actor does
not use the dispatcher configured in :class:`Props`, but takes the
``routerDispatcher`` from the :class:`RouterConfig` instead, which defaults to
the actor systems default dispatcher. All standard routers allow setting this
property in their constructor or factory method, custom routers have to
implement the method in a suitable way.
.. includecode:: code/akka/docs/routing/RouterDocSpec.scala#dispatchers

View file

@ -101,3 +101,15 @@ First you need to create a class definition of your ``Serializer`` like so:
Then you only need to fill in the blanks, bind it to a name in your :ref:`configuration` and then
list which classes that should be serialized using it.
A Word About Java Serialization
===============================
When using Java serialization without employing the :class:`JavaSerializer` for
the task, you must make sure to supply a valid :class:`ExtendedActorSystem` in
the dynamic variable ``JavaSerializer.currentSystem``. This is used when
reading in the representation of an :class:`ActorRef` for turning the string
representation into a real reference. :class:`DynamicVariable` is a
thread-local variable, so be sure to have it set while deserializing anything
which might contain actor references.

View file

@ -1,8 +1,10 @@
.. _zeromq-module:
.. _zeromq-scala:
################
ZeroMQ (Scala)
################
ZeroMQ
======
.. sidebar:: Contents
@ -16,79 +18,72 @@ The ZeroMQ module for Akka is written against an API introduced in JZMQ, which u
The benefit of the scala library is that you don't need to compile and manage native dependencies at the cost of some runtime performance. The scala-bindings are compatible with the JNI bindings so they are a drop-in replacement, in case you really need to get that extra bit of performance out.
Connection
----------
==========
ZeroMQ supports multiple connectivity patterns, each aimed to meet a different set of requirements. Currently, this module supports publisher-subscriber connections and connections based on dealers and routers. For connecting or accepting connections, a socket must be created. Sockets are always created using ``akka.zeromq.ZeroMQ.newSocket``, for example:
ZeroMQ supports multiple connectivity patterns, each aimed to meet a different set of requirements. Currently, this module supports publisher-subscriber connections and connections based on dealers and routers. For connecting or accepting connections, a socket must be created.
Sockets are always created using the ``akka.zeromq.ZeroMQExtension``, for example:
.. code-block:: scala
.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#pub-socket
import akka.zeromq._
val socket = system.zeromq.newSocket(SocketType.Pub, Bind("tcp://127.0.0.1:1234"))
or by importing the ``akka.zeromq._`` package to make newSocket method available on system, via an implicit conversion.
will create a ZeroMQ Publisher socket that is Bound to the port 1234 on localhost.
Importing the akka.zeromq._ package ensures that the implicit zeromq method is available.
Similarly you can create a subscription socket, that subscribes to all messages from the publisher using:
.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#pub-socket2
.. code-block:: scala
val socket = system.zeromq.newSocket(SocketType.Sub, Connect("tcp://127.0.0.1:1234"), SubscribeAll)
Above examples will create a ZeroMQ Publisher socket that is Bound to the port 1234 on localhost.
Also, a socket may be created with a listener that handles received messages as well as notifications:
Similarly you can create a subscription socket, with a listener, that subscribes to all messages from the publisher using:
.. code-block:: scala
val listener = system.actorOf(Props(new Actor {
def receive: Receive = {
case Connecting => ...
case _ => ...
}
}))
val socket = system.zeromq.newSocket(SocketType.Router, Listener(listener), Connect("tcp://localhost:1234"))
.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#sub-socket
The following sub-sections describe the supported connection patterns and how they can be used in an Akka environment. However, for a comprehensive discussion of connection patterns, please refer to `ZeroMQ -- The Guide <http://zguide.zeromq.org/page:all>`_.
Publisher-subscriber connection
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
-------------------------------
In a publisher-subscriber (pub-sub) connection, the publisher accepts one or more subscribers. Each subscriber shall subscribe to one or more topics, whereas the publisher publishes messages to a set of topics. Also, a subscriber can subscribe to all available topics.
In a publisher-subscriber (pub-sub) connection, the publisher accepts one or more subscribers. Each subscriber shall
subscribe to one or more topics, whereas the publisher publishes messages to a set of topics. Also, a subscriber can
subscribe to all available topics. In an Akka environment, pub-sub connections shall be used when an actor sends messages
to one or more actors that do not interact with the actor that sent the message.
When you're using zeromq pub/sub you should be aware that it needs multicast - check your cloud - to work properly and that the filtering of events for topics happens client side, so all events are always broadcasted to every subscriber.
An actor is subscribed to a topic as follows:
.. code-block:: scala
.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#sub-topic-socket
val socket = system.zeromq.newSocket(SocketType.Sub, Listener(listener), Connect("tcp://localhost:1234"), Subscribe("the-topic"))
It is a prefix match so it is subscribed to all topics starting with ``foo.bar``. Note that if the given string is empty or
``SubscribeAll`` is used, the actor is subscribed to all topics.
Note that if the given string is empty (see below), the actor is subscribed to all topics. To unsubscribe from a topic you do the following:
To unsubscribe from a topic you do the following:
.. code-block:: scala
.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#unsub-topic-socket
socket ! Unsubscribe("SomeTopic1")
To publish messages to a topic you must use two Frames with the topic in the first frame.
In an Akka environment, pub-sub connections shall be used when an actor sends messages to one or more actors that do not interact with the actor that sent the message. The following piece of code creates a publisher actor, binds the socket, and sends a message to be published:
.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#pub-topic
.. code-block:: scala
Pub-Sub in Action
^^^^^^^^^^^^^^^^^
import akka.zeromq._
val socket = system.zeromq.newSocket(SocketType.Pub, Bind("tcp://127.0.0.1:1234"))
socket ! Send("hello".getBytes)
The following example illustrates one publisher with two subscribers.
In the following code, the subscriber is configured to receive messages for all topics:
The publisher monitors current heap usage and system load and periodically publishes ``Heap`` events on the ``"health.heap"`` topic
and ``Load`` events on the ``"health.load"`` topic.
.. code-block:: scala
.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#health
import akka.zeromq._
val listener = system.actorOf(Props(new Actor {
def receive: Receive = {
case Connecting => ...
case _ => ...
}
}))
val socket = system.zeromq.newSocket(SocketType.Sub, Listener(listener), Connect("tcp://127.0.0.1:1234"), SubscribeAll)
Let's add one subscriber that logs the information. It subscribes to all topics starting with ``"health"``, i.e. both ``Heap`` and
``Load`` events.
.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#logger
Another subscriber keep track of used heap and warns if too much heap is used. It only subscribes to ``Heap`` events.
.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#alerter
Router-Dealer connection
^^^^^^^^^^^^^^^^^^^^^^^^
------------------------
While Pub/Sub is nice the real advantage of zeromq is that it is a "lego-box" for reliable messaging. And because there are so many integrations the multi-language support is fantastic.
When you're using ZeroMQ to integrate many systems you'll probably need to build your own ZeroMQ devices. This is where the router and dealer socket types come in handy.
@ -96,19 +91,6 @@ With those socket types you can build your own reliable pub sub broker that uses
To create a Router socket that has a high watermark configured, you would do:
.. code-block:: scala
import akka.zeromq._
val listener = system.actorOf(Props(new Actor {
def receive: Receive = {
case Connecting => ...
case _ => ...
}
}))
val socket = system.zeromq.newSocket(
SocketType.Router,
Listener(listener),
Bind("tcp://127.0.0.1:1234"),
HWM(50000))
.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#high-watermark
The akka-zeromq module accepts most if not all the available configuration options for a zeromq socket.

View file

@ -22,7 +22,7 @@ class BeanstalkBasedMailboxType(config: Config) extends MailboxType {
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class BeanstalkBasedMailbox(val owner: ActorContext) extends DurableMailbox(owner) with DurableMessageSerialization {
class BeanstalkBasedMailbox(_owner: ActorContext) extends DurableMailbox(_owner) with DurableMessageSerialization {
private val settings = BeanstalkBasedMailboxExtension(owner.system)
private val messageSubmitDelaySeconds = settings.MessageSubmitDelay.toSeconds.toInt

View file

@ -17,7 +17,7 @@ class FileBasedMailboxType(config: Config) extends MailboxType {
override def create(owner: ActorContext) = new FileBasedMailbox(owner)
}
class FileBasedMailbox(val owner: ActorContext) extends DurableMailbox(owner) with DurableMessageSerialization {
class FileBasedMailbox(_owner: ActorContext) extends DurableMailbox(_owner) with DurableMessageSerialization {
val log = Logging(system, "FileBasedMailbox")

View file

@ -12,7 +12,7 @@ private[akka] object DurableExecutableMailboxConfig {
val Name = "[\\.\\/\\$\\s]".r
}
abstract class DurableMailbox(owner: ActorContext) extends CustomMailbox(owner) with DefaultSystemMessageQueue {
abstract class DurableMailbox(val owner: ActorContext) extends CustomMailbox(owner) with DefaultSystemMessageQueue {
import DurableExecutableMailboxConfig._
def system: ExtendedActorSystem = owner.system.asInstanceOf[ExtendedActorSystem]
@ -22,15 +22,13 @@ abstract class DurableMailbox(owner: ActorContext) extends CustomMailbox(owner)
}
trait DurableMessageSerialization {
def owner: ActorContext
trait DurableMessageSerialization { this: DurableMailbox
def serialize(durableMessage: Envelope): Array[Byte] = {
def serializeActorRef(ref: ActorRef): ActorRefProtocol = ActorRefProtocol.newBuilder.setPath(ref.path.toString).build
val message = MessageSerializer.serialize(owner.system, durableMessage.message.asInstanceOf[AnyRef])
val message = MessageSerializer.serialize(system, durableMessage.message.asInstanceOf[AnyRef])
val builder = RemoteMessageProtocol.newBuilder
.setMessage(message)
.setRecipient(serializeActorRef(owner.self))
@ -41,13 +39,13 @@ trait DurableMessageSerialization {
def deserialize(bytes: Array[Byte]): Envelope = {
def deserializeActorRef(refProtocol: ActorRefProtocol): ActorRef = owner.system.actorFor(refProtocol.getPath)
def deserializeActorRef(refProtocol: ActorRefProtocol): ActorRef = system.actorFor(refProtocol.getPath)
val durableMessage = RemoteMessageProtocol.parseFrom(bytes)
val message = MessageSerializer.deserialize(owner.system, durableMessage.getMessage, getClass.getClassLoader)
val message = MessageSerializer.deserialize(system, durableMessage.getMessage)
val sender = deserializeActorRef(durableMessage.getSender)
new Envelope(message, sender)(owner.system)
new Envelope(message, sender)(system)
}
}

View file

@ -65,7 +65,7 @@ class BSONSerializableMailbox(system: ExtendedActorSystem) extends SerializableB
val doc = deserializer.decodeAndFetch(in).asInstanceOf[BSONDocument]
system.log.debug("Deserializing a durable message from MongoDB: {}", doc)
val msgData = MessageProtocol.parseFrom(doc.as[org.bson.types.Binary]("message").getData)
val msg = MessageSerializer.deserialize(system, msgData, system.internalClassLoader)
val msg = MessageSerializer.deserialize(system, msgData)
val ownerPath = doc.as[String]("ownerPath")
val senderPath = doc.as[String]("senderPath")
val sender = system.actorFor(senderPath)

View file

@ -32,7 +32,7 @@ class MongoBasedMailboxType(config: Config) extends MailboxType {
*
* @author <a href="http://evilmonkeylabs.com">Brendan W. McAdams</a>
*/
class MongoBasedMailbox(val owner: ActorContext) extends DurableMailbox(owner) {
class MongoBasedMailbox(_owner: ActorContext) extends DurableMailbox(_owner) {
// this implicit object provides the context for reading/writing things as MongoDurableMessage
implicit val mailboxBSONSer = new BSONSerializableMailbox(system)
implicit val safeWrite = WriteConcern.Safe // TODO - Replica Safe when appropriate!

View file

@ -19,7 +19,7 @@ class RedisBasedMailboxType(config: Config) extends MailboxType {
override def create(owner: ActorContext) = new RedisBasedMailbox(owner)
}
class RedisBasedMailbox(val owner: ActorContext) extends DurableMailbox(owner) with DurableMessageSerialization {
class RedisBasedMailbox(_owner: ActorContext) extends DurableMailbox(_owner) with DurableMessageSerialization {
private val settings = RedisBasedMailboxExtension(owner.system)

View file

@ -20,7 +20,7 @@ class ZooKeeperBasedMailboxType(config: Config) extends MailboxType {
override def create(owner: ActorContext) = new ZooKeeperBasedMailbox(owner)
}
class ZooKeeperBasedMailbox(val owner: ActorContext) extends DurableMailbox(owner) with DurableMessageSerialization {
class ZooKeeperBasedMailbox(_owner: ActorContext) extends DurableMailbox(_owner) with DurableMessageSerialization {
private val settings = ZooKeeperBasedMailboxExtension(owner.system)
val queueNode = "/queues"

View file

@ -6,32 +6,25 @@ package akka.remote
import akka.remote.RemoteProtocol._
import com.google.protobuf.ByteString
import akka.actor.ActorSystem
import akka.actor.ExtendedActorSystem
import akka.serialization.SerializationExtension
import akka.util.ReflectiveAccess
object MessageSerializer {
def deserialize(system: ActorSystem, messageProtocol: MessageProtocol, classLoader: ClassLoader): AnyRef = {
val clazz = if (messageProtocol.hasMessageManifest) {
Option(ReflectiveAccess.getClassFor[AnyRef](
messageProtocol.getMessageManifest.toStringUtf8,
classLoader) match {
case Left(e) throw e
case Right(r) r
})
} else None
SerializationExtension(system).deserialize(
messageProtocol.getMessage.toByteArray,
messageProtocol.getSerializerId,
clazz,
classLoader) match {
def deserialize(system: ExtendedActorSystem, messageProtocol: MessageProtocol): AnyRef = {
val clazz =
if (messageProtocol.hasMessageManifest) {
system.dynamicAccess.getClassFor[AnyRef](messageProtocol.getMessageManifest.toStringUtf8)
.fold(throw _, Some(_))
} else None
SerializationExtension(system)
.deserialize(messageProtocol.getMessage.toByteArray, messageProtocol.getSerializerId, clazz) match {
case Left(e) throw e
case Right(r) r
}
}
def serialize(system: ActorSystem, message: AnyRef): MessageProtocol = {
def serialize(system: ExtendedActorSystem, message: AnyRef): MessageProtocol = {
val s = SerializationExtension(system)
val serializer = s.findSerializerFor(message)
val builder = MessageProtocol.newBuilder

View file

@ -12,7 +12,6 @@ import akka.event.EventStream
import akka.config.ConfigurationException
import java.util.concurrent.{ TimeoutException }
import com.typesafe.config.Config
import akka.util.ReflectiveAccess
import akka.serialization.Serialization
import akka.serialization.SerializationExtension
@ -28,11 +27,11 @@ class RemoteActorRefProvider(
val settings: ActorSystem.Settings,
val eventStream: EventStream,
val scheduler: Scheduler,
val classloader: ClassLoader) extends ActorRefProvider {
val dynamicAccess: DynamicAccess) extends ActorRefProvider {
val remoteSettings = new RemoteSettings(settings.config, systemName)
val deployer = new RemoteDeployer(settings, classloader)
val deployer = new RemoteDeployer(settings, dynamicAccess)
private val local = new LocalActorRefProvider(systemName, settings, eventStream, scheduler, deployer)
@ -84,7 +83,7 @@ class RemoteActorRefProvider(
classOf[ActorSystemImpl] -> system,
classOf[RemoteActorRefProvider] -> this)
ReflectiveAccess.createInstance[RemoteTransport](fqn, args, system.internalClassLoader) match {
system.dynamicAccess.createInstanceFor[RemoteTransport](fqn, args) match {
case Left(problem) throw new RemoteTransportException("Could not load remote transport layer " + fqn, problem)
case Right(remote) remote
}

View file

@ -12,7 +12,7 @@ case class RemoteScope(node: Address) extends Scope {
def withFallback(other: Scope): Scope = this
}
class RemoteDeployer(_settings: ActorSystem.Settings, _classloader: ClassLoader) extends Deployer(_settings, _classloader) {
class RemoteDeployer(_settings: ActorSystem.Settings, _pm: DynamicAccess) extends Deployer(_settings, _pm) {
override protected def parseConfig(path: String, config: Config): Option[Deploy] = {
import scala.collection.JavaConverters._

View file

@ -219,7 +219,7 @@ class RemoteMessage(input: RemoteMessageProtocol, system: ActorSystemImpl) {
lazy val recipient: InternalActorRef = system.provider.actorFor(system.provider.rootGuardian, originalReceiver)
lazy val payload: AnyRef = MessageSerializer.deserialize(system, input.getMessage, getClass.getClassLoader)
lazy val payload: AnyRef = MessageSerializer.deserialize(system, input.getMessage)
override def toString = "RemoteMessage: " + payload + " to " + recipient + "<+{" + originalReceiver + "} from " + sender
}

View file

@ -61,7 +61,7 @@ class NettyRemoteTransport(val remoteSettings: RemoteSettings, val system: Actor
case sa: InetSocketAddress sa
case x throw new RemoteTransportException("unknown local address type " + x.getClass, null)
}
_address.compareAndSet(null, Address("akka", remoteSettings.systemName, Some(settings.Hostname), Some(addr.getPort)))
_address.compareAndSet(null, Address("akka", remoteSettings.systemName, settings.Hostname, addr.getPort))
}
def address = _address.get

View file

@ -183,7 +183,7 @@ class RemoteServerHandler(
instruction.getCommandType match {
case CommandType.CONNECT if settings.UsePassiveConnections
val origin = instruction.getOrigin
val inbound = Address("akka", origin.getSystem, Some(origin.getHostname), Some(origin.getPort))
val inbound = Address("akka", origin.getSystem, origin.getHostname, origin.getPort)
val client = new PassiveRemoteClient(event.getChannel, netty, inbound)
netty.bindClient(inbound, client)
case CommandType.SHUTDOWN //Will be unbound in channelClosed
@ -203,7 +203,7 @@ class RemoteServerHandler(
private def getClientAddress(c: Channel): Option[Address] =
c.getRemoteAddress match {
case inet: InetSocketAddress Some(Address("akka", "unknown(yet)", Some(inet.getAddress.toString), Some(inet.getPort)))
case inet: InetSocketAddress Some(Address("akka", "unknown(yet)", inet.getAddress.toString, inet.getPort))
case _ None
}
}

View file

@ -40,7 +40,7 @@ class NettySettings(config: Config, val systemName: String) {
case value value
}
@deprecated("WARNING: This should only be used by professionals.")
@deprecated("WARNING: This should only be used by professionals.", "2.0")
val PortSelector = getInt("port")
val ConnectionTimeout = Duration(getMilliseconds("connection-timeout"), MILLISECONDS)

View file

@ -30,6 +30,8 @@ case class RemoteRouterConfig(local: RouterConfig, nodes: Iterable[String]) exte
override def createActor(): Router = local.createActor()
override def routerDispatcher: String = local.routerDispatcher
override def resizer: Option[Resizer] = local.resizer
override def withFallback(other: RouterConfig): RouterConfig = other match {

View file

@ -5,6 +5,7 @@
package akka.serialization
import com.google.protobuf.Message
import akka.actor.DynamicAccess
/**
* This Serializer serializes `com.google.protobuf.Message`s
@ -19,7 +20,7 @@ class ProtobufSerializer extends Serializer {
case _ throw new IllegalArgumentException("Can't serialize a non-protobuf message using protobuf [" + obj + "]")
}
def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]], classLoader: Option[ClassLoader] = None): AnyRef =
def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef =
clazz match {
case None throw new IllegalArgumentException("Need a protobuf message class to be able to serialize bytes using protobuf")
case Some(c) c.getDeclaredMethod("parseFrom", ARRAY_OF_BYTE_ARRAY: _*).invoke(null, bytes).asInstanceOf[Message]

View file

@ -44,7 +44,7 @@ akka {
/looker/child/grandchild.remote = "akka://RemoteCommunicationSpec@localhost:12345"
}
}
""") with ImplicitSender {
""") with ImplicitSender with DefaultTimeout {
import RemoteCommunicationSpec._
@ -59,8 +59,6 @@ akka {
val here = system.actorFor("akka://remote_sys@localhost:12346/user/echo")
implicit val timeout = system.settings.ActorTimeout
override def atTermination() {
other.shutdown()
}

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