Merge branch 'master' of github.com:jboner/akka
This commit is contained in:
commit
1b35d59b67
119 changed files with 2330 additions and 958 deletions
|
|
@ -4,14 +4,21 @@
|
||||||
package akka.actor
|
package akka.actor
|
||||||
|
|
||||||
import java.io.File
|
import java.io.File
|
||||||
|
|
||||||
import com.typesafe.config.Config
|
import com.typesafe.config.Config
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
import com.typesafe.config.ConfigParseOptions
|
import com.typesafe.config.ConfigParseOptions
|
||||||
|
import akka.util.Timeout
|
||||||
|
import akka.util.duration._
|
||||||
|
|
||||||
@deprecated("use ActorSystem instead", "2.0")
|
@deprecated("use ActorSystem instead", "2.0")
|
||||||
object GlobalActorSystem extends ActorSystemImpl("GlobalSystem", OldConfigurationLoader.defaultConfig) {
|
object GlobalActorSystem extends ActorSystemImpl("GlobalSystem", OldConfigurationLoader.defaultConfig) {
|
||||||
start()
|
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)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -18,13 +18,13 @@ class OldFuture[T](future: Future[T]) {
|
||||||
|
|
||||||
@deprecated("use akka.dispatch.Await.result instead", "2.0")
|
@deprecated("use akka.dispatch.Await.result instead", "2.0")
|
||||||
def get: T = try {
|
def get: T = try {
|
||||||
Await.result(future, GlobalActorSystem.settings.ActorTimeout.duration)
|
Await.result(future, GlobalActorSystem.AwaitTimeout.duration)
|
||||||
} catch {
|
} catch {
|
||||||
case e: TimeoutException ⇒ throw new FutureTimeoutException(e.getMessage, e)
|
case e: TimeoutException ⇒ throw new FutureTimeoutException(e.getMessage, e)
|
||||||
}
|
}
|
||||||
|
|
||||||
@deprecated("use akka.dispatch.Await.ready instead", "2.0")
|
@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")
|
@deprecated("use akka.dispatch.Await.ready instead", "2.0")
|
||||||
def await(atMost: Duration) = try {
|
def await(atMost: Duration) = try {
|
||||||
|
|
|
||||||
|
|
@ -14,7 +14,7 @@ package object migration {
|
||||||
|
|
||||||
implicit def future2OldFuture[T](future: Future[T]): OldFuture[T] = new OldFuture[T](future)
|
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
|
implicit def defaultDispatcher: MessageDispatcher = GlobalActorSystem.dispatcher
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -40,10 +40,10 @@ public class JavaExtension {
|
||||||
static final ExtensionKey<OtherExtension> key = new ExtensionKey<OtherExtension>(OtherExtension.class) {
|
static final ExtensionKey<OtherExtension> key = new ExtensionKey<OtherExtension>(OtherExtension.class) {
|
||||||
};
|
};
|
||||||
|
|
||||||
public final ActorSystemImpl system;
|
public final ExtendedActorSystem system;
|
||||||
|
|
||||||
public OtherExtension(ActorSystemImpl i) {
|
public OtherExtension(ExtendedActorSystem system) {
|
||||||
system = i;
|
this.system = system;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -5,6 +5,7 @@ import akka.actor.ActorSystem;
|
||||||
|
|
||||||
import akka.japi.*;
|
import akka.japi.*;
|
||||||
import akka.util.Duration;
|
import akka.util.Duration;
|
||||||
|
import akka.testkit.TestKitExtension;
|
||||||
import org.junit.AfterClass;
|
import org.junit.AfterClass;
|
||||||
import org.junit.BeforeClass;
|
import org.junit.BeforeClass;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
@ -28,7 +29,7 @@ public class JavaFutureTests {
|
||||||
@BeforeClass
|
@BeforeClass
|
||||||
public static void beforeAll() {
|
public static void beforeAll() {
|
||||||
system = ActorSystem.create("JavaFutureTests", AkkaSpec.testConf());
|
system = ActorSystem.create("JavaFutureTests", AkkaSpec.testConf());
|
||||||
t = system.settings().ActorTimeout();
|
t = TestKitExtension.get(system).DefaultTimeout();
|
||||||
}
|
}
|
||||||
|
|
||||||
@AfterClass
|
@AfterClass
|
||||||
|
|
@ -61,10 +62,10 @@ public class JavaFutureTests {
|
||||||
Promise<String> cf = Futures.promise(system.dispatcher());
|
Promise<String> cf = Futures.promise(system.dispatcher());
|
||||||
Future<String> f = cf;
|
Future<String> f = cf;
|
||||||
f.onSuccess(new OnSuccess<String>() {
|
f.onSuccess(new OnSuccess<String>() {
|
||||||
public void onSuccess(String result) {
|
public void onSuccess(String result) {
|
||||||
if (result.equals("foo"))
|
if (result.equals("foo"))
|
||||||
latch.countDown();
|
latch.countDown();
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
|
||||||
cf.success("foo");
|
cf.success("foo");
|
||||||
|
|
@ -78,10 +79,10 @@ public class JavaFutureTests {
|
||||||
Promise<String> cf = Futures.promise(system.dispatcher());
|
Promise<String> cf = Futures.promise(system.dispatcher());
|
||||||
Future<String> f = cf;
|
Future<String> f = cf;
|
||||||
f.onFailure(new OnFailure() {
|
f.onFailure(new OnFailure() {
|
||||||
public void onFailure(Throwable t) {
|
public void onFailure(Throwable t) {
|
||||||
if (t instanceof NullPointerException)
|
if (t instanceof NullPointerException)
|
||||||
latch.countDown();
|
latch.countDown();
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
|
||||||
Throwable exception = new NullPointerException();
|
Throwable exception = new NullPointerException();
|
||||||
|
|
@ -296,8 +297,10 @@ public class JavaFutureTests {
|
||||||
Promise<Object> p = Futures.promise(system.dispatcher());
|
Promise<Object> p = Futures.promise(system.dispatcher());
|
||||||
Future<Object> f = p.future().recover(new Recover<Object>() {
|
Future<Object> f = p.future().recover(new Recover<Object>() {
|
||||||
public Object recover(Throwable t) throws Throwable {
|
public Object recover(Throwable t) throws Throwable {
|
||||||
if (t == fail) return "foo";
|
if (t == fail)
|
||||||
else throw t;
|
return "foo";
|
||||||
|
else
|
||||||
|
throw t;
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
Duration d = Duration.create(1, TimeUnit.SECONDS);
|
Duration d = Duration.create(1, TimeUnit.SECONDS);
|
||||||
|
|
@ -311,8 +314,10 @@ public class JavaFutureTests {
|
||||||
Promise<Object> p = Futures.promise(system.dispatcher());
|
Promise<Object> p = Futures.promise(system.dispatcher());
|
||||||
Future<Object> f = p.future().recoverWith(new Recover<Future<Object>>() {
|
Future<Object> f = p.future().recoverWith(new Recover<Future<Object>>() {
|
||||||
public Future<Object> recover(Throwable t) throws Throwable {
|
public Future<Object> recover(Throwable t) throws Throwable {
|
||||||
if (t == fail) return Futures.<Object>successful("foo", system.dispatcher()).future();
|
if (t == fail)
|
||||||
else throw t;
|
return Futures.<Object> successful("foo", system.dispatcher()).future();
|
||||||
|
else
|
||||||
|
throw t;
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
Duration d = Duration.create(1, TimeUnit.SECONDS);
|
Duration d = Duration.create(1, TimeUnit.SECONDS);
|
||||||
|
|
|
||||||
|
|
@ -305,6 +305,7 @@ class ActorLookupSpec extends AkkaSpec with DefaultTimeout {
|
||||||
intercept[MalformedURLException] { ActorPath.fromString("://hallo") }
|
intercept[MalformedURLException] { ActorPath.fromString("://hallo") }
|
||||||
intercept[MalformedURLException] { ActorPath.fromString("s://dd@:12") }
|
intercept[MalformedURLException] { ActorPath.fromString("s://dd@:12") }
|
||||||
intercept[MalformedURLException] { ActorPath.fromString("s://dd@h:hd") }
|
intercept[MalformedURLException] { ActorPath.fromString("s://dd@h:hd") }
|
||||||
|
intercept[MalformedURLException] { ActorPath.fromString("a://l:1/b") }
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -11,11 +11,10 @@ import akka.testkit._
|
||||||
import akka.util.Timeout
|
import akka.util.Timeout
|
||||||
import akka.util.duration._
|
import akka.util.duration._
|
||||||
import java.lang.IllegalStateException
|
import java.lang.IllegalStateException
|
||||||
import akka.util.ReflectiveAccess
|
|
||||||
import akka.serialization.Serialization
|
|
||||||
import java.util.concurrent.{ CountDownLatch, TimeUnit }
|
import java.util.concurrent.{ CountDownLatch, TimeUnit }
|
||||||
import akka.dispatch.{ Await, DefaultPromise, Promise, Future }
|
import akka.dispatch.{ Await, DefaultPromise, Promise, Future }
|
||||||
import akka.pattern.ask
|
import akka.pattern.ask
|
||||||
|
import akka.serialization.JavaSerializer
|
||||||
|
|
||||||
object ActorRefSpec {
|
object ActorRefSpec {
|
||||||
|
|
||||||
|
|
@ -240,6 +239,7 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout {
|
||||||
|
|
||||||
"be serializable using Java Serialization on local node" in {
|
"be serializable using Java Serialization on local node" in {
|
||||||
val a = system.actorOf(Props[InnerActor])
|
val a = system.actorOf(Props[InnerActor])
|
||||||
|
val esys = system.asInstanceOf[ExtendedActorSystem]
|
||||||
|
|
||||||
import java.io._
|
import java.io._
|
||||||
|
|
||||||
|
|
@ -251,14 +251,21 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout {
|
||||||
out.flush
|
out.flush
|
||||||
out.close
|
out.close
|
||||||
|
|
||||||
Serialization.currentSystem.withValue(system.asInstanceOf[ActorSystemImpl]) {
|
val bytes = baos.toByteArray
|
||||||
val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray))
|
|
||||||
|
JavaSerializer.currentSystem.withValue(esys) {
|
||||||
|
val in = new ObjectInputStream(new ByteArrayInputStream(bytes))
|
||||||
val readA = in.readObject
|
val readA = in.readObject
|
||||||
|
|
||||||
a.isInstanceOf[LocalActorRef] must be === true
|
a.isInstanceOf[LocalActorRef] must be === true
|
||||||
readA.isInstanceOf[LocalActorRef] must be === true
|
readA.isInstanceOf[LocalActorRef] must be === true
|
||||||
(readA eq a) 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 {
|
"throw an exception on deserialize if no system in scope" in {
|
||||||
|
|
@ -297,7 +304,7 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout {
|
||||||
out.flush
|
out.flush
|
||||||
out.close
|
out.close
|
||||||
|
|
||||||
Serialization.currentSystem.withValue(sysImpl) {
|
JavaSerializer.currentSystem.withValue(sysImpl) {
|
||||||
val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray))
|
val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray))
|
||||||
in.readObject must be === new EmptyLocalActorRef(sysImpl.provider, system.actorFor("/").path / "non-existing", system.eventStream)
|
in.readObject must be === new EmptyLocalActorRef(sysImpl.provider, system.actorFor("/").path / "non-existing", system.eventStream)
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -8,7 +8,6 @@ import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach }
|
||||||
import akka.util.Duration
|
import akka.util.Duration
|
||||||
import akka.util.Timeout
|
import akka.util.Timeout
|
||||||
import akka.util.duration._
|
import akka.util.duration._
|
||||||
import akka.serialization.Serialization
|
|
||||||
import java.util.concurrent.atomic.AtomicReference
|
import java.util.concurrent.atomic.AtomicReference
|
||||||
import annotation.tailrec
|
import annotation.tailrec
|
||||||
import akka.testkit.{ EventFilter, filterEvents, AkkaSpec }
|
import akka.testkit.{ EventFilter, filterEvents, AkkaSpec }
|
||||||
|
|
@ -19,6 +18,7 @@ import akka.japi.{ Creator, Option ⇒ JOption }
|
||||||
import akka.testkit.DefaultTimeout
|
import akka.testkit.DefaultTimeout
|
||||||
import akka.dispatch.{ Await, Dispatchers, Future, Promise }
|
import akka.dispatch.{ Await, Dispatchers, Future, Promise }
|
||||||
import akka.pattern.ask
|
import akka.pattern.ask
|
||||||
|
import akka.serialization.JavaSerializer
|
||||||
|
|
||||||
object TypedActorSpec {
|
object TypedActorSpec {
|
||||||
|
|
||||||
|
|
@ -113,7 +113,7 @@ object TypedActorSpec {
|
||||||
}
|
}
|
||||||
|
|
||||||
def futureComposePigdogFrom(foo: Foo): Future[String] = {
|
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)
|
foo.futurePigdog(500).map(_.toUpperCase)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -367,7 +367,7 @@ class TypedActorSpec extends AkkaSpec(TypedActorSpec.config)
|
||||||
|
|
||||||
"be able to serialize and deserialize invocations" in {
|
"be able to serialize and deserialize invocations" in {
|
||||||
import java.io._
|
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 m = TypedActor.MethodCall(classOf[Foo].getDeclaredMethod("pigdog"), Array[AnyRef]())
|
||||||
val baos = new ByteArrayOutputStream(8192 * 4)
|
val baos = new ByteArrayOutputStream(8192 * 4)
|
||||||
val out = new ObjectOutputStream(baos)
|
val out = new ObjectOutputStream(baos)
|
||||||
|
|
@ -386,7 +386,7 @@ class TypedActorSpec extends AkkaSpec(TypedActorSpec.config)
|
||||||
"be able to serialize and deserialize invocations' parameters" in {
|
"be able to serialize and deserialize invocations' parameters" in {
|
||||||
import java.io._
|
import java.io._
|
||||||
val someFoo: Foo = new Bar
|
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 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 baos = new ByteArrayOutputStream(8192 * 4)
|
||||||
val out = new ObjectOutputStream(baos)
|
val out = new ObjectOutputStream(baos)
|
||||||
|
|
|
||||||
|
|
@ -50,6 +50,8 @@ object ActorModelSpec {
|
||||||
|
|
||||||
case object Restart extends ActorModelMessage
|
case object Restart extends ActorModelMessage
|
||||||
|
|
||||||
|
case object DoubleStop extends ActorModelMessage
|
||||||
|
|
||||||
case class ThrowException(e: Throwable) extends ActorModelMessage
|
case class ThrowException(e: Throwable) extends ActorModelMessage
|
||||||
|
|
||||||
val Ping = "Ping"
|
val Ping = "Ping"
|
||||||
|
|
@ -86,6 +88,7 @@ object ActorModelSpec {
|
||||||
case Restart ⇒ ack; busy.switchOff(); throw new Exception("Restart requested")
|
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 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 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)
|
val stops = new AtomicLong(0)
|
||||||
|
|
||||||
def getStats(actorRef: ActorRef) = {
|
def getStats(actorRef: ActorRef) = {
|
||||||
stats.putIfAbsent(actorRef, new InterceptorStats) match {
|
val is = new InterceptorStats
|
||||||
case null ⇒ stats.get(actorRef)
|
stats.putIfAbsent(actorRef, is) match {
|
||||||
|
case null ⇒ is
|
||||||
case other ⇒ other
|
case other ⇒ other
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -124,12 +128,12 @@ object ActorModelSpec {
|
||||||
}
|
}
|
||||||
|
|
||||||
protected[akka] abstract override def register(actor: ActorCell) {
|
protected[akka] abstract override def register(actor: ActorCell) {
|
||||||
getStats(actor.self).registers.incrementAndGet()
|
assert(getStats(actor.self).registers.incrementAndGet() == 1)
|
||||||
super.register(actor)
|
super.register(actor)
|
||||||
}
|
}
|
||||||
|
|
||||||
protected[akka] abstract override def unregister(actor: ActorCell) {
|
protected[akka] abstract override def unregister(actor: ActorCell) {
|
||||||
getStats(actor.self).unregisters.incrementAndGet()
|
assert(getStats(actor.self).unregisters.incrementAndGet() == 1)
|
||||||
super.unregister(actor)
|
super.unregister(actor)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -190,13 +194,13 @@ object ActorModelSpec {
|
||||||
}
|
}
|
||||||
|
|
||||||
def assertRef(actorRef: ActorRef, dispatcher: MessageDispatcher = null)(
|
def assertRef(actorRef: ActorRef, dispatcher: MessageDispatcher = null)(
|
||||||
suspensions: Long = statsFor(actorRef).suspensions.get(),
|
suspensions: Long = statsFor(actorRef, dispatcher).suspensions.get(),
|
||||||
resumes: Long = statsFor(actorRef).resumes.get(),
|
resumes: Long = statsFor(actorRef, dispatcher).resumes.get(),
|
||||||
registers: Long = statsFor(actorRef).registers.get(),
|
registers: Long = statsFor(actorRef, dispatcher).registers.get(),
|
||||||
unregisters: Long = statsFor(actorRef).unregisters.get(),
|
unregisters: Long = statsFor(actorRef, dispatcher).unregisters.get(),
|
||||||
msgsReceived: Long = statsFor(actorRef).msgsReceived.get(),
|
msgsReceived: Long = statsFor(actorRef, dispatcher).msgsReceived.get(),
|
||||||
msgsProcessed: Long = statsFor(actorRef).msgsProcessed.get(),
|
msgsProcessed: Long = statsFor(actorRef, dispatcher).msgsProcessed.get(),
|
||||||
restarts: Long = statsFor(actorRef).restarts.get())(implicit system: ActorSystem) {
|
restarts: Long = statsFor(actorRef, dispatcher).restarts.get())(implicit system: ActorSystem) {
|
||||||
val stats = statsFor(actorRef, Option(dispatcher).getOrElse(actorRef.asInstanceOf[LocalActorRef].underlying.dispatcher))
|
val stats = statsFor(actorRef, Option(dispatcher).getOrElse(actorRef.asInstanceOf[LocalActorRef].underlying.dispatcher))
|
||||||
val deadline = System.currentTimeMillis + 1000
|
val deadline = System.currentTimeMillis + 1000
|
||||||
try {
|
try {
|
||||||
|
|
@ -362,16 +366,19 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa
|
||||||
case e ⇒
|
case e ⇒
|
||||||
dispatcher match {
|
dispatcher match {
|
||||||
case dispatcher: BalancingDispatcher ⇒
|
case dispatcher: BalancingDispatcher ⇒
|
||||||
val buddies = dispatcher.buddies
|
val team = dispatcher.team
|
||||||
val mq = dispatcher.messageQueue
|
val mq = dispatcher.messageQueue
|
||||||
|
|
||||||
System.err.println("Buddies left: ")
|
System.err.println("Teammates left: " + team.size + " stopLatch: " + stopLatch.getCount + " inhab:" + dispatcher.inhabitants)
|
||||||
buddies.toArray foreach {
|
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 ⇒
|
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(" - " + 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 _ ⇒
|
case _ ⇒
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -426,6 +433,14 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa
|
||||||
assert(f5.value.isEmpty)
|
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),
|
Duration(config.getNanoseconds("throughput-deadline-time"), TimeUnit.NANOSECONDS),
|
||||||
mailboxType,
|
mailboxType,
|
||||||
configureExecutor(),
|
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
|
override def dispatcher(): MessageDispatcher = instance
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -53,6 +53,7 @@ class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference) {
|
||||||
c.getMilliseconds("shutdown-timeout") must equal(1 * 1000)
|
c.getMilliseconds("shutdown-timeout") must equal(1 * 1000)
|
||||||
c.getInt("throughput") must equal(5)
|
c.getInt("throughput") must equal(5)
|
||||||
c.getMilliseconds("throughput-deadline-time") must equal(0)
|
c.getMilliseconds("throughput-deadline-time") must equal(0)
|
||||||
|
c.getBoolean("attempt-teamwork") must equal(true)
|
||||||
}
|
}
|
||||||
|
|
||||||
//Fork join executor config
|
//Fork join executor config
|
||||||
|
|
|
||||||
|
|
@ -860,7 +860,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa
|
||||||
|
|
||||||
val l1, l2 = new TestLatch
|
val l1, l2 = new TestLatch
|
||||||
val complex = Future() map { _ ⇒
|
val complex = Future() map { _ ⇒
|
||||||
Future.blocking(system.dispatcher)
|
Future.blocking()
|
||||||
val nested = Future(())
|
val nested = Future(())
|
||||||
nested foreach (_ ⇒ l1.open())
|
nested foreach (_ ⇒ l1.open())
|
||||||
Await.ready(l1, TestLatch.DefaultTimeout) // make sure nested is completed
|
Await.ready(l1, TestLatch.DefaultTimeout) // make sure nested is completed
|
||||||
|
|
|
||||||
|
|
@ -5,8 +5,9 @@ package akka.pattern
|
||||||
|
|
||||||
import akka.testkit.AkkaSpec
|
import akka.testkit.AkkaSpec
|
||||||
import akka.util.duration._
|
import akka.util.duration._
|
||||||
|
import akka.testkit.DefaultTimeout
|
||||||
|
|
||||||
class AskSpec extends AkkaSpec {
|
class AskSpec extends AkkaSpec with DefaultTimeout {
|
||||||
|
|
||||||
"The “ask” pattern" must {
|
"The “ask” pattern" must {
|
||||||
|
|
||||||
|
|
@ -22,7 +23,6 @@ class AskSpec extends AkkaSpec {
|
||||||
|
|
||||||
"return broken promises on EmptyLocalActorRefs" in {
|
"return broken promises on EmptyLocalActorRefs" in {
|
||||||
val empty = system.actorFor("unknown")
|
val empty = system.actorFor("unknown")
|
||||||
implicit val timeout = system.settings.ActorTimeout
|
|
||||||
val f = empty ? 3.14
|
val f = empty ? 3.14
|
||||||
f.isCompleted must be(true)
|
f.isCompleted must be(true)
|
||||||
f.value.get match {
|
f.value.get match {
|
||||||
|
|
|
||||||
|
|
@ -11,6 +11,7 @@ import akka.util.duration._
|
||||||
import akka.actor.ActorRef
|
import akka.actor.ActorRef
|
||||||
import java.util.concurrent.atomic.AtomicInteger
|
import java.util.concurrent.atomic.AtomicInteger
|
||||||
import akka.pattern.ask
|
import akka.pattern.ask
|
||||||
|
import akka.util.Duration
|
||||||
|
|
||||||
object ResizerSpec {
|
object ResizerSpec {
|
||||||
|
|
||||||
|
|
@ -24,6 +25,9 @@ object ResizerSpec {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
bal-disp {
|
||||||
|
type = BalancingDispatcher
|
||||||
|
}
|
||||||
"""
|
"""
|
||||||
|
|
||||||
class TestActor extends Actor {
|
class TestActor extends Actor {
|
||||||
|
|
@ -133,7 +137,7 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with
|
||||||
pressureThreshold = 0,
|
pressureThreshold = 0,
|
||||||
messagesPerResize = 1)
|
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)
|
val latch1 = new TestLatch(1)
|
||||||
router ! (latch1, busy)
|
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
|
// as influenced by the backlog of blocking pooled actors
|
||||||
|
|
||||||
val resizer = DefaultResizer(
|
val resizer = DefaultResizer(
|
||||||
lowerBound = 2,
|
lowerBound = 3,
|
||||||
upperBound = 4,
|
upperBound = 5,
|
||||||
rampupRate = 0.1,
|
rampupRate = 0.1,
|
||||||
|
backoffRate = 0.0,
|
||||||
pressureThreshold = 1,
|
pressureThreshold = 1,
|
||||||
messagesPerResize = 1,
|
messagesPerResize = 1,
|
||||||
backoffThreshold = 0.0)
|
backoffThreshold = 0.0)
|
||||||
|
|
||||||
val router = system.actorOf(Props(new Actor {
|
val router = system.actorOf(Props(new Actor {
|
||||||
def receive = {
|
def receive = {
|
||||||
case (n: Int, latch: TestLatch, count: AtomicInteger) ⇒
|
case d: Duration ⇒ d.dilated.sleep; sender ! "done"
|
||||||
(n millis).dilated.sleep
|
case "echo" ⇒ sender ! "reply"
|
||||||
count.incrementAndGet
|
|
||||||
latch.countDown()
|
|
||||||
}
|
}
|
||||||
}).withRouter(RoundRobinRouter(resizer = Some(resizer))))
|
}).withRouter(RoundRobinRouter(resizer = Some(resizer))))
|
||||||
|
|
||||||
// first message should create the minimum number of routees
|
// 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) = {
|
routees(router) must be(3)
|
||||||
(10 millis).dilated.sleep
|
|
||||||
for (m ← 0 until loops) {
|
def loop(loops: Int, d: Duration) = {
|
||||||
router.!((t, latch, count))
|
for (m ← 0 until loops) router ! d
|
||||||
(10 millis).dilated.sleep
|
for (m ← 0 until loops) expectMsg(d * 3, "done")
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// 2 more should go thru without triggering more
|
// 2 more should go thru without triggering more
|
||||||
val count1 = new AtomicInteger
|
loop(2, 200 millis)
|
||||||
val latch1 = TestLatch(2)
|
|
||||||
loop(2, 200, latch1, count1)
|
|
||||||
Await.ready(latch1, TestLatch.DefaultTimeout)
|
|
||||||
count1.get must be(2)
|
|
||||||
|
|
||||||
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
|
// a whole bunch should max it out
|
||||||
val count2 = new AtomicInteger
|
loop(10, 500 millis)
|
||||||
val latch2 = TestLatch(10)
|
awaitCond(routees(router) > 3)
|
||||||
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) == 5)
|
||||||
}
|
}
|
||||||
|
|
||||||
"backoff" in {
|
"backoff" in {
|
||||||
|
|
@ -236,9 +235,9 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with
|
||||||
(300 millis).dilated.sleep
|
(300 millis).dilated.sleep
|
||||||
|
|
||||||
// let it cool down
|
// let it cool down
|
||||||
for (m ← 0 to 3) {
|
for (m ← 0 to 5) {
|
||||||
router ! 1
|
router ! 1
|
||||||
(200 millis).dilated.sleep
|
(500 millis).dilated.sleep
|
||||||
}
|
}
|
||||||
|
|
||||||
Await.result(router ? CurrentRoutees, 5 seconds).asInstanceOf[RouterRoutees].routees.size must be < (z)
|
Await.result(router ? CurrentRoutees, 5 seconds).asInstanceOf[RouterRoutees].routees.size must be < (z)
|
||||||
|
|
|
||||||
|
|
@ -15,6 +15,7 @@ import com.typesafe.config.ConfigFactory
|
||||||
import akka.pattern.ask
|
import akka.pattern.ask
|
||||||
import java.util.concurrent.ConcurrentHashMap
|
import java.util.concurrent.ConcurrentHashMap
|
||||||
import com.typesafe.config.Config
|
import com.typesafe.config.Config
|
||||||
|
import akka.dispatch.Dispatchers
|
||||||
|
|
||||||
object RoutingSpec {
|
object RoutingSpec {
|
||||||
|
|
||||||
|
|
@ -51,6 +52,7 @@ object RoutingSpec {
|
||||||
case (sender, message) ⇒ Nil
|
case (sender, message) ⇒ Nil
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
def routerDispatcher: String = Dispatchers.DefaultDispatcherId
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
@ -539,6 +541,8 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
|
||||||
//#crRouter
|
//#crRouter
|
||||||
case class VoteCountRouter() extends RouterConfig {
|
case class VoteCountRouter() extends RouterConfig {
|
||||||
|
|
||||||
|
def routerDispatcher: String = Dispatchers.DefaultDispatcherId
|
||||||
|
|
||||||
//#crRoute
|
//#crRoute
|
||||||
def createRoute(routeeProps: Props, routeeProvider: RouteeProvider): Route = {
|
def createRoute(routeeProps: Props, routeeProvider: RouteeProvider): Route = {
|
||||||
val democratActor = routeeProvider.context.actorOf(Props(new DemocratActor()), "d")
|
val democratActor = routeeProvider.context.actorOf(Props(new DemocratActor()), "d")
|
||||||
|
|
|
||||||
|
|
@ -89,7 +89,7 @@ class SerializeSpec extends AkkaSpec(SerializeSpec.config) {
|
||||||
case Left(exception) ⇒ fail(exception)
|
case Left(exception) ⇒ fail(exception)
|
||||||
case Right(bytes) ⇒ bytes
|
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 Left(exception) ⇒ fail(exception)
|
||||||
case Right(add) ⇒ assert(add === addr)
|
case Right(add) ⇒ assert(add === addr)
|
||||||
}
|
}
|
||||||
|
|
@ -101,7 +101,7 @@ class SerializeSpec extends AkkaSpec(SerializeSpec.config) {
|
||||||
case Left(exception) ⇒ fail(exception)
|
case Left(exception) ⇒ fail(exception)
|
||||||
case Right(bytes) ⇒ bytes
|
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 Left(exception) ⇒ fail(exception)
|
||||||
case Right(p) ⇒ assert(p === person)
|
case Right(p) ⇒ assert(p === person)
|
||||||
}
|
}
|
||||||
|
|
@ -114,7 +114,7 @@ class SerializeSpec extends AkkaSpec(SerializeSpec.config) {
|
||||||
case Left(exception) ⇒ fail(exception)
|
case Left(exception) ⇒ fail(exception)
|
||||||
case Right(bytes) ⇒ bytes
|
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 Left(exception) ⇒ fail(exception)
|
||||||
case Right(p) ⇒ assert(p === r)
|
case Right(p) ⇒ assert(p === r)
|
||||||
}
|
}
|
||||||
|
|
@ -146,7 +146,7 @@ class SerializeSpec extends AkkaSpec(SerializeSpec.config) {
|
||||||
out.close()
|
out.close()
|
||||||
|
|
||||||
val in = new ObjectInputStream(new ByteArrayInputStream(outbuf.toByteArray))
|
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]
|
val deadLetters = in.readObject().asInstanceOf[DeadLetterActorRef]
|
||||||
(deadLetters eq a.deadLetters) must be(true)
|
(deadLetters eq a.deadLetters) must be(true)
|
||||||
}
|
}
|
||||||
|
|
@ -285,8 +285,5 @@ class TestSerializer extends Serializer {
|
||||||
Array.empty[Byte]
|
Array.empty[Byte]
|
||||||
}
|
}
|
||||||
|
|
||||||
def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]] = None,
|
def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = null
|
||||||
classLoader: Option[ClassLoader] = None): AnyRef = {
|
|
||||||
null
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -378,10 +378,11 @@ public interface Config extends ConfigMergeable {
|
||||||
Object getAnyRef(String path);
|
Object getAnyRef(String path);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Gets the value at the given path, unless the value is a null value or
|
* Gets the value at the given path, unless the value is a
|
||||||
* missing, in which case it throws just like the other getters. Use
|
* null value or missing, in which case it throws just like
|
||||||
* {@code get()} from the {@link java.util.Map Map} interface if you want an
|
* the other getters. Use {@code get()} on the {@link
|
||||||
* unprocessed value.
|
* Config#root()} object (or other object in the tree) if you
|
||||||
|
* want an unprocessed value.
|
||||||
*
|
*
|
||||||
* @param path
|
* @param path
|
||||||
* path expression
|
* path expression
|
||||||
|
|
|
||||||
|
|
@ -161,8 +161,8 @@ public final class ConfigFactory {
|
||||||
* used rather than the default
|
* used rather than the default
|
||||||
* <code>application.{conf,json,properties}</code> classpath resources.
|
* <code>application.{conf,json,properties}</code> classpath resources.
|
||||||
* These system properties should not be set in code (after all, you can
|
* 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)
|
* 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
|
* 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.
|
* are intended for use by the person or script launching the application.
|
||||||
* For example someone might have a <code>production.conf</code> that
|
* For example someone might have a <code>production.conf</code> that
|
||||||
* include <code>application.conf</code> but then change a couple of values.
|
* include <code>application.conf</code> but then change a couple of values.
|
||||||
|
|
|
||||||
|
|
@ -23,6 +23,9 @@ import com.typesafe.config.ConfigValueType;
|
||||||
|
|
||||||
abstract class AbstractConfigObject extends AbstractConfigValue implements
|
abstract class AbstractConfigObject extends AbstractConfigValue implements
|
||||||
ConfigObject {
|
ConfigObject {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
final private SimpleConfig config;
|
final private SimpleConfig config;
|
||||||
|
|
||||||
protected AbstractConfigObject(ConfigOrigin origin) {
|
protected AbstractConfigObject(ConfigOrigin origin) {
|
||||||
|
|
|
||||||
|
|
@ -20,6 +20,8 @@ import com.typesafe.config.ConfigValue;
|
||||||
*/
|
*/
|
||||||
abstract class AbstractConfigValue implements ConfigValue, MergeableValue, Serializable {
|
abstract class AbstractConfigValue implements ConfigValue, MergeableValue, Serializable {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
final private SimpleConfigOrigin origin;
|
final private SimpleConfigOrigin origin;
|
||||||
|
|
||||||
AbstractConfigValue(ConfigOrigin origin) {
|
AbstractConfigValue(ConfigOrigin origin) {
|
||||||
|
|
|
||||||
|
|
@ -8,6 +8,8 @@ import com.typesafe.config.ConfigValueType;
|
||||||
|
|
||||||
final class ConfigBoolean extends AbstractConfigValue {
|
final class ConfigBoolean extends AbstractConfigValue {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
final private boolean value;
|
final private boolean value;
|
||||||
|
|
||||||
ConfigBoolean(ConfigOrigin origin, boolean value) {
|
ConfigBoolean(ConfigOrigin origin, boolean value) {
|
||||||
|
|
|
||||||
|
|
@ -8,6 +8,8 @@ import com.typesafe.config.ConfigValueType;
|
||||||
|
|
||||||
final class ConfigDouble extends ConfigNumber {
|
final class ConfigDouble extends ConfigNumber {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
final private double value;
|
final private double value;
|
||||||
|
|
||||||
ConfigDouble(ConfigOrigin origin, double value, String originalText) {
|
ConfigDouble(ConfigOrigin origin, double value, String originalText) {
|
||||||
|
|
|
||||||
|
|
@ -8,6 +8,8 @@ import com.typesafe.config.ConfigValueType;
|
||||||
|
|
||||||
final class ConfigInt extends ConfigNumber {
|
final class ConfigInt extends ConfigNumber {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
final private int value;
|
final private int value;
|
||||||
|
|
||||||
ConfigInt(ConfigOrigin origin, int value, String originalText) {
|
ConfigInt(ConfigOrigin origin, int value, String originalText) {
|
||||||
|
|
|
||||||
|
|
@ -8,6 +8,8 @@ import com.typesafe.config.ConfigValueType;
|
||||||
|
|
||||||
final class ConfigLong extends ConfigNumber {
|
final class ConfigLong extends ConfigNumber {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
final private long value;
|
final private long value;
|
||||||
|
|
||||||
ConfigLong(ConfigOrigin origin, long value, String originalText) {
|
ConfigLong(ConfigOrigin origin, long value, String originalText) {
|
||||||
|
|
|
||||||
|
|
@ -16,6 +16,8 @@ import com.typesafe.config.ConfigValueType;
|
||||||
*/
|
*/
|
||||||
final class ConfigNull extends AbstractConfigValue {
|
final class ConfigNull extends AbstractConfigValue {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
ConfigNull(ConfigOrigin origin) {
|
ConfigNull(ConfigOrigin origin) {
|
||||||
super(origin);
|
super(origin);
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -7,6 +7,9 @@ import com.typesafe.config.ConfigException;
|
||||||
import com.typesafe.config.ConfigOrigin;
|
import com.typesafe.config.ConfigOrigin;
|
||||||
|
|
||||||
abstract class ConfigNumber extends AbstractConfigValue {
|
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
|
// 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
|
// 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
|
// config file. It's purely cosmetic; equals/hashCode don't consider this
|
||||||
|
|
|
||||||
|
|
@ -8,6 +8,8 @@ import com.typesafe.config.ConfigValueType;
|
||||||
|
|
||||||
final class ConfigString extends AbstractConfigValue {
|
final class ConfigString extends AbstractConfigValue {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
final private String value;
|
final private String value;
|
||||||
|
|
||||||
ConfigString(ConfigOrigin origin, String value) {
|
ConfigString(ConfigOrigin origin, String value) {
|
||||||
|
|
|
||||||
|
|
@ -11,6 +11,8 @@ import com.typesafe.config.ConfigException;
|
||||||
|
|
||||||
final class Path implements Serializable {
|
final class Path implements Serializable {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
final private String first;
|
final private String first;
|
||||||
final private Path remainder;
|
final private Path remainder;
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -31,6 +31,8 @@ import com.typesafe.config.ConfigValueType;
|
||||||
*/
|
*/
|
||||||
final class SimpleConfig implements Config, MergeableValue, Serializable {
|
final class SimpleConfig implements Config, MergeableValue, Serializable {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
final private AbstractConfigObject object;
|
final private AbstractConfigObject object;
|
||||||
|
|
||||||
SimpleConfig(AbstractConfigObject object) {
|
SimpleConfig(AbstractConfigObject object) {
|
||||||
|
|
|
||||||
|
|
@ -17,6 +17,8 @@ import com.typesafe.config.ConfigValue;
|
||||||
|
|
||||||
final class SimpleConfigObject extends AbstractConfigObject {
|
final class SimpleConfigObject extends AbstractConfigObject {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
// this map should never be modified - assume immutable
|
// this map should never be modified - assume immutable
|
||||||
final private Map<String, AbstractConfigValue> value;
|
final private Map<String, AbstractConfigValue> value;
|
||||||
final private boolean resolved;
|
final private boolean resolved;
|
||||||
|
|
|
||||||
|
|
@ -19,6 +19,9 @@ import com.typesafe.config.ConfigOrigin;
|
||||||
// it would be cleaner to have a class hierarchy for various origin types,
|
// 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.
|
// but was hoping this would be enough simpler to be a little messy. eh.
|
||||||
final class SimpleConfigOrigin implements ConfigOrigin, Serializable {
|
final class SimpleConfigOrigin implements ConfigOrigin, Serializable {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
final private String description;
|
final private String description;
|
||||||
final private int lineNumber;
|
final private int lineNumber;
|
||||||
final private int endLineNumber;
|
final private int endLineNumber;
|
||||||
|
|
|
||||||
|
|
@ -4,6 +4,8 @@ import java.io.Serializable;
|
||||||
|
|
||||||
final class SubstitutionExpression implements Serializable {
|
final class SubstitutionExpression implements Serializable {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
final private Path path;
|
final private Path path;
|
||||||
final private boolean optional;
|
final private boolean optional;
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -1,6 +1,6 @@
|
||||||
##############################
|
####################################
|
||||||
# Akka Reference Config File #
|
# Akka Actor Reference Config File #
|
||||||
##############################
|
####################################
|
||||||
|
|
||||||
# This the reference config file has all the default settings.
|
# This the reference config file has all the default settings.
|
||||||
# Make your edits/overrides in your application.conf.
|
# Make your edits/overrides in your application.conf.
|
||||||
|
|
@ -50,12 +50,6 @@ akka {
|
||||||
# removed from their parents
|
# removed from their parents
|
||||||
reaper-interval = 5s
|
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,
|
# Serializes and deserializes (non-primitive) messages to ensure immutability,
|
||||||
# this is only intended for testing.
|
# this is only intended for testing.
|
||||||
serialize-messages = off
|
serialize-messages = off
|
||||||
|
|
@ -64,6 +58,11 @@ akka {
|
||||||
# this is only intended for testing.
|
# this is only intended for testing.
|
||||||
serialize-creators = off
|
serialize-creators = off
|
||||||
|
|
||||||
|
typed {
|
||||||
|
# Default timeout for typed actor methods with non-void return type
|
||||||
|
timeout = 5s
|
||||||
|
}
|
||||||
|
|
||||||
deployment {
|
deployment {
|
||||||
|
|
||||||
# deployment id pattern - on the format: /parent/child etc.
|
# 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
|
# mailbox is used. The Class of the FQCN must have a constructor with a
|
||||||
# com.typesafe.config.Config parameter.
|
# com.typesafe.config.Config parameter.
|
||||||
mailbox-type = ""
|
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 {
|
debug {
|
||||||
|
|
|
||||||
|
|
@ -5,10 +5,8 @@
|
||||||
package akka
|
package akka
|
||||||
|
|
||||||
import akka.actor.newUuid
|
import akka.actor.newUuid
|
||||||
import java.net.{ InetAddress, UnknownHostException }
|
|
||||||
|
|
||||||
object AkkaException {
|
object AkkaException {
|
||||||
val hostname = try InetAddress.getLocalHost.getHostAddress catch { case e: UnknownHostException ⇒ "unknown host" }
|
|
||||||
|
|
||||||
def toStringWithStackTrace(throwable: Throwable): String = throwable match {
|
def toStringWithStackTrace(throwable: Throwable): String = throwable match {
|
||||||
case null ⇒ "Unknown Throwable: was 'null'"
|
case null ⇒ "Unknown Throwable: was 'null'"
|
||||||
|
|
@ -36,7 +34,7 @@ object AkkaException {
|
||||||
*/
|
*/
|
||||||
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
|
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
|
||||||
class AkkaException(message: String = "", cause: Throwable = null) extends RuntimeException(message, cause) with Serializable {
|
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 =
|
override lazy val toString =
|
||||||
"%s:%s\n[%s]".format(getClass.getName, message, uuid)
|
"%s:%s\n[%s]".format(getClass.getName, message, uuid)
|
||||||
|
|
|
||||||
|
|
@ -220,7 +220,7 @@ private[akka] class ActorCell(
|
||||||
val ser = SerializationExtension(system)
|
val ser = SerializationExtension(system)
|
||||||
ser.serialize(props.creator) match {
|
ser.serialize(props.creator) match {
|
||||||
case Left(t) ⇒ throw t
|
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 Left(t) ⇒ throw t
|
||||||
case _ ⇒ //All good
|
case _ ⇒ //All good
|
||||||
}
|
}
|
||||||
|
|
@ -285,14 +285,18 @@ private[akka] class ActorCell(
|
||||||
final def isTerminated: Boolean = mailbox.isClosed
|
final def isTerminated: Boolean = mailbox.isClosed
|
||||||
|
|
||||||
final def start(): Unit = {
|
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)
|
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 ⬅⬅⬅
|
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
|
||||||
parent.sendSystemMessage(akka.dispatch.Supervise(self))
|
parent.sendSystemMessage(akka.dispatch.Supervise(self))
|
||||||
|
|
||||||
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
|
// This call is expected to start off the actor by scheduling its mailbox.
|
||||||
dispatcher.systemDispatch(this, Create())
|
|
||||||
|
|
||||||
dispatcher.attach(this)
|
dispatcher.attach(this)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -384,8 +388,6 @@ private[akka] class ActorCell(
|
||||||
failedActor.preRestart(cause, if (c ne null) Some(c.message) else None)
|
failedActor.preRestart(cause, if (c ne null) Some(c.message) else None)
|
||||||
} finally {
|
} finally {
|
||||||
clearActorFields()
|
clearActorFields()
|
||||||
currentMessage = null
|
|
||||||
actor = null
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
actor = freshActor // assign it here so if preStart fails, we can null out the sef-refs next call
|
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))
|
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
|
// prevent any further messages to be processed until the actor has been restarted
|
||||||
dispatcher.suspend(this)
|
dispatcher.suspend(this)
|
||||||
actor.supervisorStrategy.handleSupervisorFailing(self, children)
|
if (actor ne null) actor.supervisorStrategy.handleSupervisorFailing(self, children)
|
||||||
parent.tell(Failed(e), self)
|
parent.tell(Failed(e), self)
|
||||||
} finally {
|
} finally {
|
||||||
checkReceiveTimeout // Reschedule receive timeout
|
checkReceiveTimeout // Reschedule receive timeout
|
||||||
|
|
@ -555,9 +557,8 @@ private[akka] class ActorCell(
|
||||||
if (system.settings.DebugLifecycle)
|
if (system.settings.DebugLifecycle)
|
||||||
system.eventStream.publish(Debug(self.path.toString, clazz(actor), "stopped")) // FIXME: can actor be null?
|
system.eventStream.publish(Debug(self.path.toString, clazz(actor), "stopped")) // FIXME: can actor be null?
|
||||||
} finally {
|
} finally {
|
||||||
currentMessage = null
|
|
||||||
clearActorFields()
|
|
||||||
if (a ne null) a.clearBehaviorStack()
|
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) {
|
final def setActorFields(context: ActorContext, self: ActorRef) {
|
||||||
@tailrec
|
@tailrec
|
||||||
|
|
@ -625,8 +630,5 @@ private[akka] class ActorCell(
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private def clazz(o: AnyRef): Class[_] = {
|
private final def clazz(o: AnyRef): Class[_] = if (o eq null) this.getClass else o.getClass
|
||||||
if (o eq null) this.getClass
|
|
||||||
else o.getClass
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -8,7 +8,7 @@ import akka.dispatch._
|
||||||
import akka.util._
|
import akka.util._
|
||||||
import scala.collection.immutable.Stack
|
import scala.collection.immutable.Stack
|
||||||
import java.lang.{ UnsupportedOperationException, IllegalStateException }
|
import java.lang.{ UnsupportedOperationException, IllegalStateException }
|
||||||
import akka.serialization.Serialization
|
import akka.serialization.{ Serialization, JavaSerializer }
|
||||||
import akka.event.EventStream
|
import akka.event.EventStream
|
||||||
import scala.annotation.tailrec
|
import scala.annotation.tailrec
|
||||||
import java.util.concurrent.{ ConcurrentHashMap }
|
import java.util.concurrent.{ ConcurrentHashMap }
|
||||||
|
|
@ -335,7 +335,7 @@ private[akka] class LocalActorRef private[akka] (
|
||||||
*/
|
*/
|
||||||
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
|
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
|
||||||
case class SerializedActorRef private (path: String) {
|
case class SerializedActorRef private (path: String) {
|
||||||
import akka.serialization.Serialization.currentSystem
|
import akka.serialization.JavaSerializer.currentSystem
|
||||||
|
|
||||||
@throws(classOf[java.io.ObjectStreamException])
|
@throws(classOf[java.io.ObjectStreamException])
|
||||||
def readResolve(): AnyRef = currentSystem.value match {
|
def readResolve(): AnyRef = currentSystem.value match {
|
||||||
|
|
@ -401,7 +401,7 @@ private[akka] object DeadLetterActorRef {
|
||||||
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
|
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
|
||||||
class SerializedDeadLetterActorRef extends Serializable { //TODO implement as Protobuf for performance?
|
class SerializedDeadLetterActorRef extends Serializable { //TODO implement as Protobuf for performance?
|
||||||
@throws(classOf[java.io.ObjectStreamException])
|
@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
|
val serialized = new SerializedDeadLetterActorRef
|
||||||
|
|
|
||||||
|
|
@ -318,12 +318,12 @@ class LocalActorRefProvider(
|
||||||
settings: ActorSystem.Settings,
|
settings: ActorSystem.Settings,
|
||||||
eventStream: EventStream,
|
eventStream: EventStream,
|
||||||
scheduler: Scheduler,
|
scheduler: Scheduler,
|
||||||
classloader: ClassLoader) =
|
dynamicAccess: DynamicAccess) =
|
||||||
this(_systemName,
|
this(_systemName,
|
||||||
settings,
|
settings,
|
||||||
eventStream,
|
eventStream,
|
||||||
scheduler,
|
scheduler,
|
||||||
new Deployer(settings, classloader))
|
new Deployer(settings, dynamicAccess))
|
||||||
|
|
||||||
val rootPath: ActorPath = RootActorPath(Address("akka", _systemName))
|
val rootPath: ActorPath = RootActorPath(Address("akka", _systemName))
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -70,7 +70,6 @@ object ActorSystem {
|
||||||
|
|
||||||
final val CreationTimeout = Timeout(Duration(getMilliseconds("akka.actor.creation-timeout"), MILLISECONDS))
|
final val CreationTimeout = Timeout(Duration(getMilliseconds("akka.actor.creation-timeout"), MILLISECONDS))
|
||||||
final val ReaperInterval = Duration(getMilliseconds("akka.actor.reaper-interval"), 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 SerializeAllMessages = getBoolean("akka.actor.serialize-messages")
|
||||||
final val SerializeAllCreators = getBoolean("akka.actor.serialize-creators")
|
final val SerializeAllCreators = getBoolean("akka.actor.serialize-creators")
|
||||||
|
|
||||||
|
|
@ -325,16 +324,16 @@ abstract class ExtendedActorSystem extends ActorSystem {
|
||||||
def deathWatch: DeathWatch
|
def deathWatch: DeathWatch
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* ClassLoader which is used for reflective accesses internally. This is set
|
* ClassLoader wrapper which is used for reflective accesses internally. This is set
|
||||||
* to the context class loader, if one is set, or the class loader which
|
* 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
|
* loaded the ActorSystem implementation. The context class loader is also
|
||||||
* set on all threads created by the ActorSystem, if one was set during
|
* set on all threads created by the ActorSystem, if one was set during
|
||||||
* creation.
|
* 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+$"""))
|
if (!name.matches("""^\w+$"""))
|
||||||
throw new IllegalArgumentException("invalid ActorSystem name [" + name + "], must contain only word characters (i.e. [a-zA-Z_0-9])")
|
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 =
|
final val threadFactory: MonitorableThreadFactory =
|
||||||
MonitorableThreadFactory(name, settings.Daemonicity, Option(Thread.currentThread.getContextClassLoader), uncaughtExceptionHandler)
|
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)
|
def logConfiguration(): Unit = log.info(settings.toString)
|
||||||
|
|
||||||
protected def systemImpl = this
|
protected def systemImpl = this
|
||||||
|
|
@ -408,17 +436,15 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten
|
||||||
|
|
||||||
val scheduler: Scheduler = createScheduler()
|
val scheduler: Scheduler = createScheduler()
|
||||||
|
|
||||||
val internalClassLoader = Option(Thread.currentThread.getContextClassLoader) getOrElse getClass.getClassLoader
|
|
||||||
|
|
||||||
val provider: ActorRefProvider = {
|
val provider: ActorRefProvider = {
|
||||||
val arguments = Seq(
|
val arguments = Seq(
|
||||||
classOf[String] -> name,
|
classOf[String] -> name,
|
||||||
classOf[Settings] -> settings,
|
classOf[Settings] -> settings,
|
||||||
classOf[EventStream] -> eventStream,
|
classOf[EventStream] -> eventStream,
|
||||||
classOf[Scheduler] -> scheduler,
|
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 Left(e) ⇒ throw e
|
||||||
case Right(p) ⇒ p
|
case Right(p) ⇒ p
|
||||||
}
|
}
|
||||||
|
|
@ -440,7 +466,7 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten
|
||||||
def locker: Locker = provider.locker
|
def locker: Locker = provider.locker
|
||||||
|
|
||||||
val dispatchers: Dispatchers = new Dispatchers(settings, DefaultDispatcherPrerequisites(
|
val dispatchers: Dispatchers = new Dispatchers(settings, DefaultDispatcherPrerequisites(
|
||||||
threadFactory, eventStream, deadLetterMailbox, scheduler, internalClassLoader))
|
threadFactory, eventStream, deadLetterMailbox, scheduler, dynamicAccess))
|
||||||
|
|
||||||
val dispatcher: MessageDispatcher = dispatchers.defaultGlobalDispatcher
|
val dispatcher: MessageDispatcher = dispatchers.defaultGlobalDispatcher
|
||||||
|
|
||||||
|
|
@ -559,8 +585,7 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten
|
||||||
private def loadExtensions() {
|
private def loadExtensions() {
|
||||||
import scala.collection.JavaConversions._
|
import scala.collection.JavaConversions._
|
||||||
settings.config.getStringList("akka.extensions") foreach { fqcn ⇒
|
settings.config.getStringList("akka.extensions") foreach { fqcn ⇒
|
||||||
import ReflectiveAccess.{ getObjectFor, createInstance, noParams, noArgs }
|
dynamicAccess.getObjectFor[AnyRef](fqcn).fold(_ ⇒ dynamicAccess.createInstanceFor[AnyRef](fqcn, Seq()), Right(_)) match {
|
||||||
getObjectFor[AnyRef](fqcn, internalClassLoader).fold(_ ⇒ createInstance[AnyRef](fqcn, noParams, noArgs), Right(_)) match {
|
|
||||||
case Right(p: ExtensionIdProvider) ⇒ registerExtension(p.lookup());
|
case Right(p: ExtensionIdProvider) ⇒ registerExtension(p.lookup());
|
||||||
case Right(p: ExtensionId[_]) ⇒ registerExtension(p);
|
case Right(p: ExtensionId[_]) ⇒ registerExtension(p);
|
||||||
case Right(other) ⇒ log.error("[{}] is not an 'ExtensionIdProvider' or 'ExtensionId', skipping...", fqcn)
|
case Right(other) ⇒ log.error("[{}] is not an 'ExtensionIdProvider' or 'ExtensionId', skipping...", fqcn)
|
||||||
|
|
|
||||||
|
|
@ -15,7 +15,7 @@ import java.net.MalformedURLException
|
||||||
* for example a remote transport would want to associate additional
|
* for example a remote transport would want to associate additional
|
||||||
* information with an address, then this must be done externally.
|
* 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) = this(protocol, system, None, None)
|
||||||
def this(protocol: String, system: String, host: String, port: Int) = this(protocol, system, Option(host), Some(port))
|
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.
|
* This object serves as extractor for Scala and as address parser for Java.
|
||||||
*/
|
*/
|
||||||
object AddressExtractor {
|
object AddressExtractor {
|
||||||
def unapply(addr: String): Option[Address] = {
|
def unapply(addr: String): Option[Address] =
|
||||||
try {
|
try {
|
||||||
val uri = new URI(addr)
|
val uri = new URI(addr)
|
||||||
if (uri.getScheme == null || (uri.getUserInfo == null && uri.getHost == null)) None
|
unapply(uri)
|
||||||
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)
|
|
||||||
}
|
|
||||||
} catch {
|
} catch {
|
||||||
case _: URISyntaxException ⇒ None
|
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.
|
* Try to construct an Address from the given String or throw a java.net.MalformedURLException.
|
||||||
|
|
@ -92,18 +98,15 @@ object AddressExtractor {
|
||||||
}
|
}
|
||||||
|
|
||||||
object ActorPathExtractor {
|
object ActorPathExtractor {
|
||||||
def unapply(addr: String): Option[(Address, Iterable[String])] = {
|
def unapply(addr: String): Option[(Address, Iterable[String])] =
|
||||||
try {
|
try {
|
||||||
val uri = new URI(addr)
|
val uri = new URI(addr)
|
||||||
if (uri.getScheme == null || (uri.getUserInfo == null && uri.getHost == null) || uri.getPath == null) None
|
if (uri.getPath == null) None
|
||||||
else {
|
else AddressExtractor.unapply(uri) match {
|
||||||
val addr = Address(uri.getScheme, if (uri.getUserInfo != null) uri.getUserInfo else uri.getHost,
|
case None ⇒ None
|
||||||
if (uri.getUserInfo == null || uri.getHost == null) None else Some(uri.getHost),
|
case Some(addr) ⇒ Some((addr, ActorPath.split(uri.getPath).drop(1)))
|
||||||
if (uri.getPort < 0) None else Some(uri.getPort))
|
|
||||||
Some((addr, ActorPath.split(uri.getPath).drop(1)))
|
|
||||||
}
|
}
|
||||||
} catch {
|
} catch {
|
||||||
case _: URISyntaxException ⇒ None
|
case _: URISyntaxException ⇒ None
|
||||||
}
|
}
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
@ -8,7 +8,6 @@ import akka.util.Duration
|
||||||
import com.typesafe.config._
|
import com.typesafe.config._
|
||||||
import akka.routing._
|
import akka.routing._
|
||||||
import java.util.concurrent.{ TimeUnit, ConcurrentHashMap }
|
import java.util.concurrent.{ TimeUnit, ConcurrentHashMap }
|
||||||
import akka.util.ReflectiveAccess
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This class represents deployment configuration for a given actor path. It is
|
* 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ér</a>
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||||
*/
|
*/
|
||||||
class Deployer(val settings: ActorSystem.Settings, val classloader: ClassLoader) {
|
class Deployer(val settings: ActorSystem.Settings, val dynamicAccess: DynamicAccess) {
|
||||||
|
|
||||||
import scala.collection.JavaConverters._
|
import scala.collection.JavaConverters._
|
||||||
|
|
||||||
|
|
@ -128,7 +127,7 @@ class Deployer(val settings: ActorSystem.Settings, val classloader: ClassLoader)
|
||||||
case "broadcast" ⇒ BroadcastRouter(nrOfInstances, routees, resizer)
|
case "broadcast" ⇒ BroadcastRouter(nrOfInstances, routees, resizer)
|
||||||
case fqn ⇒
|
case fqn ⇒
|
||||||
val args = Seq(classOf[Config] -> deployment)
|
val args = Seq(classOf[Config] -> deployment)
|
||||||
ReflectiveAccess.createInstance[RouterConfig](fqn, args, classloader) match {
|
dynamicAccess.createInstanceFor[RouterConfig](fqn, args) match {
|
||||||
case Right(router) ⇒ router
|
case Right(router) ⇒ router
|
||||||
case Left(exception) ⇒
|
case Left(exception) ⇒
|
||||||
throw new IllegalArgumentException(
|
throw new IllegalArgumentException(
|
||||||
|
|
|
||||||
129
akka-actor/src/main/scala/akka/actor/DynamicAccess.scala
Normal file
129
akka-actor/src/main/scala/akka/actor/DynamicAccess.scala
Normal 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])
|
||||||
|
}
|
||||||
|
}
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -3,8 +3,6 @@
|
||||||
*/
|
*/
|
||||||
package akka.actor
|
package akka.actor
|
||||||
|
|
||||||
import akka.util.ReflectiveAccess
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The basic ActorSystem covers all that is needed for locally running actors,
|
* 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
|
* 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
|
* 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]
|
* 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 {
|
* public class MyExt extends Extension {
|
||||||
* static final ExtensionKey<MyExt> key = new ExtensionKey<MyExt>(MyExt.class);
|
* 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
|
override def lookup(): ExtensionId[T] = this
|
||||||
def createExtension(system: ExtendedActorSystem): T =
|
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 Left(ex) ⇒ throw ex
|
||||||
case Right(r) ⇒ r
|
case Right(r) ⇒ r
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -11,7 +11,9 @@ import java.util.concurrent.atomic.{ AtomicReference ⇒ AtomVar }
|
||||||
import akka.serialization.{ Serialization, SerializationExtension }
|
import akka.serialization.{ Serialization, SerializationExtension }
|
||||||
import akka.dispatch._
|
import akka.dispatch._
|
||||||
import java.util.concurrent.TimeoutException
|
import java.util.concurrent.TimeoutException
|
||||||
|
import java.util.concurrent.TimeUnit.MILLISECONDS
|
||||||
import java.lang.IllegalStateException
|
import java.lang.IllegalStateException
|
||||||
|
import akka.util.Duration
|
||||||
|
|
||||||
trait TypedActorFactory {
|
trait TypedActorFactory {
|
||||||
|
|
||||||
|
|
@ -129,7 +131,8 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi
|
||||||
val serializedParameters = Array.ofDim[(Int, Class[_], Array[Byte])](ps.length)
|
val serializedParameters = Array.ofDim[(Int, Class[_], Array[Byte])](ps.length)
|
||||||
for (i ← 0 until ps.length) {
|
for (i ← 0 until ps.length) {
|
||||||
val p = ps(i)
|
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
|
val m = if (s.includeManifest) p.getClass else null
|
||||||
serializedParameters(i) = (s.identifier, m, s toBinary parameters(i)) //Mutable for the sake of sanity
|
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 implement writeObject and readObject to serialize
|
||||||
//TODO Possible optimization is to special encode the parameter-types to conserve space
|
//TODO Possible optimization is to special encode the parameter-types to conserve space
|
||||||
private def readResolve(): AnyRef = {
|
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(
|
if (system eq null) throw new IllegalStateException(
|
||||||
"Trying to deserialize a SerializedMethodCall without an ActorSystem in scope." +
|
"Trying to deserialize a SerializedMethodCall without an ActorSystem in scope." +
|
||||||
" Use akka.serialization.Serialization.currentSystem.withValue(system) { ... }")
|
" 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
|
val deserializedParameters: Array[AnyRef] = Array.ofDim[AnyRef](a.length) //Mutable for the sake of sanity
|
||||||
for (i ← 0 until a.length) {
|
for (i ← 0 until a.length) {
|
||||||
val (sId, manifest, bytes) = a(i)
|
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
|
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,
|
* @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
|
* 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,
|
* @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
|
* Scala API
|
||||||
*/
|
*/
|
||||||
|
|
@ -548,6 +552,11 @@ class TypedActorExtension(system: ExtendedActorSystem) extends TypedActorFactory
|
||||||
val serialization = SerializationExtension(system)
|
val serialization = SerializationExtension(system)
|
||||||
val settings = system.settings
|
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
|
* 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(
|
new TypedActorInvocationHandler(
|
||||||
this,
|
this,
|
||||||
actorVar,
|
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 {
|
proxyVar match {
|
||||||
case null ⇒
|
case null ⇒
|
||||||
|
|
|
||||||
|
|
@ -12,7 +12,6 @@ import akka.actor.ActorSystem
|
||||||
import scala.annotation.tailrec
|
import scala.annotation.tailrec
|
||||||
import akka.event.EventStream
|
import akka.event.EventStream
|
||||||
import com.typesafe.config.Config
|
import com.typesafe.config.Config
|
||||||
import akka.util.ReflectiveAccess
|
|
||||||
import akka.serialization.SerializationExtension
|
import akka.serialization.SerializationExtension
|
||||||
import akka.util.NonFatal
|
import akka.util.NonFatal
|
||||||
import akka.event.Logging.LogEventException
|
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)
|
val ser = SerializationExtension(system)
|
||||||
ser.serialize(msg) match { //Verify serializability
|
ser.serialize(msg) match { //Verify serializability
|
||||||
case Left(t) ⇒ throw t
|
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 Left(t) ⇒ throw t
|
||||||
case _ ⇒ //All good
|
case _ ⇒ //All good
|
||||||
}
|
}
|
||||||
|
|
@ -157,7 +156,10 @@ trait ExecutionContext {
|
||||||
* log the problem or whatever is appropriate for the implementation.
|
* log the problem or whatever is appropriate for the implementation.
|
||||||
*/
|
*/
|
||||||
def reportFailure(t: Throwable): Unit
|
def reportFailure(t: Throwable): Unit
|
||||||
|
}
|
||||||
|
|
||||||
|
private[akka] trait LoadMetrics { self: Executor ⇒
|
||||||
|
def atFullThrottle(): Boolean
|
||||||
}
|
}
|
||||||
|
|
||||||
object MessageDispatcher {
|
object MessageDispatcher {
|
||||||
|
|
@ -186,9 +188,14 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
|
||||||
def id: String
|
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
|
* 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 (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) {
|
protected[akka] def register(actor: ActorCell) {
|
||||||
inhabitantsUpdater.incrementAndGet(this)
|
inhabitantsUpdater.incrementAndGet(this)
|
||||||
|
|
@ -261,6 +268,8 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
|
||||||
mailBox.cleanUp()
|
mailBox.cleanUp()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
def inhabitants: Long = inhabitantsUpdater.get(this)
|
||||||
|
|
||||||
private val shutdownAction = new Runnable {
|
private val shutdownAction = new Runnable {
|
||||||
@tailrec
|
@tailrec
|
||||||
final def run() {
|
final def run() {
|
||||||
|
|
@ -369,7 +378,7 @@ abstract class MessageDispatcherConfigurator(val config: Config, val prerequisit
|
||||||
}
|
}
|
||||||
case fqcn ⇒
|
case fqcn ⇒
|
||||||
val args = Seq(classOf[Config] -> config)
|
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 Right(instance) ⇒ instance
|
||||||
case Left(exception) ⇒
|
case Left(exception) ⇒
|
||||||
throw new IllegalArgumentException(
|
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 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 "thread-pool-executor" ⇒ new ThreadPoolExecutorConfigurator(config.getConfig("thread-pool-executor"), prerequisites)
|
||||||
case fqcn ⇒
|
case fqcn ⇒
|
||||||
val constructorSignature = Array[Class[_]](classOf[Config], classOf[DispatcherPrerequisites])
|
val args = Seq(
|
||||||
ReflectiveAccess.createInstance[ExecutorServiceConfigurator](fqcn, constructorSignature, Array[AnyRef](config, prerequisites), prerequisites.classloader) match {
|
classOf[Config] -> config,
|
||||||
|
classOf[DispatcherPrerequisites] -> prerequisites)
|
||||||
|
prerequisites.dynamicAccess.createInstanceFor[ExecutorServiceConfigurator](fqcn, args) match {
|
||||||
case Right(instance) ⇒ instance
|
case Right(instance) ⇒ instance
|
||||||
case Left(exception) ⇒ throw new IllegalArgumentException(
|
case Left(exception) ⇒ throw new IllegalArgumentException(
|
||||||
("""Cannot instantiate ExecutorServiceConfigurator ("executor = [%s]"), defined in [%s],
|
("""Cannot instantiate ExecutorServiceConfigurator ("executor = [%s]"), defined in [%s],
|
||||||
|
|
@ -439,11 +450,13 @@ object ForkJoinExecutorConfigurator {
|
||||||
final class AkkaForkJoinPool(parallelism: Int,
|
final class AkkaForkJoinPool(parallelism: Int,
|
||||||
threadFactory: ForkJoinPool.ForkJoinWorkerThreadFactory,
|
threadFactory: ForkJoinPool.ForkJoinWorkerThreadFactory,
|
||||||
unhandledExceptionHandler: Thread.UncaughtExceptionHandler)
|
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 {
|
override def execute(r: Runnable): Unit = r match {
|
||||||
case m: Mailbox ⇒ super.execute(new MailboxExecutionTask(m))
|
case m: Mailbox ⇒ super.execute(new MailboxExecutionTask(m))
|
||||||
case other ⇒ super.execute(other)
|
case other ⇒ super.execute(other)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
def atFullThrottle(): Boolean = this.getActiveThreadCount() >= this.getParallelism()
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -4,12 +4,11 @@
|
||||||
|
|
||||||
package akka.dispatch
|
package akka.dispatch
|
||||||
|
|
||||||
import util.DynamicVariable
|
|
||||||
import akka.actor.{ ActorCell, ActorRef }
|
import akka.actor.{ ActorCell, ActorRef }
|
||||||
import java.util.concurrent.{ LinkedBlockingQueue, ConcurrentLinkedQueue, ConcurrentSkipListSet }
|
|
||||||
import annotation.tailrec
|
import annotation.tailrec
|
||||||
import java.util.concurrent.atomic.AtomicBoolean
|
import akka.util.{ Duration, Helpers }
|
||||||
import akka.util.Duration
|
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
|
* 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,
|
throughputDeadlineTime: Duration,
|
||||||
mailboxType: MailboxType,
|
mailboxType: MailboxType,
|
||||||
_executorServiceFactoryProvider: ExecutorServiceFactoryProvider,
|
_executorServiceFactoryProvider: ExecutorServiceFactoryProvider,
|
||||||
_shutdownTimeout: Duration)
|
_shutdownTimeout: Duration,
|
||||||
|
attemptTeamWork: Boolean)
|
||||||
extends Dispatcher(_prerequisites, _id, throughput, throughputDeadlineTime, mailboxType, _executorServiceFactoryProvider, _shutdownTimeout) {
|
extends Dispatcher(_prerequisites, _id, throughput, throughputDeadlineTime, mailboxType, _executorServiceFactoryProvider, _shutdownTimeout) {
|
||||||
|
|
||||||
val buddies = new ConcurrentSkipListSet[ActorCell](akka.util.Helpers.IdentityHashComparator)
|
val team = new ConcurrentSkipListSet[ActorCell](
|
||||||
val rebalance = new AtomicBoolean(false)
|
Helpers.identityHashComparator(new Comparator[ActorCell] {
|
||||||
|
def compare(l: ActorCell, r: ActorCell) = l.self.path compareTo r.self.path
|
||||||
|
}))
|
||||||
|
|
||||||
val messageQueue: MessageQueue = mailboxType match {
|
val messageQueue: MessageQueue = mailboxType match {
|
||||||
case u: UnboundedMailbox ⇒ new QueueBasedMessageQueue with UnboundedMessageQueueSemantics {
|
case UnboundedMailbox() ⇒
|
||||||
final val queue = new ConcurrentLinkedQueue[Envelope]
|
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 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 + "]")
|
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)
|
super.register(actor)
|
||||||
buddies.add(actor)
|
team.add(actor)
|
||||||
}
|
}
|
||||||
|
|
||||||
protected[akka] override def unregister(actor: ActorCell) = {
|
protected[akka] override def unregister(actor: ActorCell): Unit = {
|
||||||
buddies.remove(actor)
|
team.remove(actor)
|
||||||
super.unregister(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) = {
|
override protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope) = {
|
||||||
messageQueue.enqueue(receiver.self, invocation)
|
messageQueue.enqueue(receiver.self, invocation)
|
||||||
registerForExecution(receiver.mailbox, false, false)
|
if (!registerForExecution(receiver.mailbox, false, false)) teamWork()
|
||||||
intoTheFray(except = receiver)
|
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
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()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -32,12 +32,11 @@ class Dispatcher(
|
||||||
val shutdownTimeout: Duration)
|
val shutdownTimeout: Duration)
|
||||||
extends MessageDispatcher(_prerequisites) {
|
extends MessageDispatcher(_prerequisites) {
|
||||||
|
|
||||||
protected[akka] val executorServiceFactory: ExecutorServiceFactory =
|
protected val executorServiceFactory: ExecutorServiceFactory =
|
||||||
executorServiceFactoryProvider.createExecutorServiceFactory(id, prerequisites.threadFactory)
|
executorServiceFactoryProvider.createExecutorServiceFactory(id, prerequisites.threadFactory)
|
||||||
|
|
||||||
protected[akka] val executorService = new AtomicReference[ExecutorService](new ExecutorServiceDelegate {
|
protected val executorService = new AtomicReference[ExecutorServiceDelegate](
|
||||||
lazy val executor = executorServiceFactory.createExecutorService
|
new ExecutorServiceDelegate { lazy val executor = executorServiceFactory.createExecutorService })
|
||||||
})
|
|
||||||
|
|
||||||
protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope) = {
|
protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope) = {
|
||||||
val mbox = receiver.mailbox
|
val mbox = receiver.mailbox
|
||||||
|
|
|
||||||
|
|
@ -4,22 +4,24 @@
|
||||||
|
|
||||||
package akka.dispatch
|
package akka.dispatch
|
||||||
|
|
||||||
import akka.actor.newUuid
|
import java.util.concurrent.{ ConcurrentHashMap, TimeUnit, ThreadFactory }
|
||||||
import akka.util.{ Duration, ReflectiveAccess }
|
|
||||||
import akka.actor.ActorSystem
|
import scala.collection.JavaConverters.mapAsJavaMapConverter
|
||||||
import akka.event.EventStream
|
|
||||||
import akka.actor.Scheduler
|
import com.typesafe.config.{ ConfigFactory, Config }
|
||||||
import com.typesafe.config.Config
|
|
||||||
import com.typesafe.config.ConfigFactory
|
import Dispatchers.DefaultDispatcherId
|
||||||
|
import akka.actor.{ Scheduler, DynamicAccess, ActorSystem }
|
||||||
import akka.event.Logging.Warning
|
import akka.event.Logging.Warning
|
||||||
import java.util.concurrent.{ ThreadFactory, TimeUnit, ConcurrentHashMap }
|
import akka.event.EventStream
|
||||||
|
import akka.util.Duration
|
||||||
|
|
||||||
trait DispatcherPrerequisites {
|
trait DispatcherPrerequisites {
|
||||||
def threadFactory: ThreadFactory
|
def threadFactory: ThreadFactory
|
||||||
def eventStream: EventStream
|
def eventStream: EventStream
|
||||||
def deadLetterMailbox: Mailbox
|
def deadLetterMailbox: Mailbox
|
||||||
def scheduler: Scheduler
|
def scheduler: Scheduler
|
||||||
def classloader: ClassLoader
|
def dynamicAccess: DynamicAccess
|
||||||
}
|
}
|
||||||
|
|
||||||
case class DefaultDispatcherPrerequisites(
|
case class DefaultDispatcherPrerequisites(
|
||||||
|
|
@ -27,7 +29,7 @@ case class DefaultDispatcherPrerequisites(
|
||||||
val eventStream: EventStream,
|
val eventStream: EventStream,
|
||||||
val deadLetterMailbox: Mailbox,
|
val deadLetterMailbox: Mailbox,
|
||||||
val scheduler: Scheduler,
|
val scheduler: Scheduler,
|
||||||
val classloader: ClassLoader) extends DispatcherPrerequisites
|
val dynamicAccess: DynamicAccess) extends DispatcherPrerequisites
|
||||||
|
|
||||||
object Dispatchers {
|
object Dispatchers {
|
||||||
/**
|
/**
|
||||||
|
|
@ -137,7 +139,7 @@ class Dispatchers(val settings: ActorSystem.Settings, val prerequisites: Dispatc
|
||||||
case "PinnedDispatcher" ⇒ new PinnedDispatcherConfigurator(cfg, prerequisites)
|
case "PinnedDispatcher" ⇒ new PinnedDispatcherConfigurator(cfg, prerequisites)
|
||||||
case fqn ⇒
|
case fqn ⇒
|
||||||
val args = Seq(classOf[Config] -> cfg, classOf[DispatcherPrerequisites] -> prerequisites)
|
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 Right(configurator) ⇒ configurator
|
||||||
case Left(exception) ⇒
|
case Left(exception) ⇒
|
||||||
throw new IllegalArgumentException(
|
throw new IllegalArgumentException(
|
||||||
|
|
@ -187,7 +189,8 @@ class BalancingDispatcherConfigurator(config: Config, prerequisites: DispatcherP
|
||||||
config.getInt("throughput"),
|
config.getInt("throughput"),
|
||||||
Duration(config.getNanoseconds("throughput-deadline-time"), TimeUnit.NANOSECONDS),
|
Duration(config.getNanoseconds("throughput-deadline-time"), TimeUnit.NANOSECONDS),
|
||||||
mailboxType, configureExecutor(),
|
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
|
* Returns the same dispatcher instance for each invocation
|
||||||
|
|
|
||||||
|
|
@ -20,6 +20,7 @@ import akka.event.Logging.Debug
|
||||||
import java.util.concurrent.TimeUnit.NANOSECONDS
|
import java.util.concurrent.TimeUnit.NANOSECONDS
|
||||||
import java.util.concurrent.{ ExecutionException, Callable, TimeoutException }
|
import java.util.concurrent.{ ExecutionException, Callable, TimeoutException }
|
||||||
import java.util.concurrent.atomic.{ AtomicInteger, AtomicReferenceFieldUpdater }
|
import java.util.concurrent.atomic.{ AtomicInteger, AtomicReferenceFieldUpdater }
|
||||||
|
import akka.pattern.AskTimeoutException
|
||||||
|
|
||||||
object Await {
|
object Await {
|
||||||
|
|
||||||
|
|
@ -151,6 +152,26 @@ object Futures {
|
||||||
for (r ← fr; b ← fb) yield { r add b; r }
|
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 {
|
object Future {
|
||||||
|
|
@ -317,17 +338,22 @@ object Future {
|
||||||
* }
|
* }
|
||||||
* </pre>
|
* </pre>
|
||||||
*/
|
*/
|
||||||
def blocking(implicit executor: ExecutionContext): Unit =
|
def blocking(): Unit =
|
||||||
_taskStack.get match {
|
_taskStack.get match {
|
||||||
case stack if (stack ne null) && stack.nonEmpty ⇒
|
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
|
val tasks = stack.elems
|
||||||
stack.clear()
|
stack.clear()
|
||||||
_taskStack.remove()
|
_taskStack.remove()
|
||||||
dispatchTask(() ⇒ _taskStack.get.elems = tasks, true)
|
dispatchTask(() ⇒ _taskStack.get.elems = tasks, true)(executionContext)
|
||||||
case _ ⇒ _taskStack.remove()
|
case _ ⇒ _taskStack.remove()
|
||||||
}
|
}
|
||||||
|
|
||||||
private val _taskStack = new ThreadLocal[Stack[() ⇒ Unit]]()
|
private val _taskStack = new ThreadLocal[Stack[() ⇒ Unit]]()
|
||||||
|
private val _executionContext = new ThreadLocal[ExecutionContext]()
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Internal API, do not call
|
* Internal API, do not call
|
||||||
|
|
@ -339,7 +365,7 @@ object Future {
|
||||||
new Runnable {
|
new Runnable {
|
||||||
def run =
|
def run =
|
||||||
try {
|
try {
|
||||||
|
_executionContext set executor
|
||||||
val taskStack = Stack.empty[() ⇒ Unit]
|
val taskStack = Stack.empty[() ⇒ Unit]
|
||||||
taskStack push task
|
taskStack push task
|
||||||
_taskStack set taskStack
|
_taskStack set taskStack
|
||||||
|
|
@ -352,7 +378,10 @@ object Future {
|
||||||
case NonFatal(e) ⇒ executor.reportFailure(e)
|
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 =
|
def result(atMost: Duration)(implicit permit: CanAwait): T =
|
||||||
ready(atMost).value.get match {
|
ready(atMost).value.get match {
|
||||||
case Left(e) ⇒ throw e
|
case Left(e: AskTimeoutException) ⇒ throw new AskTimeoutException(e.getMessage, e) // to get meaningful stack trace
|
||||||
case Right(r) ⇒ r
|
case Left(e) ⇒ throw e
|
||||||
|
case Right(r) ⇒ r
|
||||||
}
|
}
|
||||||
|
|
||||||
def value: Option[Either[Throwable, T]] = getState match {
|
def value: Option[Either[Throwable, T]] = getState match {
|
||||||
|
|
|
||||||
|
|
@ -189,8 +189,11 @@ private[akka] abstract class Mailbox(val actor: ActorCell) extends MessageQueue
|
||||||
final def processAllSystemMessages() {
|
final def processAllSystemMessages() {
|
||||||
var nextMessage = systemDrain()
|
var nextMessage = systemDrain()
|
||||||
try {
|
try {
|
||||||
while (nextMessage ne null) {
|
while ((nextMessage ne null) && !isClosed) {
|
||||||
if (debug) println(actor.self + " processing system message " + nextMessage + " with children " + actor.childrenRefs)
|
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
|
actor systemInvoke nextMessage
|
||||||
nextMessage = nextMessage.next
|
nextMessage = nextMessage.next
|
||||||
// don’t ever execute normal message when system message present!
|
// don’t ever execute normal message when system message present!
|
||||||
|
|
@ -236,15 +239,26 @@ private[akka] abstract class Mailbox(val actor: ActorCell) extends MessageQueue
|
||||||
}
|
}
|
||||||
|
|
||||||
trait 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
|
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
|
def numberOfMessages: Int
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Indicates whether this queue is non-empty.
|
||||||
|
*/
|
||||||
def hasMessages: Boolean
|
def hasMessages: Boolean
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -292,15 +306,15 @@ trait DefaultSystemMessageQueue { self: Mailbox ⇒
|
||||||
}
|
}
|
||||||
|
|
||||||
trait UnboundedMessageQueueSemantics extends QueueBasedMessageQueue {
|
trait UnboundedMessageQueueSemantics extends QueueBasedMessageQueue {
|
||||||
final def enqueue(receiver: ActorRef, handle: Envelope): Unit = queue add handle
|
def enqueue(receiver: ActorRef, handle: Envelope): Unit = queue add handle
|
||||||
final def dequeue(): Envelope = queue.poll()
|
def dequeue(): Envelope = queue.poll()
|
||||||
}
|
}
|
||||||
|
|
||||||
trait BoundedMessageQueueSemantics extends QueueBasedMessageQueue {
|
trait BoundedMessageQueueSemantics extends QueueBasedMessageQueue {
|
||||||
def pushTimeOut: Duration
|
def pushTimeOut: Duration
|
||||||
override def queue: BlockingQueue[Envelope]
|
override def queue: BlockingQueue[Envelope]
|
||||||
|
|
||||||
final def enqueue(receiver: ActorRef, handle: Envelope) {
|
def enqueue(receiver: ActorRef, handle: Envelope) {
|
||||||
if (pushTimeOut.length > 0) {
|
if (pushTimeOut.length > 0) {
|
||||||
queue.offer(handle, pushTimeOut.length, pushTimeOut.unit) || {
|
queue.offer(handle, pushTimeOut.length, pushTimeOut.unit) || {
|
||||||
throw new MessageQueueAppendFailedException("Couldn't enqueue message " + handle + " to " + receiver)
|
throw new MessageQueueAppendFailedException("Couldn't enqueue message " + handle + " to " + receiver)
|
||||||
|
|
@ -308,13 +322,13 @@ trait BoundedMessageQueueSemantics extends QueueBasedMessageQueue {
|
||||||
} else queue put handle
|
} else queue put handle
|
||||||
}
|
}
|
||||||
|
|
||||||
final def dequeue(): Envelope = queue.poll()
|
def dequeue(): Envelope = queue.poll()
|
||||||
}
|
}
|
||||||
|
|
||||||
trait QueueBasedMessageQueue extends MessageQueue {
|
trait QueueBasedMessageQueue extends MessageQueue {
|
||||||
def queue: Queue[Envelope]
|
def queue: Queue[Envelope]
|
||||||
final def numberOfMessages = queue.size
|
def numberOfMessages = queue.size
|
||||||
final def hasMessages = !queue.isEmpty
|
def hasMessages = !queue.isEmpty
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -81,14 +81,16 @@ case class ThreadPoolConfig(allowCorePoolTimeout: Boolean = ThreadPoolConfig.def
|
||||||
extends ExecutorServiceFactoryProvider {
|
extends ExecutorServiceFactoryProvider {
|
||||||
class ThreadPoolExecutorServiceFactory(val threadFactory: ThreadFactory) extends ExecutorServiceFactory {
|
class ThreadPoolExecutorServiceFactory(val threadFactory: ThreadFactory) extends ExecutorServiceFactory {
|
||||||
def createExecutorService: ExecutorService = {
|
def createExecutorService: ExecutorService = {
|
||||||
val service = new ThreadPoolExecutor(
|
val service: ThreadPoolExecutor = new ThreadPoolExecutor(
|
||||||
corePoolSize,
|
corePoolSize,
|
||||||
maxPoolSize,
|
maxPoolSize,
|
||||||
threadTimeout.length,
|
threadTimeout.length,
|
||||||
threadTimeout.unit,
|
threadTimeout.unit,
|
||||||
queueFactory(),
|
queueFactory(),
|
||||||
threadFactory,
|
threadFactory,
|
||||||
rejectionPolicy)
|
rejectionPolicy) with LoadMetrics {
|
||||||
|
def atFullThrottle(): Boolean = this.getActiveCount >= this.getPoolSize
|
||||||
|
}
|
||||||
service.allowCoreThreadTimeOut(allowCorePoolTimeout)
|
service.allowCoreThreadTimeOut(allowCorePoolTimeout)
|
||||||
service
|
service
|
||||||
}
|
}
|
||||||
|
|
@ -182,7 +184,7 @@ case class MonitorableThreadFactory(name: String,
|
||||||
protected def wire[T <: Thread](t: T): T = {
|
protected def wire[T <: Thread](t: T): T = {
|
||||||
t.setUncaughtExceptionHandler(exceptionHandler)
|
t.setUncaughtExceptionHandler(exceptionHandler)
|
||||||
t.setDaemon(daemonic)
|
t.setDaemon(daemonic)
|
||||||
contextClassLoader foreach (t.setContextClassLoader(_))
|
contextClassLoader foreach t.setContextClassLoader
|
||||||
t
|
t
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -6,7 +6,6 @@ package akka.event
|
||||||
import akka.actor._
|
import akka.actor._
|
||||||
import akka.AkkaException
|
import akka.AkkaException
|
||||||
import akka.actor.ActorSystem.Settings
|
import akka.actor.ActorSystem.Settings
|
||||||
import akka.util.ReflectiveAccess
|
|
||||||
import akka.config.ConfigurationException
|
import akka.config.ConfigurationException
|
||||||
import akka.util.ReentrantGuard
|
import akka.util.ReentrantGuard
|
||||||
import akka.util.duration._
|
import akka.util.duration._
|
||||||
|
|
@ -101,7 +100,7 @@ trait LoggingBus extends ActorEventBus {
|
||||||
if loggerName != StandardOutLoggerName
|
if loggerName != StandardOutLoggerName
|
||||||
} yield {
|
} yield {
|
||||||
try {
|
try {
|
||||||
ReflectiveAccess.getClassFor[Actor](loggerName, system.internalClassLoader) match {
|
system.dynamicAccess.getClassFor[Actor](loggerName) match {
|
||||||
case Right(actorClass) ⇒ addLogger(system, actorClass, level, logName)
|
case Right(actorClass) ⇒ addLogger(system, actorClass, level, logName)
|
||||||
case Left(exception) ⇒ throw exception
|
case Left(exception) ⇒ throw exception
|
||||||
}
|
}
|
||||||
|
|
@ -350,7 +349,7 @@ object Logging {
|
||||||
|
|
||||||
object Extension extends ExtensionKey[LogExt]
|
object Extension extends ExtensionKey[LogExt]
|
||||||
|
|
||||||
class LogExt(system: ActorSystemImpl) extends Extension {
|
class LogExt(system: ExtendedActorSystem) extends Extension {
|
||||||
private val loggerId = new AtomicInteger
|
private val loggerId = new AtomicInteger
|
||||||
def id() = loggerId.incrementAndGet()
|
def id() = loggerId.incrementAndGet()
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -9,15 +9,14 @@ import akka.util.duration._
|
||||||
import akka.config.ConfigurationException
|
import akka.config.ConfigurationException
|
||||||
import akka.pattern.pipe
|
import akka.pattern.pipe
|
||||||
import akka.pattern.AskSupport
|
import akka.pattern.AskSupport
|
||||||
|
|
||||||
import com.typesafe.config.Config
|
import com.typesafe.config.Config
|
||||||
|
|
||||||
import scala.collection.JavaConversions.iterableAsScalaIterable
|
import scala.collection.JavaConversions.iterableAsScalaIterable
|
||||||
|
|
||||||
import java.util.concurrent.atomic.{ AtomicLong, AtomicBoolean }
|
import java.util.concurrent.atomic.{ AtomicLong, AtomicBoolean }
|
||||||
import java.util.concurrent.TimeUnit
|
import java.util.concurrent.TimeUnit
|
||||||
|
import java.util.concurrent.locks.ReentrantLock
|
||||||
import akka.jsr166y.ThreadLocalRandom
|
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
|
* 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)
|
private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _supervisor: InternalActorRef, _path: ActorPath)
|
||||||
extends LocalActorRef(
|
extends LocalActorRef(
|
||||||
_system,
|
_system,
|
||||||
_props.copy(creator = () ⇒ _props.routerConfig.createActor()),
|
_props.copy(creator = () ⇒ _props.routerConfig.createActor(), dispatcher = _props.routerConfig.routerDispatcher),
|
||||||
_supervisor,
|
_supervisor,
|
||||||
_path) {
|
_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
|
private val resizeCounter = new AtomicLong
|
||||||
|
|
||||||
@volatile
|
@volatile
|
||||||
private var _routees: IndexedSeq[ActorRef] = IndexedSeq.empty[ActorRef] // this MUST be initialized during createRoute
|
private var _routees: IndexedSeq[ActorRef] = IndexedSeq.empty[ActorRef] // this MUST be initialized during createRoute
|
||||||
def routees = _routees
|
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 Router’s 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 the routees to existing routees.
|
||||||
* Adds death watch of the routees so that they are removed when terminated.
|
* 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
|
* Not thread safe, but intended to be called from protected points, such as
|
||||||
* `RouterConfig.createRoute` and `Resizer.resize`
|
* `RouterConfig.createRoute` and `Resizer.resize`
|
||||||
*/
|
*/
|
||||||
private[akka] def addRoutees(newRoutees: IndexedSeq[ActorRef]) {
|
private[akka] def addRoutees(newRoutees: IndexedSeq[ActorRef]): Unit = {
|
||||||
_routees = _routees ++ newRoutees
|
_routees = _routees ++ newRoutees
|
||||||
// subscribe to Terminated messages for all route destinations, to be handled by Router actor
|
// subscribe to Terminated messages for all route destinations, to be handled by Router actor
|
||||||
newRoutees foreach underlying.watch
|
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
|
* Not thread safe, but intended to be called from protected points, such as
|
||||||
* `Resizer.resize`
|
* `Resizer.resize`
|
||||||
*/
|
*/
|
||||||
private[akka] def removeRoutees(abandonedRoutees: IndexedSeq[ActorRef]) {
|
private[akka] def removeRoutees(abandonedRoutees: IndexedSeq[ActorRef]): Unit = {
|
||||||
_routees = _routees diff abandonedRoutees
|
_routees = _routees diff abandonedRoutees
|
||||||
abandonedRoutees foreach underlying.unwatch
|
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 = {
|
override def !(message: Any)(implicit sender: ActorRef = null): Unit = {
|
||||||
resize()
|
resize()
|
||||||
|
|
||||||
|
|
@ -95,20 +134,15 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup
|
||||||
}
|
}
|
||||||
|
|
||||||
applyRoute(s, message) match {
|
applyRoute(s, message) match {
|
||||||
case Nil ⇒ super.!(message)(s)
|
case Destination(_, x) :: Nil if x eq this ⇒ super.!(message)(s)
|
||||||
case refs ⇒ refs foreach (p ⇒ p.recipient.!(msg)(p.sender))
|
case refs ⇒ refs foreach (p ⇒ p.recipient.!(msg)(p.sender))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
def resize() {
|
def resize(): Unit = {
|
||||||
for (r ← _props.routerConfig.resizer) {
|
for (r ← routerConfig.resizer) {
|
||||||
if (r.isTimeForResize(resizeCounter.getAndIncrement()) && resizeProgress.compareAndSet(false, true)) {
|
if (r.isTimeForResize(resizeCounter.getAndIncrement()) && resizeInProgress.compareAndSet(false, true))
|
||||||
try {
|
super.!(Router.Resize)
|
||||||
r.resize(routeeProps, routeeProvider)
|
|
||||||
} finally {
|
|
||||||
resizeProgress.set(false)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -139,6 +173,11 @@ trait RouterConfig {
|
||||||
|
|
||||||
def createActor(): Router = new Router {}
|
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).
|
* Overridable merge strategy, by default completely prefers “this” (i.e. no merge).
|
||||||
*/
|
*/
|
||||||
|
|
@ -246,13 +285,20 @@ trait CustomRoute {
|
||||||
*/
|
*/
|
||||||
trait Router extends Actor {
|
trait Router extends Actor {
|
||||||
|
|
||||||
val ref = self match {
|
// make sure that we synchronize properly to get the childrenRefs into our CPU cache
|
||||||
case x: RoutedActorRef ⇒ x
|
val ref = context.synchronized {
|
||||||
case _ ⇒ throw new ActorInitializationException("Router actor can only be used in RoutedActorRef")
|
self match {
|
||||||
|
case x: RoutedActorRef ⇒ x
|
||||||
|
case _ ⇒ throw new ActorInitializationException("Router actor can only be used in RoutedActorRef")
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
final def receive = ({
|
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) ⇒
|
case Terminated(child) ⇒
|
||||||
ref.removeRoutees(IndexedSeq(child))
|
ref.removeRoutees(IndexedSeq(child))
|
||||||
if (ref.routees.isEmpty) context.stop(self)
|
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
|
* Used to broadcast a message to all connections in a router; only the
|
||||||
* contained message will be forwarded, i.e. the `Broadcast(...)`
|
* 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
|
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
|
||||||
case object NoRouter extends RouterConfig {
|
case object NoRouter extends RouterConfig {
|
||||||
def createRoute(props: Props, routeeProvider: RouteeProvider): Route = null
|
def createRoute(props: Props, routeeProvider: RouteeProvider): Route = null
|
||||||
|
def routerDispatcher: String = ""
|
||||||
override def withFallback(other: RouterConfig): RouterConfig = other
|
override def withFallback(other: RouterConfig): RouterConfig = other
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -311,13 +362,20 @@ case object NoRouter extends RouterConfig {
|
||||||
case object FromConfig extends RouterConfig {
|
case object FromConfig extends RouterConfig {
|
||||||
def createRoute(props: Props, routeeProvider: RouteeProvider): Route =
|
def createRoute(props: Props, routeeProvider: RouteeProvider): Route =
|
||||||
throw new ConfigurationException("router " + routeeProvider.context.self + " needs external configuration from file (e.g. application.conf)")
|
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.
|
* 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
|
//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 =
|
def createRoute(props: Props, routeeProvider: RouteeProvider): Route =
|
||||||
throw new ConfigurationException("router " + routeeProvider.context.self + " needs external configuration from file (e.g. application.conf)")
|
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]]
|
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||||
*/
|
*/
|
||||||
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
|
//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 {
|
extends RouterConfig with RoundRobinLike {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -374,6 +433,11 @@ case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] =
|
||||||
* Java API
|
* Java API
|
||||||
*/
|
*/
|
||||||
def this(resizer: Resizer) = this(resizer = Some(resizer))
|
def this(resizer: Resizer) = this(resizer = Some(resizer))
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Java API for setting routerDispatcher
|
||||||
|
*/
|
||||||
|
def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId)
|
||||||
}
|
}
|
||||||
|
|
||||||
trait RoundRobinLike { this: RouterConfig ⇒
|
trait RoundRobinLike { this: RouterConfig ⇒
|
||||||
|
|
@ -428,7 +492,8 @@ object RandomRouter {
|
||||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||||
*/
|
*/
|
||||||
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
|
//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 {
|
extends RouterConfig with RandomLike {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -454,6 +519,11 @@ case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil,
|
||||||
* Java API
|
* Java API
|
||||||
*/
|
*/
|
||||||
def this(resizer: Resizer) = this(resizer = Some(resizer))
|
def this(resizer: Resizer) = this(resizer = Some(resizer))
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Java API for setting routerDispatcher
|
||||||
|
*/
|
||||||
|
def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId)
|
||||||
}
|
}
|
||||||
|
|
||||||
trait RandomLike { this: RouterConfig ⇒
|
trait RandomLike { this: RouterConfig ⇒
|
||||||
|
|
@ -514,7 +584,8 @@ object SmallestMailboxRouter {
|
||||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||||
*/
|
*/
|
||||||
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
|
//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 {
|
extends RouterConfig with SmallestMailboxLike {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -540,6 +611,11 @@ case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[Strin
|
||||||
* Java API
|
* Java API
|
||||||
*/
|
*/
|
||||||
def this(resizer: Resizer) = this(resizer = Some(resizer))
|
def this(resizer: Resizer) = this(resizer = Some(resizer))
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Java API for setting routerDispatcher
|
||||||
|
*/
|
||||||
|
def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId)
|
||||||
}
|
}
|
||||||
|
|
||||||
trait SmallestMailboxLike { this: RouterConfig ⇒
|
trait SmallestMailboxLike { this: RouterConfig ⇒
|
||||||
|
|
@ -659,7 +735,8 @@ object BroadcastRouter {
|
||||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||||
*/
|
*/
|
||||||
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
|
//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 {
|
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))
|
def this(resizer: Resizer) = this(resizer = Some(resizer))
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Java API for setting routerDispatcher
|
||||||
|
*/
|
||||||
|
def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId)
|
||||||
}
|
}
|
||||||
|
|
||||||
trait BroadcastLike { this: RouterConfig ⇒
|
trait BroadcastLike { this: RouterConfig ⇒
|
||||||
|
|
@ -732,7 +813,8 @@ object ScatterGatherFirstCompletedRouter {
|
||||||
*/
|
*/
|
||||||
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
|
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
|
||||||
case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, within: Duration,
|
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 {
|
extends RouterConfig with ScatterGatherFirstCompletedLike {
|
||||||
|
|
||||||
if (within <= Duration.Zero) throw new IllegalArgumentException(
|
if (within <= Duration.Zero) throw new IllegalArgumentException(
|
||||||
|
|
@ -761,6 +843,11 @@ case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: It
|
||||||
* Java API
|
* Java API
|
||||||
*/
|
*/
|
||||||
def this(resizer: Resizer, w: Duration) = this(resizer = Some(resizer), within = w)
|
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 ⇒
|
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
|
* 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
|
* initial resize before first message (messageCounter == 0), because there is no guarantee
|
||||||
* that resize will be done when concurrent messages are in play.
|
* 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
|
def isTimeForResize(messageCounter: Long): Boolean
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Decide if the capacity of the router need to be changed. Will be invoked when `isTimeForResize`
|
* 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.
|
* returns true and no other resize is in progress.
|
||||||
* Create and register more routees with `routeeProvider.registerRoutees(newRoutees)
|
* Create and register more routees with `routeeProvider.registerRoutees(newRoutees)
|
||||||
* or remove routees with `routeeProvider.unregisterRoutees(abandonedRoutees)` and
|
* or remove routees with `routeeProvider.unregisterRoutees(abandonedRoutees)` and
|
||||||
* sending [[akka.actor.PoisonPill]] to them.
|
* 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)
|
def resize(props: Props, routeeProvider: RouteeProvider)
|
||||||
}
|
}
|
||||||
|
|
@ -933,7 +1028,8 @@ case class DefaultResizer(
|
||||||
*/
|
*/
|
||||||
def capacity(routees: IndexedSeq[ActorRef]): Int = {
|
def capacity(routees: IndexedSeq[ActorRef]): Int = {
|
||||||
val currentSize = routees.size
|
val currentSize = routees.size
|
||||||
val delta = filter(pressure(routees), currentSize)
|
val press = pressure(routees)
|
||||||
|
val delta = filter(press, currentSize)
|
||||||
val proposed = currentSize + delta
|
val proposed = currentSize + delta
|
||||||
|
|
||||||
if (proposed < lowerBound) delta + (lowerBound - proposed)
|
if (proposed < lowerBound) delta + (lowerBound - proposed)
|
||||||
|
|
@ -963,7 +1059,7 @@ case class DefaultResizer(
|
||||||
case a: LocalActorRef ⇒
|
case a: LocalActorRef ⇒
|
||||||
val cell = a.underlying
|
val cell = a.underlying
|
||||||
pressureThreshold match {
|
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 i if i < 1 ⇒ cell.mailbox.isScheduled && cell.currentMessage != null
|
||||||
case threshold ⇒ cell.mailbox.numberOfMessages >= threshold
|
case threshold ⇒ cell.mailbox.numberOfMessages >= threshold
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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)
|
|
||||||
}
|
|
||||||
|
|
@ -5,12 +5,12 @@
|
||||||
package akka.serialization
|
package akka.serialization
|
||||||
|
|
||||||
import akka.AkkaException
|
import akka.AkkaException
|
||||||
import akka.util.ReflectiveAccess
|
|
||||||
import scala.util.DynamicVariable
|
import scala.util.DynamicVariable
|
||||||
import com.typesafe.config.Config
|
import com.typesafe.config.Config
|
||||||
import akka.actor.{ Extension, ActorSystem, ExtendedActorSystem, Address }
|
import akka.actor.{ Extension, ExtendedActorSystem, Address, DynamicAccess }
|
||||||
import java.util.concurrent.ConcurrentHashMap
|
|
||||||
import akka.event.Logging
|
import akka.event.Logging
|
||||||
|
import java.util.concurrent.ConcurrentHashMap
|
||||||
|
import akka.util.NonFatal
|
||||||
import scala.collection.mutable.ArrayBuffer
|
import scala.collection.mutable.ArrayBuffer
|
||||||
import java.io.NotSerializableException
|
import java.io.NotSerializableException
|
||||||
|
|
||||||
|
|
@ -23,19 +23,6 @@ object Serialization {
|
||||||
*/
|
*/
|
||||||
type ClassSerializer = (Class[_], Serializer)
|
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
|
* This holds a reference to the current transport address to be inserted
|
||||||
* into local actor refs during serialization.
|
* 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
|
* 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.
|
* to either an Array of Bytes or an Exception if one was thrown.
|
||||||
*/
|
*/
|
||||||
def serialize(o: AnyRef): Either[Exception, Array[Byte]] =
|
def serialize(o: AnyRef): Either[Throwable, Array[Byte]] =
|
||||||
try { Right(findSerializerFor(o).toBinary(o)) } catch { case e: Exception ⇒ Left(e) }
|
try Right(findSerializerFor(o).toBinary(o))
|
||||||
|
catch { case NonFatal(e) ⇒ Left(e) }
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Deserializes the given array of bytes using the specified serializer id,
|
* 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],
|
def deserialize(bytes: Array[Byte],
|
||||||
serializerId: Int,
|
serializerId: Int,
|
||||||
clazz: Option[Class[_]],
|
clazz: Option[Class[_]]): Either[Throwable, AnyRef] =
|
||||||
classLoader: ClassLoader): Either[Exception, AnyRef] =
|
try Right(serializerByIdentity(serializerId).fromBinary(bytes, clazz))
|
||||||
try {
|
catch { case NonFatal(e) ⇒ Left(e) }
|
||||||
currentSystem.withValue(system) {
|
|
||||||
Right(serializerByIdentity(serializerId).fromBinary(bytes, clazz, Some(classLoader)))
|
|
||||||
}
|
|
||||||
} catch { case e: Exception ⇒ Left(e) }
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Deserializes the given array of bytes using the specified type to look up what Serializer should be used.
|
* 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.
|
* You can specify an optional ClassLoader to load the object into.
|
||||||
* Returns either the resulting object or an Exception if one was thrown.
|
* Returns either the resulting object or an Exception if one was thrown.
|
||||||
*/
|
*/
|
||||||
def deserialize(
|
def deserialize(bytes: Array[Byte], clazz: Class[_]): Either[Throwable, AnyRef] =
|
||||||
bytes: Array[Byte],
|
try Right(serializerFor(clazz).fromBinary(bytes, Some(clazz)))
|
||||||
clazz: Class[_],
|
catch { case NonFatal(e) ⇒ Left(e) }
|
||||||
classLoader: Option[ClassLoader]): Either[Exception, AnyRef] =
|
|
||||||
try {
|
|
||||||
currentSystem.withValue(system) { Right(serializerFor(clazz).fromBinary(bytes, Some(clazz), classLoader)) }
|
|
||||||
} catch { case e: Exception ⇒ Left(e) }
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the Serializer configured for the given object, returns the NullSerializer if it's null.
|
* 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 system’s [[akka.actor.DynamicAccess]].
|
||||||
*/
|
*/
|
||||||
def serializerOf(serializerFQN: String): Either[Exception, Serializer] =
|
def serializerOf(serializerFQN: String): Either[Throwable, Serializer] =
|
||||||
ReflectiveAccess.createInstance(serializerFQN, ReflectiveAccess.noParams, ReflectiveAccess.noArgs, system.internalClassLoader)
|
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)
|
* 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] = {
|
private[akka] val bindings: Seq[ClassSerializer] = {
|
||||||
val configuredBindings = for ((k: String, v: String) ← settings.SerializationBindings if v != "none") yield {
|
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))
|
(c, serializers(v))
|
||||||
}
|
}
|
||||||
sort(configuredBindings)
|
sort(configuredBindings)
|
||||||
|
|
|
||||||
|
|
@ -6,11 +6,31 @@ package akka.serialization
|
||||||
|
|
||||||
import java.io.{ ObjectOutputStream, ByteArrayOutputStream, ObjectInputStream, ByteArrayInputStream }
|
import java.io.{ ObjectOutputStream, ByteArrayOutputStream, ObjectInputStream, ByteArrayInputStream }
|
||||||
import akka.util.ClassLoaderObjectInputStream
|
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 {
|
trait Serializer {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Completely unique value to identify this implementation of Serializer, used to optimize network traffic
|
* 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
|
* Values from 0 to 16 is reserved for Akka internal usage
|
||||||
|
|
@ -28,42 +48,61 @@ trait Serializer {
|
||||||
def includeManifest: Boolean
|
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 {
|
abstract class JSerializer extends Serializer {
|
||||||
def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]] = None, classLoader: Option[ClassLoader] = None): AnyRef =
|
final def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]]): AnyRef =
|
||||||
fromBinary(bytes, manifest.orNull, classLoader.orNull)
|
fromBinaryJava(bytes, manifest.orNull)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This method should be overridden,
|
* This method must be implemented, manifest may be null.
|
||||||
* manifest and classLoader 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 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
|
* This Serializer uses standard Java Serialization
|
||||||
*/
|
*/
|
||||||
class JavaSerializer extends Serializer {
|
class JavaSerializer(val system: ExtendedActorSystem) extends Serializer {
|
||||||
|
|
||||||
def includeManifest: Boolean = false
|
def includeManifest: Boolean = false
|
||||||
|
|
||||||
|
|
@ -77,12 +116,11 @@ class JavaSerializer extends Serializer {
|
||||||
bos.toByteArray
|
bos.toByteArray
|
||||||
}
|
}
|
||||||
|
|
||||||
def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]] = None,
|
def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = {
|
||||||
classLoader: Option[ClassLoader] = None): AnyRef = {
|
val in = new ClassLoaderObjectInputStream(system.dynamicAccess.classLoader, new ByteArrayInputStream(bytes))
|
||||||
val in =
|
val obj = JavaSerializer.currentSystem.withValue(system) {
|
||||||
if (classLoader.isDefined) new ClassLoaderObjectInputStream(classLoader.get, new ByteArrayInputStream(bytes)) else
|
in.readObject
|
||||||
new ObjectInputStream(new ByteArrayInputStream(bytes))
|
}
|
||||||
val obj = in.readObject
|
|
||||||
in.close()
|
in.close()
|
||||||
obj
|
obj
|
||||||
}
|
}
|
||||||
|
|
@ -96,5 +134,5 @@ class NullSerializer extends Serializer {
|
||||||
def includeManifest: Boolean = false
|
def includeManifest: Boolean = false
|
||||||
def identifier = 0
|
def identifier = 0
|
||||||
def toBinary(o: AnyRef) = nullAsBytes
|
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
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -21,8 +21,18 @@ object Helpers {
|
||||||
if (diff > 0) 1 else if (diff < 0) -1 else 0
|
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+~"
|
final val base64chars = "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789+~"
|
||||||
|
|
|
||||||
30
akka-actor/src/main/scala/akka/util/Reflect.scala
Normal file
30
akka-actor/src/main/scala/akka/util/Reflect.scala
Normal 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
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -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)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
@ -9,7 +9,7 @@ class AccrualFailureDetectorSpec extends AkkaSpec("""
|
||||||
""") {
|
""") {
|
||||||
|
|
||||||
"An AccrualFailureDetector" must {
|
"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 {
|
"mark node as available after a series of successful heartbeats" in {
|
||||||
val fd = new AccrualFailureDetector(system)
|
val fd = new AccrualFailureDetector(system)
|
||||||
|
|
|
||||||
|
|
@ -18,28 +18,46 @@ The multi-JVM testing is an sbt plugin that you can find here:
|
||||||
|
|
||||||
http://github.com/typesafehub/sbt-multi-jvm
|
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
|
resolvers += Classpaths.typesafeResolver
|
||||||
|
|
||||||
addSbtPlugin("com.typesafe.sbtmultijvm" % "sbt-multi-jvm" % "0.1.9")
|
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
|
settings and config. For example, here is how the akka-remote project adds
|
||||||
multi-JVM testing::
|
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(
|
object AkkaBuild extends Build {
|
||||||
id = "akka-remote",
|
|
||||||
base = file("akka-remote"),
|
lazy val remote = Project(
|
||||||
settings = defaultSettings ++ MultiJvmPlugin.settings ++ Seq(
|
id = "akka-remote",
|
||||||
extraOptions in MultiJvm <<= (sourceDirectory in MultiJvm) { src =>
|
base = file("akka-remote"),
|
||||||
(name: String) => (src ** (name + ".conf")).get.headOption.map("-Dconfig.file=" + _.absolutePath).toSeq
|
settings = defaultSettings ++ MultiJvmPlugin.settings ++ Seq(
|
||||||
},
|
extraOptions in MultiJvm <<= (sourceDirectory in MultiJvm) { src =>
|
||||||
test in Test <<= (test in Test) dependsOn (test in MultiJvm)
|
(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::
|
You can specify JVM options for the forked JVMs::
|
||||||
|
|
||||||
|
|
@ -87,8 +105,8 @@ options after the test names and ``--``. For example:
|
||||||
Creating application tests
|
Creating application tests
|
||||||
==========================
|
==========================
|
||||||
|
|
||||||
The tests are discovered, and combined, through a naming convention. A test is
|
The tests are discovered, and combined, through a naming convention. MultiJvm tests are
|
||||||
named with the following pattern:
|
located in ``src/multi-jvm/scala`` directory. A test is named with the following pattern:
|
||||||
|
|
||||||
.. code-block:: none
|
.. code-block:: none
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -75,6 +75,24 @@ akka-testkit
|
||||||
.. literalinclude:: ../../akka-testkit/src/main/resources/reference.conf
|
.. literalinclude:: ../../akka-testkit/src/main/resources/reference.conf
|
||||||
:language: none
|
: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
|
akka-beanstalk-mailbox
|
||||||
~~~~~~~~~~~~~~~~~~~~~~
|
~~~~~~~~~~~~~~~~~~~~~~
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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.
|
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
|
STM and the Java Memory Model
|
||||||
-----------------------------
|
-----------------------------
|
||||||
Akka's Software Transactional Memory (STM) also provides a "happens before" rule:
|
Akka's Software Transactional Memory (STM) also provides a "happens before" rule:
|
||||||
|
|
|
||||||
|
|
@ -78,7 +78,7 @@ public class FutureDocTestBase {
|
||||||
ActorRef actor = system.actorOf(new Props(MyActor.class));
|
ActorRef actor = system.actorOf(new Props(MyActor.class));
|
||||||
String msg = "hello";
|
String msg = "hello";
|
||||||
//#ask-blocking
|
//#ask-blocking
|
||||||
Timeout timeout = system.settings().ActorTimeout();
|
Timeout timeout = new Timeout(Duration.parse("5 seconds"));
|
||||||
Future<Object> future = Patterns.ask(actor, msg, timeout);
|
Future<Object> future = Patterns.ask(actor, msg, timeout);
|
||||||
String result = (String) Await.result(future, timeout.duration());
|
String result = (String) Await.result(future, timeout.duration());
|
||||||
//#ask-blocking
|
//#ask-blocking
|
||||||
|
|
@ -196,19 +196,17 @@ public class FutureDocTestBase {
|
||||||
Iterable<Future<Integer>> listOfFutureInts = source;
|
Iterable<Future<Integer>> listOfFutureInts = source;
|
||||||
|
|
||||||
// now we have a Future[Iterable[Integer]]
|
// now we have a Future[Iterable[Integer]]
|
||||||
Future<Iterable<Integer>> futureListOfInts =
|
Future<Iterable<Integer>> futureListOfInts = sequence(listOfFutureInts, system.dispatcher());
|
||||||
sequence(listOfFutureInts, system.dispatcher());
|
|
||||||
|
|
||||||
// Find the sum of the odd numbers
|
// Find the sum of the odd numbers
|
||||||
Future<Long> futureSum = futureListOfInts.map(
|
Future<Long> futureSum = futureListOfInts.map(new Mapper<Iterable<Integer>, Long>() {
|
||||||
new Mapper<Iterable<Integer>, Long>() {
|
public Long apply(Iterable<Integer> ints) {
|
||||||
public Long apply(Iterable<Integer> ints) {
|
long sum = 0;
|
||||||
long sum = 0;
|
for (Integer i : ints)
|
||||||
for (Integer i : ints)
|
sum += i;
|
||||||
sum += i;
|
return sum;
|
||||||
return sum;
|
}
|
||||||
}
|
});
|
||||||
});
|
|
||||||
|
|
||||||
long result = Await.result(futureSum, Duration.create(1, SECONDS));
|
long result = Await.result(futureSum, Duration.create(1, SECONDS));
|
||||||
//#sequence
|
//#sequence
|
||||||
|
|
@ -221,20 +219,18 @@ public class FutureDocTestBase {
|
||||||
//Just a sequence of Strings
|
//Just a sequence of Strings
|
||||||
Iterable<String> listStrings = Arrays.asList("a", "b", "c");
|
Iterable<String> listStrings = Arrays.asList("a", "b", "c");
|
||||||
|
|
||||||
Future<Iterable<String>> futureResult = traverse(listStrings,
|
Future<Iterable<String>> futureResult = traverse(listStrings, new Function<String, Future<String>>() {
|
||||||
new Function<String, Future<String>>() {
|
public Future<String> apply(final String r) {
|
||||||
public Future<String> apply(final String r) {
|
return future(new Callable<String>() {
|
||||||
return future(new Callable<String>() {
|
public String call() {
|
||||||
public String call() {
|
return r.toUpperCase();
|
||||||
return r.toUpperCase();
|
}
|
||||||
}
|
}, system.dispatcher());
|
||||||
}, system.dispatcher());
|
}
|
||||||
}
|
}, system.dispatcher());
|
||||||
}, system.dispatcher());
|
|
||||||
|
|
||||||
//Returns the sequence of strings as upper case
|
//Returns the sequence of strings as upper case
|
||||||
Iterable<String> result =
|
Iterable<String> result = Await.result(futureResult, Duration.create(1, SECONDS));
|
||||||
Await.result(futureResult, Duration.create(1, SECONDS));
|
|
||||||
assertEquals(Arrays.asList("A", "B", "C"), result);
|
assertEquals(Arrays.asList("A", "B", "C"), result);
|
||||||
//#traverse
|
//#traverse
|
||||||
}
|
}
|
||||||
|
|
@ -250,12 +246,11 @@ public class FutureDocTestBase {
|
||||||
Iterable<Future<String>> futures = source;
|
Iterable<Future<String>> futures = source;
|
||||||
|
|
||||||
//Start value is the empty string
|
//Start value is the empty string
|
||||||
Future<String> resultFuture = fold("", futures,
|
Future<String> resultFuture = fold("", futures, new Function2<String, String, String>() {
|
||||||
new Function2<String, String, String>() {
|
public String apply(String r, String t) {
|
||||||
public String apply(String r, String t) {
|
return r + t; //Just concatenate
|
||||||
return r + t; //Just concatenate
|
}
|
||||||
}
|
}, system.dispatcher());
|
||||||
}, system.dispatcher());
|
|
||||||
String result = Await.result(resultFuture, Duration.create(1, SECONDS));
|
String result = Await.result(resultFuture, Duration.create(1, SECONDS));
|
||||||
//#fold
|
//#fold
|
||||||
|
|
||||||
|
|
@ -272,12 +267,11 @@ public class FutureDocTestBase {
|
||||||
//A sequence of Futures, in this case Strings
|
//A sequence of Futures, in this case Strings
|
||||||
Iterable<Future<String>> futures = source;
|
Iterable<Future<String>> futures = source;
|
||||||
|
|
||||||
Future<Object> resultFuture = reduce(futures,
|
Future<Object> resultFuture = reduce(futures, new Function2<Object, String, Object>() {
|
||||||
new Function2<Object, String, Object>() {
|
public Object apply(Object r, String t) {
|
||||||
public Object apply(Object r, String t) {
|
return r + t; //Just concatenate
|
||||||
return r + t; //Just concatenate
|
}
|
||||||
}
|
}, system.dispatcher());
|
||||||
}, system.dispatcher());
|
|
||||||
|
|
||||||
Object result = Await.result(resultFuture, Duration.create(1, SECONDS));
|
Object result = Await.result(resultFuture, Duration.create(1, SECONDS));
|
||||||
//#reduce
|
//#reduce
|
||||||
|
|
@ -285,32 +279,35 @@ public class FutureDocTestBase {
|
||||||
assertEquals("ab", result);
|
assertEquals("ab", result);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test public void useSuccessfulAndFailed() {
|
@Test
|
||||||
|
public void useSuccessfulAndFailed() {
|
||||||
//#successful
|
//#successful
|
||||||
Future<String> future = Futures.successful("Yay!", system.dispatcher());
|
Future<String> future = Futures.successful("Yay!", system.dispatcher());
|
||||||
//#successful
|
//#successful
|
||||||
//#failed
|
//#failed
|
||||||
Future<String> otherFuture =
|
Future<String> otherFuture = Futures.failed(new IllegalArgumentException("Bang!"), system.dispatcher());
|
||||||
Futures.failed(new IllegalArgumentException("Bang!"), system.dispatcher());
|
|
||||||
//#failed
|
//#failed
|
||||||
Object result = Await.result(future, Duration.create(1, SECONDS));
|
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));
|
Throwable result2 = Await.result(otherFuture.failed(), Duration.create(1, SECONDS));
|
||||||
assertEquals("Bang!",result2.getMessage());
|
assertEquals("Bang!", result2.getMessage());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test public void useFilter() {
|
@Test
|
||||||
//#filter
|
public void useFilter() {
|
||||||
|
//#filter
|
||||||
Future<Integer> future1 = Futures.successful(4, system.dispatcher());
|
Future<Integer> future1 = Futures.successful(4, system.dispatcher());
|
||||||
Future<Integer> successfulFilter =
|
Future<Integer> successfulFilter = future1.filter(new Filter<Integer>() {
|
||||||
future1.filter(new Filter<Integer>() {
|
public boolean filter(Integer i) {
|
||||||
public boolean filter(Integer i) { return i % 2 == 0; }
|
return i % 2 == 0;
|
||||||
});
|
}
|
||||||
|
});
|
||||||
|
|
||||||
Future<Integer> failedFilter =
|
Future<Integer> failedFilter = future1.filter(new Filter<Integer>() {
|
||||||
future1.filter(new Filter<Integer>() {
|
public boolean filter(Integer i) {
|
||||||
public boolean filter(Integer i) { return i % 2 != 0; }
|
return i % 2 != 0;
|
||||||
});
|
}
|
||||||
|
});
|
||||||
//When filter fails, the returned Future will be failed with a scala.MatchError
|
//When filter fails, the returned Future will be failed with a scala.MatchError
|
||||||
//#filter
|
//#filter
|
||||||
}
|
}
|
||||||
|
|
@ -323,137 +320,140 @@ public class FutureDocTestBase {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test public void useAndThen() {
|
@Test
|
||||||
|
public void useAndThen() {
|
||||||
//#and-then
|
//#and-then
|
||||||
Future<String> future1 = Futures.successful("value", system.dispatcher()).
|
Future<String> future1 = Futures.successful("value", system.dispatcher()).andThen(new OnComplete<String>() {
|
||||||
andThen(new OnComplete<String>() {
|
public void onComplete(Throwable failure, String result) {
|
||||||
public void onComplete(Throwable failure, String result) {
|
if (failure != null)
|
||||||
if (failure != null) sendToIssueTracker(failure);
|
sendToIssueTracker(failure);
|
||||||
}
|
}
|
||||||
}).andThen(new OnComplete<String>() {
|
}).andThen(new OnComplete<String>() {
|
||||||
public void onComplete(Throwable failure, String result) {
|
public void onComplete(Throwable failure, String result) {
|
||||||
if (result != null) sendToTheInternetz(result);
|
if (result != null)
|
||||||
}
|
sendToTheInternetz(result);
|
||||||
|
}
|
||||||
});
|
});
|
||||||
//#and-then
|
//#and-then
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test public void useRecover() {
|
@Test
|
||||||
|
public void useRecover() {
|
||||||
//#recover
|
//#recover
|
||||||
Future<Integer> future = future(new Callable<Integer>() {
|
Future<Integer> future = future(new Callable<Integer>() {
|
||||||
public Integer call() {
|
public Integer call() {
|
||||||
return 1 / 0;
|
return 1 / 0;
|
||||||
}
|
}
|
||||||
}, system.dispatcher()).recover(new Recover<Integer>() {
|
}, system.dispatcher()).recover(new Recover<Integer>() {
|
||||||
public Integer recover(Throwable problem) throws Throwable {
|
public Integer recover(Throwable problem) throws Throwable {
|
||||||
if (problem instanceof ArithmeticException) return 0;
|
if (problem instanceof ArithmeticException)
|
||||||
else throw problem;
|
return 0;
|
||||||
}
|
else
|
||||||
|
throw problem;
|
||||||
|
}
|
||||||
});
|
});
|
||||||
int result = Await.result(future, Duration.create(1, SECONDS));
|
int result = Await.result(future, Duration.create(1, SECONDS));
|
||||||
assertEquals(result, 0);
|
assertEquals(result, 0);
|
||||||
//#recover
|
//#recover
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test public void useTryRecover() {
|
@Test
|
||||||
|
public void useTryRecover() {
|
||||||
//#try-recover
|
//#try-recover
|
||||||
Future<Integer> future = future(new Callable<Integer>() {
|
Future<Integer> future = future(new Callable<Integer>() {
|
||||||
public Integer call() {
|
public Integer call() {
|
||||||
return 1 / 0;
|
return 1 / 0;
|
||||||
}
|
}
|
||||||
}, system.dispatcher()).recoverWith(new Recover<Future<Integer>>() {
|
}, system.dispatcher()).recoverWith(new Recover<Future<Integer>>() {
|
||||||
public Future<Integer> recover(Throwable problem) throws Throwable {
|
public Future<Integer> recover(Throwable problem) throws Throwable {
|
||||||
if (problem instanceof ArithmeticException) {
|
if (problem instanceof ArithmeticException) {
|
||||||
return future(new Callable<Integer>() {
|
return future(new Callable<Integer>() {
|
||||||
public Integer call() {
|
public Integer call() {
|
||||||
return 0;
|
return 0;
|
||||||
}
|
|
||||||
}, system.dispatcher());
|
|
||||||
}
|
}
|
||||||
else throw problem;
|
}, system.dispatcher());
|
||||||
}
|
} else
|
||||||
|
throw problem;
|
||||||
|
}
|
||||||
});
|
});
|
||||||
int result = Await.result(future, Duration.create(1, SECONDS));
|
int result = Await.result(future, Duration.create(1, SECONDS));
|
||||||
assertEquals(result, 0);
|
assertEquals(result, 0);
|
||||||
//#try-recover
|
//#try-recover
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test public void useOnSuccessOnFailureAndOnComplete() {
|
@Test
|
||||||
{
|
public void useOnSuccessOnFailureAndOnComplete() {
|
||||||
|
{
|
||||||
Future<String> future = Futures.successful("foo", system.dispatcher());
|
Future<String> future = Futures.successful("foo", system.dispatcher());
|
||||||
|
|
||||||
//#onSuccess
|
//#onSuccess
|
||||||
future.onSuccess(new OnSuccess<String>() {
|
future.onSuccess(new OnSuccess<String>() {
|
||||||
public void onSuccess(String result) {
|
public void onSuccess(String result) {
|
||||||
if ("bar" == result) {
|
if ("bar" == result) {
|
||||||
//Do something if it resulted in "bar"
|
//Do something if it resulted in "bar"
|
||||||
} else {
|
} else {
|
||||||
//Do something if it was some other String
|
//Do something if it was some other String
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
}
|
||||||
});
|
});
|
||||||
//#onSuccess
|
//#onSuccess
|
||||||
}
|
}
|
||||||
{
|
{
|
||||||
Future<String> future =
|
Future<String> future = Futures.failed(new IllegalStateException("OHNOES"), system.dispatcher());
|
||||||
Futures.failed(new IllegalStateException("OHNOES"), system.dispatcher());
|
//#onFailure
|
||||||
//#onFailure
|
future.onFailure(new OnFailure() {
|
||||||
future.onFailure( new OnFailure() {
|
|
||||||
public void onFailure(Throwable failure) {
|
public void onFailure(Throwable failure) {
|
||||||
if (failure instanceof IllegalStateException) {
|
if (failure instanceof IllegalStateException) {
|
||||||
//Do something if it was this particular failure
|
//Do something if it was this particular failure
|
||||||
} else {
|
} else {
|
||||||
//Do something if it was some other failure
|
//Do something if it was some other failure
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
//#onFailure
|
//#onFailure
|
||||||
}
|
}
|
||||||
{
|
{
|
||||||
Future<String> future = Futures.successful("foo", system.dispatcher());
|
Future<String> future = Futures.successful("foo", system.dispatcher());
|
||||||
//#onComplete
|
//#onComplete
|
||||||
future.onComplete(new OnComplete<String>() {
|
future.onComplete(new OnComplete<String>() {
|
||||||
public void onComplete(Throwable failure, String result) {
|
public void onComplete(Throwable failure, String result) {
|
||||||
if (failure != null) {
|
if (failure != null) {
|
||||||
//We got a failure, handle it here
|
//We got a failure, handle it here
|
||||||
} else {
|
} else {
|
||||||
// We got a result, do something with it
|
// We got a result, do something with it
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
//#onComplete
|
//#onComplete
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test public void useOrAndZip(){
|
@Test
|
||||||
|
public void useOrAndZip() {
|
||||||
{
|
{
|
||||||
//#zip
|
//#zip
|
||||||
Future<String> future1 = Futures.successful("foo", system.dispatcher());
|
Future<String> future1 = Futures.successful("foo", system.dispatcher());
|
||||||
Future<String> future2 = Futures.successful("bar", system.dispatcher());
|
Future<String> future2 = Futures.successful("bar", system.dispatcher());
|
||||||
Future<String> future3 =
|
Future<String> future3 = future1.zip(future2).map(new Mapper<scala.Tuple2<String, String>, String>() {
|
||||||
future1.zip(future2).map(new Mapper<scala.Tuple2<String,String>, String>() {
|
public String apply(scala.Tuple2<String, String> zipped) {
|
||||||
public String apply(scala.Tuple2<String,String> zipped) {
|
return zipped._1() + " " + zipped._2();
|
||||||
return zipped._1() + " " + zipped._2();
|
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
|
||||||
String result = Await.result(future3, Duration.create(1, SECONDS));
|
String result = Await.result(future3, Duration.create(1, SECONDS));
|
||||||
assertEquals("foo bar", result);
|
assertEquals("foo bar", result);
|
||||||
//#zip
|
//#zip
|
||||||
}
|
}
|
||||||
|
|
||||||
{
|
{
|
||||||
//#fallback-to
|
//#fallback-to
|
||||||
Future<String> future1 =
|
Future<String> future1 = Futures.failed(new IllegalStateException("OHNOES1"), system.dispatcher());
|
||||||
Futures.failed(new IllegalStateException("OHNOES1"), system.dispatcher());
|
Future<String> future2 = Futures.failed(new IllegalStateException("OHNOES2"), system.dispatcher());
|
||||||
Future<String> future2 =
|
Future<String> future3 = Futures.successful("bar", system.dispatcher());
|
||||||
Futures.failed(new IllegalStateException("OHNOES2"), system.dispatcher());
|
Future<String> future4 = future1.fallbackTo(future2).fallbackTo(future3); // Will have "bar" in this case
|
||||||
Future<String> future3 =
|
String result = Await.result(future4, Duration.create(1, SECONDS));
|
||||||
Futures.successful("bar", system.dispatcher());
|
assertEquals("bar", result);
|
||||||
Future<String> future4 =
|
//#fallback-to
|
||||||
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
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -17,6 +17,7 @@ import akka.util.Duration;
|
||||||
import akka.util.Timeout;
|
import akka.util.Timeout;
|
||||||
import akka.dispatch.Await;
|
import akka.dispatch.Await;
|
||||||
import akka.dispatch.Future;
|
import akka.dispatch.Future;
|
||||||
|
import akka.dispatch.Dispatchers;
|
||||||
import akka.testkit.AkkaSpec;
|
import akka.testkit.AkkaSpec;
|
||||||
import com.typesafe.config.ConfigFactory;
|
import com.typesafe.config.ConfigFactory;
|
||||||
import static akka.pattern.Patterns.ask;
|
import static akka.pattern.Patterns.ask;
|
||||||
|
|
@ -38,6 +39,19 @@ public class CustomRouterDocTestBase {
|
||||||
public void tearDown() {
|
public void tearDown() {
|
||||||
system.shutdown();
|
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
|
//#crTest
|
||||||
@Test
|
@Test
|
||||||
|
|
@ -105,6 +119,10 @@ public class CustomRouterDocTestBase {
|
||||||
|
|
||||||
//#crRouter
|
//#crRouter
|
||||||
public static class VoteCountRouter extends CustomRouterConfig {
|
public static class VoteCountRouter extends CustomRouterConfig {
|
||||||
|
|
||||||
|
@Override public String routerDispatcher() {
|
||||||
|
return Dispatchers.DefaultDispatcherId();
|
||||||
|
}
|
||||||
|
|
||||||
//#crRoute
|
//#crRoute
|
||||||
@Override
|
@Override
|
||||||
|
|
|
||||||
|
|
@ -54,9 +54,9 @@ public class ParentActor extends UntypedActor {
|
||||||
ActorRef scatterGatherFirstCompletedRouter = getContext().actorOf(
|
ActorRef scatterGatherFirstCompletedRouter = getContext().actorOf(
|
||||||
new Props(FibonacciActor.class).withRouter(new ScatterGatherFirstCompletedRouter(5, Duration
|
new Props(FibonacciActor.class).withRouter(new ScatterGatherFirstCompletedRouter(5, Duration
|
||||||
.parse("2 seconds"))), "router");
|
.parse("2 seconds"))), "router");
|
||||||
Timeout timeout = getContext().system().settings().ActorTimeout();
|
Timeout timeout = new Timeout(Duration.parse("5 seconds"));
|
||||||
Future<Object> futureResult = akka.pattern.Patterns.ask(
|
Future<Object> futureResult = akka.pattern.Patterns.ask(scatterGatherFirstCompletedRouter,
|
||||||
scatterGatherFirstCompletedRouter, new FibonacciActor.FibonacciNumber(10), timeout);
|
new FibonacciActor.FibonacciNumber(10), timeout);
|
||||||
int result = (Integer) Await.result(futureResult, timeout.duration());
|
int result = (Integer) Await.result(futureResult, timeout.duration());
|
||||||
//#scatterGatherFirstCompletedRouter
|
//#scatterGatherFirstCompletedRouter
|
||||||
System.out.println(String.format("The result of calculating Fibonacci for 10 is %d", result));
|
System.out.println(String.format("The result of calculating Fibonacci for 10 is %d", result));
|
||||||
|
|
|
||||||
|
|
@ -3,6 +3,7 @@
|
||||||
*/
|
*/
|
||||||
package akka.docs.serialization;
|
package akka.docs.serialization;
|
||||||
|
|
||||||
|
import akka.japi.Option;
|
||||||
import akka.serialization.JSerializer;
|
import akka.serialization.JSerializer;
|
||||||
import akka.serialization.Serialization;
|
import akka.serialization.Serialization;
|
||||||
import akka.serialization.SerializationExtension;
|
import akka.serialization.SerializationExtension;
|
||||||
|
|
@ -43,10 +44,8 @@ public class SerializationDocTestBase {
|
||||||
|
|
||||||
// "fromBinary" deserializes the given array,
|
// "fromBinary" deserializes the given array,
|
||||||
// using the type hint (if any, see "includeManifest" above)
|
// using the type hint (if any, see "includeManifest" above)
|
||||||
// into the optionally provided classLoader.
|
@Override public Object fromBinaryJava(byte[] bytes,
|
||||||
@Override public Object fromBinary(byte[] bytes,
|
Class<?> clazz) {
|
||||||
Class clazz,
|
|
||||||
ClassLoader classLoader) {
|
|
||||||
// Put your code that deserializes here
|
// Put your code that deserializes here
|
||||||
//#...
|
//#...
|
||||||
return null;
|
return null;
|
||||||
|
|
|
||||||
8
akka-docs/java/code/akka/docs/zeromq/ZeromqDocTest.scala
Normal file
8
akka-docs/java/code/akka/docs/zeromq/ZeromqDocTest.scala
Normal 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
|
||||||
286
akka-docs/java/code/akka/docs/zeromq/ZeromqDocTestBase.java
Normal file
286
akka-docs/java/code/akka/docs/zeromq/ZeromqDocTestBase.java
Normal 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
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -23,3 +23,4 @@ Java API
|
||||||
transactors
|
transactors
|
||||||
fsm
|
fsm
|
||||||
extending-akka
|
extending-akka
|
||||||
|
zeromq
|
||||||
|
|
|
||||||
|
|
@ -8,11 +8,6 @@ Routing (Java)
|
||||||
|
|
||||||
.. contents:: :local:
|
.. 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.
|
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'.
|
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
|
*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.*
|
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
|
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
|
in ``resizer`` method of the ``RouterConfig``. See ``akka.routing.DefaultResizer`` for inspiration
|
||||||
of how to write your own resize strategy.
|
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 system’s 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
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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
|
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.
|
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
98
akka-docs/java/zeromq.rst
Normal 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.
|
||||||
6
akka-docs/modules/code/Global.scala
Normal file
6
akka-docs/modules/code/Global.scala
Normal 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
|
||||||
128
akka-docs/modules/code/akka/docs/http/PlayMiniApplication.scala
Normal file
128
akka-docs/modules/code/akka/docs/http/PlayMiniApplication.scala
Normal 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
|
||||||
|
}
|
||||||
|
|
@ -7,8 +7,194 @@ HTTP
|
||||||
|
|
||||||
.. contents:: :local:
|
.. 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.
|
||||||
|
|
@ -11,6 +11,7 @@ import akka.actor.Props
|
||||||
import akka.actor.Status.Failure
|
import akka.actor.Status.Failure
|
||||||
import akka.dispatch.Future
|
import akka.dispatch.Future
|
||||||
import akka.dispatch.Await
|
import akka.dispatch.Await
|
||||||
|
import akka.util.Timeout
|
||||||
import akka.util.duration._
|
import akka.util.duration._
|
||||||
import akka.dispatch.Promise
|
import akka.dispatch.Promise
|
||||||
import java.lang.IllegalStateException
|
import java.lang.IllegalStateException
|
||||||
|
|
@ -46,8 +47,10 @@ class FutureDocSpec extends AkkaSpec {
|
||||||
//#ask-blocking
|
//#ask-blocking
|
||||||
import akka.dispatch.Await
|
import akka.dispatch.Await
|
||||||
import akka.pattern.ask
|
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 future = actor ? msg // enabled by the “ask” import
|
||||||
val result = Await.result(future, timeout.duration).asInstanceOf[String]
|
val result = Await.result(future, timeout.duration).asInstanceOf[String]
|
||||||
//#ask-blocking
|
//#ask-blocking
|
||||||
|
|
@ -57,7 +60,7 @@ class FutureDocSpec extends AkkaSpec {
|
||||||
"demonstrate usage of mapTo" in {
|
"demonstrate usage of mapTo" in {
|
||||||
val actor = system.actorOf(Props[MyActor])
|
val actor = system.actorOf(Props[MyActor])
|
||||||
val msg = "hello"
|
val msg = "hello"
|
||||||
implicit val timeout = system.settings.ActorTimeout
|
implicit val timeout = Timeout(5 seconds)
|
||||||
//#map-to
|
//#map-to
|
||||||
import akka.dispatch.Future
|
import akka.dispatch.Future
|
||||||
import akka.pattern.ask
|
import akka.pattern.ask
|
||||||
|
|
@ -164,7 +167,7 @@ class FutureDocSpec extends AkkaSpec {
|
||||||
val actor3 = system.actorOf(Props[MyActor])
|
val actor3 = system.actorOf(Props[MyActor])
|
||||||
val msg1 = 1
|
val msg1 = 1
|
||||||
val msg2 = 2
|
val msg2 = 2
|
||||||
implicit val timeout = system.settings.ActorTimeout
|
implicit val timeout = Timeout(5 seconds)
|
||||||
import akka.dispatch.Await
|
import akka.dispatch.Await
|
||||||
import akka.pattern.ask
|
import akka.pattern.ask
|
||||||
//#composing-wrong
|
//#composing-wrong
|
||||||
|
|
@ -188,7 +191,7 @@ class FutureDocSpec extends AkkaSpec {
|
||||||
val actor3 = system.actorOf(Props[MyActor])
|
val actor3 = system.actorOf(Props[MyActor])
|
||||||
val msg1 = 1
|
val msg1 = 1
|
||||||
val msg2 = 2
|
val msg2 = 2
|
||||||
implicit val timeout = system.settings.ActorTimeout
|
implicit val timeout = Timeout(5 seconds)
|
||||||
import akka.dispatch.Await
|
import akka.dispatch.Await
|
||||||
import akka.pattern.ask
|
import akka.pattern.ask
|
||||||
//#composing
|
//#composing
|
||||||
|
|
@ -208,7 +211,7 @@ class FutureDocSpec extends AkkaSpec {
|
||||||
}
|
}
|
||||||
|
|
||||||
"demonstrate usage of sequence with actors" in {
|
"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])
|
val oddActor = system.actorOf(Props[OddActor])
|
||||||
//#sequence-ask
|
//#sequence-ask
|
||||||
// oddActor returns odd numbers sequentially from 1 as a List[Future[Int]]
|
// 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 {
|
"demonstrate usage of recover" in {
|
||||||
implicit val timeout = system.settings.ActorTimeout
|
implicit val timeout = Timeout(5 seconds)
|
||||||
val actor = system.actorOf(Props[MyActor])
|
val actor = system.actorOf(Props[MyActor])
|
||||||
val msg1 = -1
|
val msg1 = -1
|
||||||
//#recover
|
//#recover
|
||||||
|
|
@ -268,7 +271,7 @@ class FutureDocSpec extends AkkaSpec {
|
||||||
}
|
}
|
||||||
|
|
||||||
"demonstrate usage of recoverWith" in {
|
"demonstrate usage of recoverWith" in {
|
||||||
implicit val timeout = system.settings.ActorTimeout
|
implicit val timeout = Timeout(5 seconds)
|
||||||
val actor = system.actorOf(Props[MyActor])
|
val actor = system.actorOf(Props[MyActor])
|
||||||
val msg1 = -1
|
val msg1 = -1
|
||||||
//#try-recover
|
//#try-recover
|
||||||
|
|
|
||||||
|
|
@ -28,7 +28,7 @@ class RemoteDeploymentDocSpec extends AkkaSpec("""
|
||||||
import RemoteDeploymentDocSpec._
|
import RemoteDeploymentDocSpec._
|
||||||
|
|
||||||
val other = ActorSystem("remote", system.settings.config)
|
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() }
|
override def atTermination() { other.shutdown() }
|
||||||
|
|
||||||
|
|
|
||||||
29
akka-docs/scala/code/akka/docs/routing/RouterDocSpec.scala
Normal file
29
akka-docs/scala/code/akka/docs/routing/RouterDocSpec.scala
Normal 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
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -7,6 +7,7 @@ import akka.routing.{ ScatterGatherFirstCompletedRouter, BroadcastRouter, Random
|
||||||
import annotation.tailrec
|
import annotation.tailrec
|
||||||
import akka.actor.{ Props, Actor }
|
import akka.actor.{ Props, Actor }
|
||||||
import akka.util.duration._
|
import akka.util.duration._
|
||||||
|
import akka.util.Timeout
|
||||||
import akka.dispatch.Await
|
import akka.dispatch.Await
|
||||||
import akka.pattern.ask
|
import akka.pattern.ask
|
||||||
import akka.routing.SmallestMailboxRouter
|
import akka.routing.SmallestMailboxRouter
|
||||||
|
|
@ -80,7 +81,7 @@ class ParentActor extends Actor {
|
||||||
val scatterGatherFirstCompletedRouter = context.actorOf(
|
val scatterGatherFirstCompletedRouter = context.actorOf(
|
||||||
Props[FibonacciActor].withRouter(ScatterGatherFirstCompletedRouter(
|
Props[FibonacciActor].withRouter(ScatterGatherFirstCompletedRouter(
|
||||||
nrOfInstances = 5, within = 2 seconds)), "router")
|
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 futureResult = scatterGatherFirstCompletedRouter ? FibonacciNumber(10)
|
||||||
val result = Await.result(futureResult, timeout.duration)
|
val result = Await.result(futureResult, timeout.duration)
|
||||||
//#scatterGatherFirstCompletedRouter
|
//#scatterGatherFirstCompletedRouter
|
||||||
|
|
|
||||||
|
|
@ -35,8 +35,7 @@ class MyOwnSerializer extends Serializer {
|
||||||
// using the type hint (if any, see "includeManifest" above)
|
// using the type hint (if any, see "includeManifest" above)
|
||||||
// into the optionally provided classLoader.
|
// into the optionally provided classLoader.
|
||||||
def fromBinary(bytes: Array[Byte],
|
def fromBinary(bytes: Array[Byte],
|
||||||
clazz: Option[Class[_]],
|
clazz: Option[Class[_]]): AnyRef = {
|
||||||
classLoader: Option[ClassLoader] = None): AnyRef = {
|
|
||||||
// Put your code that deserializes here
|
// Put your code that deserializes here
|
||||||
//#...
|
//#...
|
||||||
null
|
null
|
||||||
|
|
@ -143,9 +142,7 @@ class SerializationDocSpec extends AkkaSpec {
|
||||||
val bytes = serializer.toBinary(original)
|
val bytes = serializer.toBinary(original)
|
||||||
|
|
||||||
// Turn it back into an object
|
// Turn it back into an object
|
||||||
val back = serializer.fromBinary(bytes,
|
val back = serializer.fromBinary(bytes, manifest = None)
|
||||||
manifest = None,
|
|
||||||
classLoader = None)
|
|
||||||
|
|
||||||
// Voilá!
|
// Voilá!
|
||||||
back must equal(original)
|
back must equal(original)
|
||||||
|
|
|
||||||
195
akka-docs/scala/code/akka/docs/zeromq/ZeromqDocSpec.scala
Normal file
195
akka-docs/scala/code/akka/docs/zeromq/ZeromqDocSpec.scala
Normal 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
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -8,11 +8,6 @@ Routing (Scala)
|
||||||
|
|
||||||
.. contents:: :local:
|
.. 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.
|
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'.
|
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
|
*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.*
|
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
|
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
|
in ``resizer`` method of the ``RouterConfig``. See ``akka.routing.DefaultResizer`` for inspiration
|
||||||
of how to write your own resize strategy.
|
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 system’s 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
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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
|
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.
|
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.
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -1,8 +1,10 @@
|
||||||
|
|
||||||
.. _zeromq-module:
|
.. _zeromq-scala:
|
||||||
|
|
||||||
|
################
|
||||||
|
ZeroMQ (Scala)
|
||||||
|
################
|
||||||
|
|
||||||
ZeroMQ
|
|
||||||
======
|
|
||||||
|
|
||||||
.. sidebar:: Contents
|
.. sidebar:: Contents
|
||||||
|
|
||||||
|
|
@ -12,83 +14,76 @@ Akka provides a ZeroMQ module which abstracts a ZeroMQ connection and therefore
|
||||||
|
|
||||||
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.
|
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 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.
|
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
|
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._
|
or by importing the ``akka.zeromq._`` package to make newSocket method available on system, via an implicit conversion.
|
||||||
val socket = system.zeromq.newSocket(SocketType.Pub, Bind("tcp://127.0.0.1:1234"))
|
|
||||||
|
|
||||||
will create a ZeroMQ Publisher socket that is Bound to the port 1234 on localhost.
|
.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#pub-socket2
|
||||||
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:
|
|
||||||
|
|
||||||
.. 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
|
.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#sub-socket
|
||||||
|
|
||||||
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"))
|
|
||||||
|
|
||||||
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>`_.
|
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
|
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.
|
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:
|
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._
|
The following example illustrates one publisher with two subscribers.
|
||||||
val socket = system.zeromq.newSocket(SocketType.Pub, Bind("tcp://127.0.0.1:1234"))
|
|
||||||
socket ! Send("hello".getBytes)
|
|
||||||
|
|
||||||
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._
|
Let's add one subscriber that logs the information. It subscribes to all topics starting with ``"health"``, i.e. both ``Heap`` and
|
||||||
val listener = system.actorOf(Props(new Actor {
|
``Load`` events.
|
||||||
def receive: Receive = {
|
|
||||||
case Connecting => ...
|
.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#logger
|
||||||
case _ => ...
|
|
||||||
}
|
Another subscriber keep track of used heap and warns if too much heap is used. It only subscribes to ``Heap`` events.
|
||||||
}))
|
|
||||||
val socket = system.zeromq.newSocket(SocketType.Sub, Listener(listener), Connect("tcp://127.0.0.1:1234"), SubscribeAll)
|
.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#alerter
|
||||||
|
|
||||||
Router-Dealer connection
|
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.
|
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.
|
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:
|
To create a Router socket that has a high watermark configured, you would do:
|
||||||
|
|
||||||
.. code-block:: scala
|
.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#high-watermark
|
||||||
|
|
||||||
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))
|
|
||||||
|
|
||||||
The akka-zeromq module accepts most if not all the available configuration options for a zeromq socket.
|
The akka-zeromq module accepts most if not all the available configuration options for a zeromq socket.
|
||||||
|
|
|
||||||
|
|
@ -22,7 +22,7 @@ class BeanstalkBasedMailboxType(config: Config) extends MailboxType {
|
||||||
/**
|
/**
|
||||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
* @author <a href="http://jonasboner.com">Jonas Boné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 settings = BeanstalkBasedMailboxExtension(owner.system)
|
||||||
private val messageSubmitDelaySeconds = settings.MessageSubmitDelay.toSeconds.toInt
|
private val messageSubmitDelaySeconds = settings.MessageSubmitDelay.toSeconds.toInt
|
||||||
|
|
|
||||||
|
|
@ -17,7 +17,7 @@ class FileBasedMailboxType(config: Config) extends MailboxType {
|
||||||
override def create(owner: ActorContext) = new FileBasedMailbox(owner)
|
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")
|
val log = Logging(system, "FileBasedMailbox")
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -12,7 +12,7 @@ private[akka] object DurableExecutableMailboxConfig {
|
||||||
val Name = "[\\.\\/\\$\\s]".r
|
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._
|
import DurableExecutableMailboxConfig._
|
||||||
|
|
||||||
def system: ExtendedActorSystem = owner.system.asInstanceOf[ExtendedActorSystem]
|
def system: ExtendedActorSystem = owner.system.asInstanceOf[ExtendedActorSystem]
|
||||||
|
|
@ -22,15 +22,13 @@ abstract class DurableMailbox(owner: ActorContext) extends CustomMailbox(owner)
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
trait DurableMessageSerialization {
|
trait DurableMessageSerialization { this: DurableMailbox ⇒
|
||||||
|
|
||||||
def owner: ActorContext
|
|
||||||
|
|
||||||
def serialize(durableMessage: Envelope): Array[Byte] = {
|
def serialize(durableMessage: Envelope): Array[Byte] = {
|
||||||
|
|
||||||
def serializeActorRef(ref: ActorRef): ActorRefProtocol = ActorRefProtocol.newBuilder.setPath(ref.path.toString).build
|
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
|
val builder = RemoteMessageProtocol.newBuilder
|
||||||
.setMessage(message)
|
.setMessage(message)
|
||||||
.setRecipient(serializeActorRef(owner.self))
|
.setRecipient(serializeActorRef(owner.self))
|
||||||
|
|
@ -41,13 +39,13 @@ trait DurableMessageSerialization {
|
||||||
|
|
||||||
def deserialize(bytes: Array[Byte]): Envelope = {
|
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 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)
|
val sender = deserializeActorRef(durableMessage.getSender)
|
||||||
|
|
||||||
new Envelope(message, sender)(owner.system)
|
new Envelope(message, sender)(system)
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -65,7 +65,7 @@ class BSONSerializableMailbox(system: ExtendedActorSystem) extends SerializableB
|
||||||
val doc = deserializer.decodeAndFetch(in).asInstanceOf[BSONDocument]
|
val doc = deserializer.decodeAndFetch(in).asInstanceOf[BSONDocument]
|
||||||
system.log.debug("Deserializing a durable message from MongoDB: {}", doc)
|
system.log.debug("Deserializing a durable message from MongoDB: {}", doc)
|
||||||
val msgData = MessageProtocol.parseFrom(doc.as[org.bson.types.Binary]("message").getData)
|
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 ownerPath = doc.as[String]("ownerPath")
|
||||||
val senderPath = doc.as[String]("senderPath")
|
val senderPath = doc.as[String]("senderPath")
|
||||||
val sender = system.actorFor(senderPath)
|
val sender = system.actorFor(senderPath)
|
||||||
|
|
|
||||||
|
|
@ -32,7 +32,7 @@ class MongoBasedMailboxType(config: Config) extends MailboxType {
|
||||||
*
|
*
|
||||||
* @author <a href="http://evilmonkeylabs.com">Brendan W. McAdams</a>
|
* @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
|
// this implicit object provides the context for reading/writing things as MongoDurableMessage
|
||||||
implicit val mailboxBSONSer = new BSONSerializableMailbox(system)
|
implicit val mailboxBSONSer = new BSONSerializableMailbox(system)
|
||||||
implicit val safeWrite = WriteConcern.Safe // TODO - Replica Safe when appropriate!
|
implicit val safeWrite = WriteConcern.Safe // TODO - Replica Safe when appropriate!
|
||||||
|
|
|
||||||
|
|
@ -19,7 +19,7 @@ class RedisBasedMailboxType(config: Config) extends MailboxType {
|
||||||
override def create(owner: ActorContext) = new RedisBasedMailbox(owner)
|
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)
|
private val settings = RedisBasedMailboxExtension(owner.system)
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -20,7 +20,7 @@ class ZooKeeperBasedMailboxType(config: Config) extends MailboxType {
|
||||||
override def create(owner: ActorContext) = new ZooKeeperBasedMailbox(owner)
|
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)
|
private val settings = ZooKeeperBasedMailboxExtension(owner.system)
|
||||||
val queueNode = "/queues"
|
val queueNode = "/queues"
|
||||||
|
|
|
||||||
|
|
@ -6,32 +6,25 @@ package akka.remote
|
||||||
|
|
||||||
import akka.remote.RemoteProtocol._
|
import akka.remote.RemoteProtocol._
|
||||||
import com.google.protobuf.ByteString
|
import com.google.protobuf.ByteString
|
||||||
import akka.actor.ActorSystem
|
import akka.actor.ExtendedActorSystem
|
||||||
import akka.serialization.SerializationExtension
|
import akka.serialization.SerializationExtension
|
||||||
import akka.util.ReflectiveAccess
|
|
||||||
|
|
||||||
object MessageSerializer {
|
object MessageSerializer {
|
||||||
|
|
||||||
def deserialize(system: ActorSystem, messageProtocol: MessageProtocol, classLoader: ClassLoader): AnyRef = {
|
def deserialize(system: ExtendedActorSystem, messageProtocol: MessageProtocol): AnyRef = {
|
||||||
val clazz = if (messageProtocol.hasMessageManifest) {
|
val clazz =
|
||||||
Option(ReflectiveAccess.getClassFor[AnyRef](
|
if (messageProtocol.hasMessageManifest) {
|
||||||
messageProtocol.getMessageManifest.toStringUtf8,
|
system.dynamicAccess.getClassFor[AnyRef](messageProtocol.getMessageManifest.toStringUtf8)
|
||||||
classLoader) match {
|
.fold(throw _, Some(_))
|
||||||
case Left(e) ⇒ throw e
|
} else None
|
||||||
case Right(r) ⇒ r
|
SerializationExtension(system)
|
||||||
})
|
.deserialize(messageProtocol.getMessage.toByteArray, messageProtocol.getSerializerId, clazz) match {
|
||||||
} else None
|
|
||||||
SerializationExtension(system).deserialize(
|
|
||||||
messageProtocol.getMessage.toByteArray,
|
|
||||||
messageProtocol.getSerializerId,
|
|
||||||
clazz,
|
|
||||||
classLoader) match {
|
|
||||||
case Left(e) ⇒ throw e
|
case Left(e) ⇒ throw e
|
||||||
case Right(r) ⇒ r
|
case Right(r) ⇒ r
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
def serialize(system: ActorSystem, message: AnyRef): MessageProtocol = {
|
def serialize(system: ExtendedActorSystem, message: AnyRef): MessageProtocol = {
|
||||||
val s = SerializationExtension(system)
|
val s = SerializationExtension(system)
|
||||||
val serializer = s.findSerializerFor(message)
|
val serializer = s.findSerializerFor(message)
|
||||||
val builder = MessageProtocol.newBuilder
|
val builder = MessageProtocol.newBuilder
|
||||||
|
|
|
||||||
|
|
@ -12,7 +12,6 @@ import akka.event.EventStream
|
||||||
import akka.config.ConfigurationException
|
import akka.config.ConfigurationException
|
||||||
import java.util.concurrent.{ TimeoutException }
|
import java.util.concurrent.{ TimeoutException }
|
||||||
import com.typesafe.config.Config
|
import com.typesafe.config.Config
|
||||||
import akka.util.ReflectiveAccess
|
|
||||||
import akka.serialization.Serialization
|
import akka.serialization.Serialization
|
||||||
import akka.serialization.SerializationExtension
|
import akka.serialization.SerializationExtension
|
||||||
|
|
||||||
|
|
@ -28,11 +27,11 @@ class RemoteActorRefProvider(
|
||||||
val settings: ActorSystem.Settings,
|
val settings: ActorSystem.Settings,
|
||||||
val eventStream: EventStream,
|
val eventStream: EventStream,
|
||||||
val scheduler: Scheduler,
|
val scheduler: Scheduler,
|
||||||
val classloader: ClassLoader) extends ActorRefProvider {
|
val dynamicAccess: DynamicAccess) extends ActorRefProvider {
|
||||||
|
|
||||||
val remoteSettings = new RemoteSettings(settings.config, systemName)
|
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)
|
private val local = new LocalActorRefProvider(systemName, settings, eventStream, scheduler, deployer)
|
||||||
|
|
||||||
|
|
@ -84,7 +83,7 @@ class RemoteActorRefProvider(
|
||||||
classOf[ActorSystemImpl] -> system,
|
classOf[ActorSystemImpl] -> system,
|
||||||
classOf[RemoteActorRefProvider] -> this)
|
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 Left(problem) ⇒ throw new RemoteTransportException("Could not load remote transport layer " + fqn, problem)
|
||||||
case Right(remote) ⇒ remote
|
case Right(remote) ⇒ remote
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -12,7 +12,7 @@ case class RemoteScope(node: Address) extends Scope {
|
||||||
def withFallback(other: Scope): Scope = this
|
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] = {
|
override protected def parseConfig(path: String, config: Config): Option[Deploy] = {
|
||||||
import scala.collection.JavaConverters._
|
import scala.collection.JavaConverters._
|
||||||
|
|
|
||||||
|
|
@ -219,7 +219,7 @@ class RemoteMessage(input: RemoteMessageProtocol, system: ActorSystemImpl) {
|
||||||
|
|
||||||
lazy val recipient: InternalActorRef = system.provider.actorFor(system.provider.rootGuardian, originalReceiver)
|
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
|
override def toString = "RemoteMessage: " + payload + " to " + recipient + "<+{" + originalReceiver + "} from " + sender
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -61,7 +61,7 @@ class NettyRemoteTransport(val remoteSettings: RemoteSettings, val system: Actor
|
||||||
case sa: InetSocketAddress ⇒ sa
|
case sa: InetSocketAddress ⇒ sa
|
||||||
case x ⇒ throw new RemoteTransportException("unknown local address type " + x.getClass, null)
|
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
|
def address = _address.get
|
||||||
|
|
|
||||||
|
|
@ -183,7 +183,7 @@ class RemoteServerHandler(
|
||||||
instruction.getCommandType match {
|
instruction.getCommandType match {
|
||||||
case CommandType.CONNECT if settings.UsePassiveConnections ⇒
|
case CommandType.CONNECT if settings.UsePassiveConnections ⇒
|
||||||
val origin = instruction.getOrigin
|
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)
|
val client = new PassiveRemoteClient(event.getChannel, netty, inbound)
|
||||||
netty.bindClient(inbound, client)
|
netty.bindClient(inbound, client)
|
||||||
case CommandType.SHUTDOWN ⇒ //Will be unbound in channelClosed
|
case CommandType.SHUTDOWN ⇒ //Will be unbound in channelClosed
|
||||||
|
|
@ -203,7 +203,7 @@ class RemoteServerHandler(
|
||||||
|
|
||||||
private def getClientAddress(c: Channel): Option[Address] =
|
private def getClientAddress(c: Channel): Option[Address] =
|
||||||
c.getRemoteAddress match {
|
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
|
case _ ⇒ None
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -40,7 +40,7 @@ class NettySettings(config: Config, val systemName: String) {
|
||||||
case value ⇒ value
|
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 PortSelector = getInt("port")
|
||||||
|
|
||||||
val ConnectionTimeout = Duration(getMilliseconds("connection-timeout"), MILLISECONDS)
|
val ConnectionTimeout = Duration(getMilliseconds("connection-timeout"), MILLISECONDS)
|
||||||
|
|
|
||||||
|
|
@ -30,6 +30,8 @@ case class RemoteRouterConfig(local: RouterConfig, nodes: Iterable[String]) exte
|
||||||
|
|
||||||
override def createActor(): Router = local.createActor()
|
override def createActor(): Router = local.createActor()
|
||||||
|
|
||||||
|
override def routerDispatcher: String = local.routerDispatcher
|
||||||
|
|
||||||
override def resizer: Option[Resizer] = local.resizer
|
override def resizer: Option[Resizer] = local.resizer
|
||||||
|
|
||||||
override def withFallback(other: RouterConfig): RouterConfig = other match {
|
override def withFallback(other: RouterConfig): RouterConfig = other match {
|
||||||
|
|
|
||||||
|
|
@ -5,6 +5,7 @@
|
||||||
package akka.serialization
|
package akka.serialization
|
||||||
|
|
||||||
import com.google.protobuf.Message
|
import com.google.protobuf.Message
|
||||||
|
import akka.actor.DynamicAccess
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This Serializer serializes `com.google.protobuf.Message`s
|
* 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 + "]")
|
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 {
|
clazz match {
|
||||||
case None ⇒ throw new IllegalArgumentException("Need a protobuf message class to be able to serialize bytes using protobuf")
|
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]
|
case Some(c) ⇒ c.getDeclaredMethod("parseFrom", ARRAY_OF_BYTE_ARRAY: _*).invoke(null, bytes).asInstanceOf[Message]
|
||||||
|
|
|
||||||
|
|
@ -44,7 +44,7 @@ akka {
|
||||||
/looker/child/grandchild.remote = "akka://RemoteCommunicationSpec@localhost:12345"
|
/looker/child/grandchild.remote = "akka://RemoteCommunicationSpec@localhost:12345"
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
""") with ImplicitSender {
|
""") with ImplicitSender with DefaultTimeout {
|
||||||
|
|
||||||
import RemoteCommunicationSpec._
|
import RemoteCommunicationSpec._
|
||||||
|
|
||||||
|
|
@ -59,8 +59,6 @@ akka {
|
||||||
|
|
||||||
val here = system.actorFor("akka://remote_sys@localhost:12346/user/echo")
|
val here = system.actorFor("akka://remote_sys@localhost:12346/user/echo")
|
||||||
|
|
||||||
implicit val timeout = system.settings.ActorTimeout
|
|
||||||
|
|
||||||
override def atTermination() {
|
override def atTermination() {
|
||||||
other.shutdown()
|
other.shutdown()
|
||||||
}
|
}
|
||||||
|
|
|
||||||
Some files were not shown because too many files have changed in this diff Show more
Loading…
Add table
Add a link
Reference in a new issue