add akka-typed project with generic ActorRef
This is the first step towards more type-safety in Actor interactions, comprising: * generic ActorRef[T] that only accepts T messages * generic ActorSystem[T] extends ActorRef[T] (sending to the guardian, whose Props[T] are provided for ActorSystem construction) * removed the Actor trait: everything in there has been made into messages and signals * new Behavior[T] abstraction that consumes messages (of type T) or Signals (lifecycle hooks, Terminated, ReceiveTimeout, Failed), producing the next Behavior[T] as the result each time * the ask pattern is provided and yields properly typed Futures * variants of ActorContext are provided for synchronous testing of Behaviors All of this is implemented without touching code outside akka-typed (apart from making guardianProps configurable), creating wrapper objects around ActorRef, ActorContext, ActorSystem, Props and providing an Actor implementation that just runs a Behavior.
This commit is contained in:
parent
50d1569f37
commit
d9efd041f7
40 changed files with 4724 additions and 21 deletions
|
|
@ -0,0 +1,40 @@
|
|||
/**
|
||||
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.util;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
/*
|
||||
* IMPORTANT: do not change this file, the line numbers are verified in LineNumberSpec
|
||||
*/
|
||||
|
||||
public class LineNumberSpecCodeForJava {
|
||||
|
||||
// @FunctionalInterface // will be uncommented as soon as '-source 1.8' is set
|
||||
public static interface F extends Serializable {
|
||||
public String doit(String arg);
|
||||
}
|
||||
|
||||
// public F f1() { // FIXME These are commented out until the build is switched to Java 8
|
||||
// return (s) -> s;
|
||||
// }
|
||||
|
||||
// public F f2() {
|
||||
// return (s) -> {
|
||||
// System.out.println(s);
|
||||
// return s;
|
||||
// };
|
||||
// }
|
||||
|
||||
public F f3() {
|
||||
return new F() {
|
||||
private static final long serialVersionUID = 1L;
|
||||
@Override
|
||||
public String doit(String arg) {
|
||||
return arg;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -373,6 +373,20 @@ class ActorSystemSpec extends AkkaSpec(ActorSystemSpec.config) with ImplicitSend
|
|||
shutdown(system2)
|
||||
}
|
||||
}
|
||||
|
||||
"not allow top-level actor creation with custom guardian" in {
|
||||
val sys = new ActorSystemImpl("custom", ConfigFactory.defaultReference(),
|
||||
getClass.getClassLoader, None, Some(Props.empty))
|
||||
sys.start()
|
||||
try {
|
||||
intercept[UnsupportedOperationException] {
|
||||
sys.actorOf(Props.empty)
|
||||
}
|
||||
intercept[UnsupportedOperationException] {
|
||||
sys.actorOf(Props.empty, "empty")
|
||||
}
|
||||
} finally shutdown(sys)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -0,0 +1,50 @@
|
|||
/**
|
||||
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.util
|
||||
|
||||
import akka.testkit.AkkaSpec
|
||||
import LineNumbers._
|
||||
|
||||
class LineNumberSpec extends AkkaSpec {
|
||||
|
||||
"LineNumbers" when {
|
||||
|
||||
"writing Scala" must {
|
||||
import LineNumberSpecCodeForScala._
|
||||
|
||||
"work for small functions" in {
|
||||
LineNumbers(oneline) should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 12, 12))
|
||||
}
|
||||
|
||||
"work for larger functions" in {
|
||||
LineNumbers(twoline) should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 14, 16))
|
||||
}
|
||||
|
||||
"work for partial functions" in {
|
||||
LineNumbers(partial) should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 19, 21))
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
"writing Java" must {
|
||||
val l = new LineNumberSpecCodeForJava
|
||||
|
||||
// FIXME uncomment when compiling with '-source 1.8'
|
||||
// "work for small functions" in {
|
||||
// LineNumbers(l.f1()) should ===(SourceFileLines("LineNumberSpecCodeForJava.java", 20, 20))
|
||||
// }
|
||||
|
||||
// "work for larger functions" in {
|
||||
// LineNumbers(l.f2()) should ===(SourceFileLines("LineNumberSpecCodeForJava.java", 25, 26))
|
||||
// }
|
||||
|
||||
"work for anonymous classes" in {
|
||||
LineNumbers(l.f3()) should ===(SourceFileLines("LineNumberSpecCodeForJava.java", 31, 35))
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,23 @@
|
|||
/**
|
||||
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.util
|
||||
|
||||
/*
|
||||
* IMPORTANT: do not change this file, the line numbers are verified in LineNumberSpec!
|
||||
*/
|
||||
|
||||
object LineNumberSpecCodeForScala {
|
||||
|
||||
val oneline = (s: String) ⇒ println(s)
|
||||
|
||||
val twoline = (s: String) ⇒ {
|
||||
println(s)
|
||||
Integer.parseInt(s)
|
||||
}
|
||||
|
||||
val partial: PartialFunction[String, Unit] = {
|
||||
case "a" ⇒
|
||||
case "b" ⇒
|
||||
}
|
||||
}
|
||||
|
|
@ -213,6 +213,8 @@ trait ActorRefFactory {
|
|||
*
|
||||
* @throws akka.ConfigurationException if deployment, dispatcher
|
||||
* or mailbox configuration is wrong
|
||||
* @throws UnsupportedOperationException if invoked on an ActorSystem that
|
||||
* uses a custom user guardian
|
||||
*/
|
||||
def actorOf(props: Props): ActorRef
|
||||
|
||||
|
|
@ -222,10 +224,13 @@ trait ActorRefFactory {
|
|||
* an `InvalidActorNameException` is thrown.
|
||||
*
|
||||
* See [[akka.actor.Props]] for details on how to obtain a `Props` object.
|
||||
*
|
||||
* @throws akka.actor.InvalidActorNameException if the given name is
|
||||
* invalid or already in use
|
||||
* @throws akka.ConfigurationException if deployment, dispatcher
|
||||
* or mailbox configuration is wrong
|
||||
* @throws UnsupportedOperationException if invoked on an ActorSystem that
|
||||
* uses a custom user guardian
|
||||
*/
|
||||
def actorOf(props: Props, name: String): ActorRef
|
||||
|
||||
|
|
@ -587,7 +592,7 @@ private[akka] class LocalActorRefProvider private[akka] (
|
|||
override lazy val guardian: LocalActorRef = {
|
||||
val cell = rootGuardian.underlying
|
||||
cell.reserveChild("user")
|
||||
val ref = new LocalActorRef(system, Props(classOf[LocalActorRefProvider.Guardian], guardianStrategy),
|
||||
val ref = new LocalActorRef(system, system.guardianProps.getOrElse(Props(classOf[LocalActorRefProvider.Guardian], guardianStrategy)),
|
||||
defaultDispatcher, defaultMailbox, rootGuardian, rootPath / "user")
|
||||
cell.initChild(ref)
|
||||
ref.start()
|
||||
|
|
|
|||
|
|
@ -140,7 +140,7 @@ object ActorSystem {
|
|||
def apply(name: String, config: Option[Config] = None, classLoader: Option[ClassLoader] = None, defaultExecutionContext: Option[ExecutionContext] = None): ActorSystem = {
|
||||
val cl = classLoader.getOrElse(findClassLoader())
|
||||
val appConfig = config.getOrElse(ConfigFactory.load(cl))
|
||||
new ActorSystemImpl(name, appConfig, cl, defaultExecutionContext).start()
|
||||
new ActorSystemImpl(name, appConfig, cl, defaultExecutionContext, None).start()
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -519,7 +519,12 @@ abstract class ExtendedActorSystem extends ActorSystem {
|
|||
|
||||
}
|
||||
|
||||
private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config, classLoader: ClassLoader, defaultExecutionContext: Option[ExecutionContext]) extends ExtendedActorSystem {
|
||||
private[akka] class ActorSystemImpl(
|
||||
val name: String,
|
||||
applicationConfig: Config,
|
||||
classLoader: ClassLoader,
|
||||
defaultExecutionContext: Option[ExecutionContext],
|
||||
val guardianProps: Option[Props]) extends ExtendedActorSystem {
|
||||
|
||||
if (!name.matches("""^[a-zA-Z0-9][a-zA-Z0-9-_]*$"""))
|
||||
throw new IllegalArgumentException(
|
||||
|
|
@ -577,9 +582,13 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
|
|||
|
||||
def systemActorOf(props: Props, name: String): ActorRef = systemGuardian.underlying.attachChild(props, name, systemService = true)
|
||||
|
||||
def actorOf(props: Props, name: String): ActorRef = guardian.underlying.attachChild(props, name, systemService = false)
|
||||
def actorOf(props: Props, name: String): ActorRef =
|
||||
if (guardianProps.isEmpty) guardian.underlying.attachChild(props, name, systemService = false)
|
||||
else throw new UnsupportedOperationException("cannot create top-level actor from the outside on ActorSystem with custom user guardian")
|
||||
|
||||
def actorOf(props: Props): ActorRef = guardian.underlying.attachChild(props, systemService = false)
|
||||
def actorOf(props: Props): ActorRef =
|
||||
if (guardianProps.isEmpty) guardian.underlying.attachChild(props, systemService = false)
|
||||
else throw new UnsupportedOperationException("cannot create top-level actor from the outside on ActorSystem with custom user guardian")
|
||||
|
||||
def stop(actor: ActorRef): Unit = {
|
||||
val path = actor.path
|
||||
|
|
|
|||
|
|
@ -138,7 +138,7 @@ final class AskableActorRef(val actorRef: ActorRef) extends AnyVal {
|
|||
if (timeout.duration.length <= 0)
|
||||
Future.failed[Any](new IllegalArgumentException(s"""Timeout length must not be negative, question not sent to [$actorRef]. Sender[$sender] sent the message of type "${message.getClass.getName}"."""))
|
||||
else {
|
||||
val a = PromiseActorRef(ref.provider, timeout, targetName = actorRef, message, sender)
|
||||
val a = PromiseActorRef(ref.provider, timeout, targetName = actorRef, message.getClass.getName, sender)
|
||||
actorRef.tell(message, a)
|
||||
a.result.future
|
||||
}
|
||||
|
|
@ -159,7 +159,7 @@ final class AskableActorSelection(val actorSel: ActorSelection) extends AnyVal {
|
|||
Future.failed[Any](
|
||||
new IllegalArgumentException(s"""Timeout length must not be negative, question not sent to [$actorSel]. Sender[$sender] sent the message of type "${message.getClass.getName}"."""))
|
||||
else {
|
||||
val a = PromiseActorRef(ref.provider, timeout, targetName = actorSel, message, sender)
|
||||
val a = PromiseActorRef(ref.provider, timeout, targetName = actorSel, message.getClass.getName, sender)
|
||||
actorSel.tell(message, a)
|
||||
a.result.future
|
||||
}
|
||||
|
|
@ -175,12 +175,15 @@ final class AskableActorSelection(val actorSel: ActorSelection) extends AnyVal {
|
|||
*
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] final class PromiseActorRef private (val provider: ActorRefProvider, val result: Promise[Any])
|
||||
private[akka] final class PromiseActorRef private (val provider: ActorRefProvider, val result: Promise[Any], _mcn: String)
|
||||
extends MinimalActorRef {
|
||||
import PromiseActorRef._
|
||||
import AbstractPromiseActorRef.stateOffset
|
||||
import AbstractPromiseActorRef.watchedByOffset
|
||||
|
||||
// This is necessary for weaving the PromiseActorRef into the asked message, i.e. the replyTo pattern.
|
||||
@volatile var messageClassName = _mcn
|
||||
|
||||
/**
|
||||
* As an optimization for the common (local) case we only register this PromiseActorRef
|
||||
* with the provider when the `path` member is actually queried, which happens during
|
||||
|
|
@ -330,15 +333,14 @@ private[akka] object PromiseActorRef {
|
|||
|
||||
private val ActorStopResult = Failure(new ActorKilledException("Stopped"))
|
||||
|
||||
def apply(provider: ActorRefProvider, timeout: Timeout, targetName: Any, message: Any, sender: ActorRef = Actor.noSender): PromiseActorRef = {
|
||||
def apply(provider: ActorRefProvider, timeout: Timeout, targetName: Any, messageClassName: String, sender: ActorRef = Actor.noSender): PromiseActorRef = {
|
||||
val result = Promise[Any]()
|
||||
val scheduler = provider.guardian.underlying.system.scheduler
|
||||
val a = new PromiseActorRef(provider, result)
|
||||
val a = new PromiseActorRef(provider, result, messageClassName)
|
||||
implicit val ec = a.internalCallingThreadExecutionContext
|
||||
val messageClassName = message.getClass.getName
|
||||
val f = scheduler.scheduleOnce(timeout.duration) {
|
||||
result tryComplete Failure(
|
||||
new AskTimeoutException(s"""Ask timed out on [$targetName] after [${timeout.duration.toMillis} ms]. Sender[$sender] sent message of type "$messageClassName"."""))
|
||||
new AskTimeoutException(s"""Ask timed out on [$targetName] after [${timeout.duration.toMillis} ms]. Sender[$sender] sent message of type "${a.messageClassName}"."""))
|
||||
}
|
||||
result.future onComplete { _ ⇒ try a.stop() finally f.cancel() }
|
||||
a
|
||||
|
|
|
|||
|
|
@ -49,7 +49,7 @@ trait GracefulStopSupport {
|
|||
if (target.isTerminated) Future successful true
|
||||
else {
|
||||
val internalTarget = target.asInstanceOf[InternalActorRef]
|
||||
val ref = PromiseActorRef(internalTarget.provider, Timeout(timeout), targetName = target, message = stopMessage)
|
||||
val ref = PromiseActorRef(internalTarget.provider, Timeout(timeout), target, stopMessage.getClass.getName)
|
||||
internalTarget.sendSystemMessage(Watch(internalTarget, ref))
|
||||
target.tell(stopMessage, Actor.noSender)
|
||||
ref.result.future.transform(
|
||||
|
|
|
|||
359
akka-actor/src/main/scala/akka/util/LineNumbers.scala
Normal file
359
akka-actor/src/main/scala/akka/util/LineNumbers.scala
Normal file
|
|
@ -0,0 +1,359 @@
|
|||
/**
|
||||
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.util
|
||||
|
||||
import java.io.DataInputStream
|
||||
import java.io.BufferedInputStream
|
||||
import scala.annotation.{ switch, tailrec }
|
||||
import scala.util.control.NonFatal
|
||||
import java.io.InputStream
|
||||
import scala.util.Try
|
||||
|
||||
/**
|
||||
* This is a minimized byte-code parser that concentrates exclusively on line
|
||||
* numbers and source file extraction. It works for all normal classes up to
|
||||
* format 52:0 (JDK8), and it also works for Lambdas that are Serializable. The
|
||||
* latter restriction is due to the fact that the proxy object generated by
|
||||
* LambdaMetafactory otherwise contains no information about which method backs
|
||||
* this particular lambda (and there might be multiple defined within a single
|
||||
* class).
|
||||
*/
|
||||
object LineNumbers {
|
||||
|
||||
sealed trait Result
|
||||
case object NoSourceInfo extends Result
|
||||
final case class UnknownSourceFormat(explanation: String) extends Result
|
||||
final case class SourceFile(filename: String) extends Result {
|
||||
override def toString = filename
|
||||
}
|
||||
final case class SourceFileLines(filename: String, from: Int, to: Int) extends Result {
|
||||
override def toString = if (from != to) s"$filename:$from-$to" else s"$filename:$from"
|
||||
}
|
||||
|
||||
/**
|
||||
* Scala API: Obtain line number information for the class defining the given object.
|
||||
* This is done by reading the byte code (a potentially blocking IO operation)
|
||||
* and interpreting the debug information that it may contain.
|
||||
*
|
||||
* This does not work for Java 8 lambdas that are not Serializable, because
|
||||
* the language designers have consciously made it impossible to obtain the
|
||||
* byte code for those.
|
||||
*/
|
||||
// FIXME: this needs memoization with an LRU cache
|
||||
def apply(obj: AnyRef): Result = forObject(obj)
|
||||
|
||||
/**
|
||||
* Java API: Obtain line number information for the class defining the given object.
|
||||
* This is done by reading the byte code (a potentially blocking IO operation)
|
||||
* and interpreting the debug information that it may contain.
|
||||
*
|
||||
* This does not work for Java 8 lambdas that are not Serializable, because
|
||||
* the language designers have consciously made it impossible to obtain the
|
||||
* byte code for those.
|
||||
*/
|
||||
def `for`(obj: AnyRef): Result = apply(obj)
|
||||
|
||||
/**
|
||||
* Extract source information if available and format a string to identify the
|
||||
* class definition in question. This will include the package name and either
|
||||
* source file information or the class name.
|
||||
*/
|
||||
def prettyName(obj: AnyRef): String =
|
||||
apply(obj) match {
|
||||
case NoSourceInfo ⇒ obj.getClass.getName
|
||||
case UnknownSourceFormat(msg) ⇒ s"${obj.getClass.getName}($msg)"
|
||||
case SourceFile(f) ⇒ s"${obj.getClass.getName}($f)"
|
||||
case l: SourceFileLines ⇒ s"${obj.getClass.getPackage.getName}/$l"
|
||||
}
|
||||
|
||||
/*
|
||||
* IMPLEMENTATION
|
||||
*/
|
||||
|
||||
// compile-time constant; conditionals below will be elided if false
|
||||
private final val debug = false
|
||||
|
||||
private class Constants(count: Int) {
|
||||
|
||||
private var _fwd = Map.empty[Int, String]
|
||||
private var _rev = Map.empty[String, Int]
|
||||
private var _xref = Map.empty[Int, Int]
|
||||
|
||||
def fwd: Map[Int, String] = _fwd
|
||||
def rev: Map[String, Int] = _rev
|
||||
|
||||
private var nextIdx = 1
|
||||
|
||||
def isDone: Boolean = nextIdx >= count
|
||||
|
||||
def apply(idx: Int): String = _fwd(idx)
|
||||
def apply(str: String): Int = _rev(str)
|
||||
|
||||
def resolve(): Unit = _xref foreach (p ⇒ put(p._1, apply(p._2)))
|
||||
def contains(str: String): Boolean = _rev contains str
|
||||
|
||||
private def put(idx: Int, str: String): Unit = {
|
||||
if (!(_rev contains str)) _rev = _rev.updated(str, idx)
|
||||
_fwd = _fwd.updated(idx, str)
|
||||
}
|
||||
|
||||
def readOne(d: DataInputStream): Unit =
|
||||
(d.readByte(): @switch) match {
|
||||
case 1 ⇒ // Utf8
|
||||
val str = d.readUTF()
|
||||
put(nextIdx, str)
|
||||
nextIdx += 1
|
||||
case 3 ⇒ // Integer
|
||||
skip(d, 4)
|
||||
nextIdx += 1
|
||||
case 4 ⇒ // Float
|
||||
skip(d, 4)
|
||||
nextIdx += 1
|
||||
case 5 ⇒ // Long
|
||||
skip(d, 8)
|
||||
nextIdx += 2
|
||||
case 6 ⇒ // Double
|
||||
skip(d, 8)
|
||||
nextIdx += 2
|
||||
case 7 ⇒ // Class
|
||||
val other = d.readUnsignedShort()
|
||||
_xref = _xref.updated(nextIdx, other)
|
||||
nextIdx += 1
|
||||
case 8 ⇒ // String
|
||||
skip(d, 2)
|
||||
nextIdx += 1
|
||||
case 9 ⇒ // FieldRef
|
||||
skip(d, 4) // two shorts
|
||||
nextIdx += 1
|
||||
case 10 ⇒ // MethodRef
|
||||
skip(d, 4) // two shorts
|
||||
nextIdx += 1
|
||||
case 11 ⇒ // InterfaceMethodRef
|
||||
skip(d, 4) // two shorts
|
||||
nextIdx += 1
|
||||
case 12 ⇒ // NameAndType
|
||||
skip(d, 4) // two shorts
|
||||
nextIdx += 1
|
||||
case 15 ⇒ // MethodHandle
|
||||
skip(d, 3) // a byte and a short
|
||||
nextIdx += 1
|
||||
case 16 ⇒ // MethodType
|
||||
skip(d, 2)
|
||||
nextIdx += 1
|
||||
case 18 ⇒ // InvokeDynamic
|
||||
skip(d, 4) // two shorts
|
||||
nextIdx += 1
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private def forObject(obj: AnyRef): Result =
|
||||
getStreamForClass(obj.getClass).orElse(getStreamForLambda(obj)) match {
|
||||
case None ⇒ NoSourceInfo
|
||||
case Some((stream, filter)) ⇒ getInfo(stream, filter)
|
||||
}
|
||||
|
||||
private def getInfo(stream: InputStream, filter: Option[String]): Result = {
|
||||
val dis = new DataInputStream(stream)
|
||||
|
||||
try {
|
||||
skipID(dis)
|
||||
skipVersion(dis)
|
||||
implicit val constants = getConstants(dis)
|
||||
if (debug) println(s"LNB: fwd(${constants.fwd.size}) rev(${constants.rev.size}) ${constants.fwd.keys.toList.sorted}")
|
||||
skipClassInfo(dis)
|
||||
skipInterfaceInfo(dis)
|
||||
skipFields(dis)
|
||||
val lines = readMethods(dis, filter)
|
||||
val source = readAttributes(dis)
|
||||
|
||||
if (source.isEmpty) NoSourceInfo
|
||||
else lines match {
|
||||
case None ⇒ SourceFile(source.get)
|
||||
case Some((from, to)) ⇒ SourceFileLines(source.get, from, to)
|
||||
}
|
||||
|
||||
} catch {
|
||||
case NonFatal(ex) ⇒ UnknownSourceFormat(s"parse error: ${ex.getMessage}")
|
||||
} finally {
|
||||
try dis.close() catch {
|
||||
case ex: InterruptedException ⇒ throw ex
|
||||
case NonFatal(ex) ⇒ // ignore
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private def getStreamForClass(c: Class[_]): Option[(InputStream, None.type)] = {
|
||||
val resource = c.getName.replace('.', '/') + ".class"
|
||||
val cl = c.getClassLoader
|
||||
val r = cl.getResourceAsStream(resource)
|
||||
if (debug) println(s"LNB: resource '$resource' resolved to stream $r")
|
||||
Option(r).map(_ -> None)
|
||||
}
|
||||
|
||||
private def getStreamForLambda(l: AnyRef): Option[(InputStream, Some[String])] =
|
||||
try {
|
||||
val c = l.getClass
|
||||
val writeReplace = c.getDeclaredMethod("writeReplace")
|
||||
writeReplace.setAccessible(true)
|
||||
writeReplace.invoke(l) match {
|
||||
// case serialized: SerializedLambda ⇒
|
||||
// if (debug) println(s"LNB: found Lambda implemented in ${serialized.getImplClass}:${serialized.getImplMethodName}")
|
||||
// Option(c.getClassLoader.getResourceAsStream(serialized.getImplClass + ".class"))
|
||||
// .map(_ -> Some(serialized.getImplMethodName))
|
||||
case _ ⇒ None
|
||||
}
|
||||
} catch {
|
||||
case NonFatal(ex) ⇒
|
||||
if (debug) ex.printStackTrace()
|
||||
None
|
||||
}
|
||||
|
||||
private def skipID(d: DataInputStream): Unit = {
|
||||
val magic = d.readInt()
|
||||
if (debug) println(f"LNB: magic=0x$magic%08X")
|
||||
if (magic != 0xcafebabe) throw new IllegalArgumentException("not a Java class file")
|
||||
}
|
||||
|
||||
private def skipVersion(d: DataInputStream): Unit = {
|
||||
val minor = d.readShort()
|
||||
val major = d.readShort()
|
||||
if (debug) println(s"LNB: version=$major:$minor")
|
||||
}
|
||||
|
||||
private def getConstants(d: DataInputStream): Constants = {
|
||||
val count = d.readUnsignedShort()
|
||||
if (debug) println(s"LNB: reading $count constants")
|
||||
val c = new Constants(count)
|
||||
while (!c.isDone) c.readOne(d)
|
||||
c.resolve()
|
||||
c
|
||||
}
|
||||
|
||||
private def skipClassInfo(d: DataInputStream)(implicit c: Constants): Unit = {
|
||||
skip(d, 2) // access flags
|
||||
val name = d.readUnsignedShort() // class name
|
||||
skip(d, 2) // superclass name
|
||||
if (debug) println(s"LNB: class name = ${c(name)}")
|
||||
}
|
||||
|
||||
private def skipInterfaceInfo(d: DataInputStream)(implicit c: Constants): Unit = {
|
||||
val count = d.readUnsignedShort()
|
||||
for (_ ← 1 to count) {
|
||||
val intf = d.readUnsignedShort()
|
||||
if (debug) println(s"LNB: implements ${c(intf)}")
|
||||
}
|
||||
}
|
||||
|
||||
private def skipFields(d: DataInputStream)(implicit c: Constants): Unit = {
|
||||
val count = d.readUnsignedShort()
|
||||
if (debug) println(s"LNB: reading $count fields:")
|
||||
for (_ ← 1 to count) skipMethodOrField(d)
|
||||
}
|
||||
|
||||
private def skipMethodOrField(d: DataInputStream)(implicit c: Constants): Unit = {
|
||||
skip(d, 2) // access flags
|
||||
val name = d.readUnsignedShort() // name
|
||||
skip(d, 2) // signature
|
||||
val attributes = d.readUnsignedShort()
|
||||
for (_ ← 1 to attributes) skipAttribute(d)
|
||||
if (debug) println(s"LNB: ${c(name)} ($attributes attributes)")
|
||||
}
|
||||
|
||||
private def skipAttribute(d: DataInputStream): Unit = {
|
||||
skip(d, 2) // tag
|
||||
val length = d.readInt()
|
||||
skip(d, length)
|
||||
}
|
||||
|
||||
private def readMethods(d: DataInputStream, filter: Option[String])(implicit c: Constants): Option[(Int, Int)] = {
|
||||
val count = d.readUnsignedShort()
|
||||
if (debug) println(s"LNB: reading $count methods")
|
||||
if (c.contains("Code") && c.contains("LineNumberTable")) {
|
||||
(1 to count).map(_ ⇒ readMethod(d, c("Code"), c("LineNumberTable"), filter)).flatten.foldLeft(Int.MaxValue -> 0) {
|
||||
case ((low, high), (start, end)) ⇒ (Math.min(low, start), Math.max(high, end))
|
||||
} match {
|
||||
case (Int.MaxValue, 0) ⇒ None
|
||||
case other ⇒ Some(other)
|
||||
}
|
||||
} else {
|
||||
if (debug) println(s"LNB: (skipped)")
|
||||
for (_ ← 1 to count) skipMethodOrField(d)
|
||||
None
|
||||
}
|
||||
}
|
||||
|
||||
private def readMethod(d: DataInputStream,
|
||||
codeTag: Int,
|
||||
lineNumberTableTag: Int,
|
||||
filter: Option[String])(implicit c: Constants): Option[(Int, Int)] = {
|
||||
skip(d, 2) // access flags
|
||||
val name = d.readUnsignedShort() // name
|
||||
skip(d, 2) // signature
|
||||
if (debug) println(s"LNB: ${c(name)}")
|
||||
val attributes =
|
||||
for (_ ← 1 to d.readUnsignedShort()) yield {
|
||||
val tag = d.readUnsignedShort()
|
||||
val length = d.readInt()
|
||||
if (tag != codeTag || (filter.isDefined && c(name) != filter.get)) {
|
||||
skip(d, length)
|
||||
None
|
||||
} else {
|
||||
skip(d, 4) // shorts: max stack, max locals
|
||||
skip(d, d.readInt()) // skip byte-code
|
||||
// skip exception table: N records of 4 shorts (start PC, end PC, handler PC, catch type)
|
||||
skip(d, 8 * d.readUnsignedShort())
|
||||
val possibleLines =
|
||||
for (_ ← 1 to d.readUnsignedShort()) yield {
|
||||
val tag = d.readUnsignedShort()
|
||||
val length = d.readInt()
|
||||
if (tag != lineNumberTableTag) {
|
||||
skip(d, length)
|
||||
None
|
||||
} else {
|
||||
val lines =
|
||||
for (_ ← 1 to d.readUnsignedShort()) yield {
|
||||
skip(d, 2) // start PC
|
||||
d.readUnsignedShort() // finally: the line number
|
||||
}
|
||||
Some(lines.min -> lines.max)
|
||||
}
|
||||
}
|
||||
if (debug) println(s"LNB: nested attributes yielded: $possibleLines")
|
||||
possibleLines.flatten.headOption
|
||||
}
|
||||
}
|
||||
attributes.flatten.headOption
|
||||
}
|
||||
|
||||
private def readAttributes(d: DataInputStream)(implicit c: Constants): Option[String] = {
|
||||
val count = d.readUnsignedShort()
|
||||
if (debug) println(s"LNB: reading $count attributes")
|
||||
if (c contains "SourceFile") {
|
||||
val s = c("SourceFile")
|
||||
val attributes =
|
||||
for (_ ← 1 to count) yield {
|
||||
val tag = d.readUnsignedShort()
|
||||
val length = d.readInt()
|
||||
if (debug) println(s"LNB: tag ${c(tag)} ($length bytes)")
|
||||
if (tag != s) {
|
||||
skip(d, length)
|
||||
None
|
||||
} else {
|
||||
val name = d.readUnsignedShort()
|
||||
Some(c(name))
|
||||
}
|
||||
}
|
||||
if (debug) println(s"LNB: yielded $attributes")
|
||||
attributes.flatten.headOption
|
||||
} else {
|
||||
if (debug) println(s"LNB: (skipped)")
|
||||
None
|
||||
}
|
||||
}
|
||||
|
||||
private def skip(d: DataInputStream, length: Int): Unit =
|
||||
if (d.skipBytes(length) != length) throw new IllegalArgumentException("class file ends prematurely")
|
||||
|
||||
}
|
||||
|
|
@ -11,6 +11,18 @@ When migrating from earlier versions you should first follow the instructions fo
|
|||
migrating :ref:`1.3.x to 2.0.x <migration-2.0>` and then :ref:`2.0.x to 2.1.x <migration-2.1>`
|
||||
and then :ref:`2.1.x to 2.2.x <migration-2.2>` and then :ref:`2.2.x to 2.3.x <migration-2.3>`.
|
||||
|
||||
Advanced Notice: TypedActors will go away
|
||||
========================================
|
||||
|
||||
While technically not yet deprecated, the current ``akka.actor.TypedActor`` support will be superseded by
|
||||
the Akka Typed project that is currently being developed in open preview mode. If you are using TypedActors
|
||||
in your projects you are advised to look into this, as it is superior to the Active Object pattern expressed
|
||||
in TypedActors. The generic ActorRefs in Akka Typed allow the same type-safety that is afforded by
|
||||
TypedActors while retaining all the other benefits of an explicit actor model (including the ability to
|
||||
change behaviors etc.).
|
||||
|
||||
It is likely that TypedActors will be officially deprecated in the next major update of Akka and subsequently removed.
|
||||
|
||||
TestKit.remaining throws AssertionError
|
||||
=======================================
|
||||
|
||||
|
|
@ -160,6 +172,7 @@ In case you want to perserve the old behavior of providing the actor's path as t
|
|||
implement it yourself either as a helper trait or simply by overriding ``persistenceId`` as follows::
|
||||
|
||||
override def persistenceId = self.path.toStringWithoutAddress
|
||||
|
||||
Secure Cookies
|
||||
==============
|
||||
|
||||
|
|
|
|||
131
akka-docs/rst/scala/code/docs/akka/typed/IntroSpec.scala
Normal file
131
akka-docs/rst/scala/code/docs/akka/typed/IntroSpec.scala
Normal file
|
|
@ -0,0 +1,131 @@
|
|||
/**
|
||||
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package docs.akka.typed
|
||||
|
||||
//#imports
|
||||
import akka.typed._
|
||||
import akka.typed.ScalaDSL._
|
||||
import akka.typed.AskPattern._
|
||||
import scala.concurrent.Future
|
||||
import scala.concurrent.duration._
|
||||
import scala.concurrent.Await
|
||||
//#imports
|
||||
import akka.testkit.AkkaSpec
|
||||
import akka.typed.TypedSpec
|
||||
|
||||
object IntroSpec {
|
||||
|
||||
//#hello-world-actor
|
||||
object HelloWorld {
|
||||
final case class Greet(whom: String, replyTo: ActorRef[Greeted])
|
||||
final case class Greeted(whom: String)
|
||||
|
||||
val greeter = Static[Greet] { msg ⇒
|
||||
println(s"Hello ${msg.whom}!")
|
||||
msg.replyTo ! Greeted(msg.whom)
|
||||
}
|
||||
}
|
||||
//#hello-world-actor
|
||||
|
||||
//#chatroom-actor
|
||||
object ChatRoom {
|
||||
//#chatroom-protocol
|
||||
sealed trait Command
|
||||
final case class GetSession(screenName: String, replyTo: ActorRef[SessionEvent])
|
||||
extends Command
|
||||
//#chatroom-protocol
|
||||
//#chatroom-behavior
|
||||
private final case class PostSessionMessage(screenName: String, message: String)
|
||||
extends Command
|
||||
//#chatroom-behavior
|
||||
//#chatroom-protocol
|
||||
|
||||
sealed trait SessionEvent
|
||||
final case class SessionGranted(handle: ActorRef[PostMessage]) extends SessionEvent
|
||||
final case class SessionDenied(reason: String) extends SessionEvent
|
||||
final case class MessagePosted(screenName: String, message: String) extends SessionEvent
|
||||
|
||||
final case class PostMessage(message: String)
|
||||
//#chatroom-protocol
|
||||
//#chatroom-behavior
|
||||
|
||||
val behavior: Behavior[GetSession] =
|
||||
ContextAware[Command] { ctx ⇒
|
||||
var sessions = List.empty[ActorRef[SessionEvent]]
|
||||
|
||||
Static {
|
||||
case GetSession(screenName, client) ⇒
|
||||
sessions ::= client
|
||||
val wrapper = ctx.spawnAdapter {
|
||||
p: PostMessage ⇒ PostSessionMessage(screenName, p.message)
|
||||
}
|
||||
client ! SessionGranted(wrapper)
|
||||
case PostSessionMessage(screenName, message) ⇒
|
||||
val mp = MessagePosted(screenName, message)
|
||||
sessions foreach (_ ! mp)
|
||||
}
|
||||
}.narrow // only expose GetSession to the outside
|
||||
//#chatroom-behavior
|
||||
}
|
||||
//#chatroom-actor
|
||||
|
||||
}
|
||||
|
||||
class IntroSpec extends TypedSpec {
|
||||
import IntroSpec._
|
||||
|
||||
def `must say hello`(): Unit = {
|
||||
//#hello-world
|
||||
import HelloWorld._
|
||||
// using global pool since we want to run tasks after system shutdown
|
||||
import scala.concurrent.ExecutionContext.Implicits.global
|
||||
|
||||
val system: ActorSystem[Greet] = ActorSystem("hello", Props(greeter))
|
||||
|
||||
val future: Future[Greeted] = system ? (Greet("world", _))
|
||||
|
||||
for {
|
||||
greeting <- future.recover { case ex => ex.getMessage }
|
||||
done <- { println(s"result: $greeting"); system.terminate() }
|
||||
} println("system terminated")
|
||||
//#hello-world
|
||||
}
|
||||
|
||||
def `must chat`(): Unit = {
|
||||
//#chatroom-gabbler
|
||||
import ChatRoom._
|
||||
|
||||
val gabbler: Behavior[SessionEvent] =
|
||||
Total {
|
||||
case SessionDenied(reason) ⇒
|
||||
println(s"cannot start chat room session: $reason")
|
||||
Stopped
|
||||
case SessionGranted(handle) ⇒
|
||||
handle ! PostMessage("Hello World!")
|
||||
Same
|
||||
case MessagePosted(screenName, message) ⇒
|
||||
println(s"message has been posted by '$screenName': $message")
|
||||
Stopped
|
||||
}
|
||||
//#chatroom-gabbler
|
||||
|
||||
//#chatroom-main
|
||||
val main: Behavior[Unit] =
|
||||
Full {
|
||||
case Sig(ctx, PreStart) ⇒
|
||||
val chatRoom = ctx.spawn(Props(ChatRoom.behavior), "chatroom")
|
||||
val gabblerRef = ctx.spawn(Props(gabbler), "gabbler")
|
||||
ctx.watch(gabblerRef)
|
||||
chatRoom ! GetSession("ol’ Gabbler", gabblerRef)
|
||||
Same
|
||||
case Sig(_, Terminated(ref)) ⇒
|
||||
Stopped
|
||||
}
|
||||
|
||||
val system = ActorSystem("ChatRoomDemo", Props(main))
|
||||
Await.result(system.whenTerminated, 1.second)
|
||||
//#chatroom-main
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -5,7 +5,7 @@ Actors
|
|||
:maxdepth: 2
|
||||
|
||||
actors
|
||||
typed-actors
|
||||
typed
|
||||
fault-tolerance
|
||||
dispatchers
|
||||
mailboxes
|
||||
|
|
@ -14,3 +14,4 @@ Actors
|
|||
persistence
|
||||
testing
|
||||
actordsl
|
||||
typed-actors
|
||||
|
|
|
|||
|
|
@ -356,7 +356,7 @@ succeeds, the persistent actor receives a ``SaveSnapshotSuccess`` message, other
|
|||
|
||||
where ``metadata`` is of type ``SnapshotMetadata``:
|
||||
|
||||
.. includecode:: ../../../akka-persistence/src/main/scala/akka/persistence/Snapshot.scala#snapshot-metadata
|
||||
.. includecode:: ../../../akka-persistence/src/main/scala/akka/persistence/SnapshotProtocol.scala#snapshot-metadata
|
||||
|
||||
During recovery, the persistent actor is offered a previously saved snapshot via a ``SnapshotOffer`` message from
|
||||
which it can initialize internal state.
|
||||
|
|
|
|||
|
|
@ -1,6 +1,13 @@
|
|||
.. _typed-actors-scala:
|
||||
|
||||
Typed Actors
|
||||
====================
|
||||
|
||||
.. note::
|
||||
|
||||
This module will be deprecated as it will be superseded by the :ref:`typed-scala`
|
||||
project which is currently being developed in open preview mode.
|
||||
|
||||
Akka Typed Actors is an implementation of the `Active Objects <http://en.wikipedia.org/wiki/Active_object>`_ pattern.
|
||||
Essentially turning method invocations into asynchronous dispatch instead of synchronous that has been the default way since Smalltalk came out.
|
||||
|
||||
|
|
|
|||
321
akka-docs/rst/scala/typed.rst
Normal file
321
akka-docs/rst/scala/typed.rst
Normal file
|
|
@ -0,0 +1,321 @@
|
|||
.. _typed-scala:
|
||||
|
||||
##########
|
||||
Akka Typed
|
||||
##########
|
||||
|
||||
.. warning::
|
||||
|
||||
This module is currently experimental in the sense of being the subject of
|
||||
active research. This means that API or semantics can change without warning
|
||||
or deprecation period and it is not recommended to use this module in
|
||||
production just yet—you have been warned.
|
||||
|
||||
As discussed in :ref:`actor-systems` (and following chapters) Actors are about
|
||||
sending messages between independent units of computation, but how does that
|
||||
look like? In all of the following these imports are assumed:
|
||||
|
||||
.. includecode:: code/docs/akka/typed/IntroSpec.scala#imports
|
||||
|
||||
With these in place we can define our first Actor, and of course it will say
|
||||
hello!
|
||||
|
||||
.. includecode:: code/docs/akka/typed/IntroSpec.scala#hello-world-actor
|
||||
|
||||
This small piece of code defines two message types, one for commanding the
|
||||
Actor to greet someone and one that the Actor will use to confirm that it has
|
||||
done so. The :class:`Greet` type contains not only the information of whom to
|
||||
greet, it also holds an :class:`ActorRef` that the sender of the message
|
||||
supplies so that the :class:`HelloWorld` Actor can send back the confirmation
|
||||
message.
|
||||
|
||||
The behavior of the Actor is defined as the :meth:`greeter` value with the help
|
||||
of the :class:`Static` behavior constructor—there are many different ways of
|
||||
formulating behaviors as we shall see in the following. The “static” behavior
|
||||
is not capable of changing in response to a message, it will stay the same
|
||||
until the Actor is stopped by its parent.
|
||||
|
||||
The type of the messages handled by this behavior is declared to be of class
|
||||
:class:`Greet`, which implies that the supplied function’s ``msg`` argument is
|
||||
also typed as such. This is why we can access the ``whom`` and ``replyTo``
|
||||
members without needing to use a pattern match.
|
||||
|
||||
On the last line we see the :class:`HelloWorld` Actor send a message to another
|
||||
Actor, which is done using the ``!`` operator (pronounced “tell”). Since the
|
||||
``replyTo`` address is declared to be of type ``ActorRef[Greeted]`` the
|
||||
compiler will only permit us to send messages of this type, other usage will
|
||||
not be accepted.
|
||||
|
||||
The accepted message types of an Actor together with all reply types defines
|
||||
the protocol spoken by this Actor; in this case it is a simply request–reply
|
||||
protocol but Actors can model arbitrarily complex protocols when needed. The
|
||||
protocol is bundled together with the behavior that implements it in a nicely
|
||||
wrapped scope—the ``HelloWorld`` object.
|
||||
|
||||
Now we want to try out this Actor, so we must start an Actor system to host it:
|
||||
|
||||
.. includecode:: code/docs/akka/typed/IntroSpec.scala#hello-world
|
||||
|
||||
After importing the Actor’s protocol definition we start an Actor system from
|
||||
the defined behavior, wrapping it in :class:`Props` like an actor on stage. The
|
||||
props we are giving to this one are just the defaults, we could at this point
|
||||
also configure how and where the Actor should be deployed in a clustered
|
||||
system.
|
||||
|
||||
As Carl Hewitt said, one Actor is no Actor—it would be quite lonely with
|
||||
nobody to talk to. In this sense the example is a little cruel because we only
|
||||
give the ``HelloWorld`` Actor a fake person to talk to—the “ask” pattern
|
||||
(represented by the ``?`` operator) can be used to send a message such that the
|
||||
reply fulfills a Promise to which we get back the corresponding Future.
|
||||
|
||||
Note that the :class:`Future` that is returned by the “ask” operation is
|
||||
properly typed already, no type checks or casts needed. This is possible due to
|
||||
the type information that is part of the message protocol: the ``?`` operator
|
||||
takes as argument a function that accepts an :class:`ActorRef[U]` (which
|
||||
explains the ``_`` hole in the expression on line 6 above) and the ``replyTo``
|
||||
parameter which we fill in like that is of type ``ActorRef[Greeted]``, which
|
||||
means that the value that fulfills the :class:`Promise` can only be of type
|
||||
:class:`Greeted`.
|
||||
|
||||
We use this here to send the :class:`Greet` command to the Actor and when the
|
||||
reply comes back we will print it out and tell the actor system to shut down.
|
||||
Once that is done as well we print the ``"system terminated"`` messages and the
|
||||
program ends. The ``recovery`` combinator on the original :class:`Future` is
|
||||
needed in order to ensure proper system shutdown even in case something went
|
||||
wrong; the ``flatMap`` and ``map`` combinators that the ``for`` expression gets
|
||||
turned into care only about the “happy path” and if the ``future`` failed with
|
||||
a timeout then no ``greeting`` would be extracted and nothing would happen.
|
||||
|
||||
This shows that there are aspects of Actor messaging that can be type-checked
|
||||
by the compiler, but this ability is not unlimited, there are bounds to what we
|
||||
can statically express. Before we go on with a more complex (and realistic)
|
||||
example we make a small detour to highlight some of the theory behind this.
|
||||
|
||||
A Little Bit of Theory
|
||||
======================
|
||||
|
||||
The `Actor Model`_ as defined by Hewitt, Bishop and Steiger in 1973 is a
|
||||
computational model that expresses exactly what it means for computation to be
|
||||
distributed. The processing units—Actors—can only community by exchanging
|
||||
messages and upon reception of a message an Actor can do the following three
|
||||
fundamental actions:
|
||||
|
||||
.. _`Actor Model`: http://en.wikipedia.org/wiki/Actor_model
|
||||
|
||||
1. send a finite number of messages to Actors it knows
|
||||
|
||||
2. create a finite number of new Actors
|
||||
|
||||
3. designate the behavior to be applied to the next message
|
||||
|
||||
The Akka Typed project expresses these actions using behaviors and addresses.
|
||||
Messages can be sent to an address and behind this façade there is a behavior
|
||||
that receives the message and acts upon it. The binding between address and
|
||||
behavior can change over time as per the third point above, but that is not
|
||||
visible on the outside.
|
||||
|
||||
With this preamble we can get to the unique property of this project, namely
|
||||
that it introduces static type checking to Actor interactions: addresses are
|
||||
parameterized and only messages that are of the specified type can be sent to
|
||||
them. The association between an address and its type parameter must be made
|
||||
when the address (and its Actor) is created. For this purpose each behavior is
|
||||
also parameterized with the type of messages it is able to process. Since the
|
||||
behavior can change behind the address façade, designating the next behavior is
|
||||
a constrained operation: the successor must handle the same type of messages as
|
||||
its predecessor. This is necessary in order to not invalidate the addresses
|
||||
that refer to this Actor.
|
||||
|
||||
What this enables is that wherever a message is sent to an Actor we can
|
||||
statically ensure that the type of the message is one that the Actor declares
|
||||
to handle—we can avoid the mistake of sending completely pointless messages.
|
||||
What we cannot statically ensure, though, is that the behavior behind the
|
||||
address will be in a given state when our message is received. The fundamental
|
||||
reason is that the association between address and behavior is a dynamic
|
||||
runtime property, the compiler cannot know it while it translates the source
|
||||
code.
|
||||
|
||||
This is the same as for normal Java objects with internal variables: when
|
||||
compiling the program we cannot know what their value will be, and if the
|
||||
result of a method call depends on those variables then the outcome is
|
||||
uncertain to a degree—we can only be certain that the returned value is of a
|
||||
given type.
|
||||
|
||||
We have seen above that the return type of an Actor command is described by the
|
||||
type of reply-to address that is contained within the message. This allows a
|
||||
conversation to be described in terms of its types: the reply will be of type
|
||||
A, but it might also contain an address of type B, which then allows the other
|
||||
Actor to continue the conversation by sending a message of type B to this new
|
||||
address. While we cannot statically express the “current” state of an Actor, we
|
||||
can express the current state of a protocol between two Actors, since that is
|
||||
just given by the last message type that was received or sent.
|
||||
|
||||
In the next section we demonstrate this on a more realistic example.
|
||||
|
||||
A More Complex Example
|
||||
======================
|
||||
|
||||
Consider an Actor that runs a chat room: client Actors may connect by sending
|
||||
a message that contains their screen name and then they can post messages. The
|
||||
chat room Actor will disseminate all posted messages to all currently connected
|
||||
client Actors. The protocol definition could look like the following:
|
||||
|
||||
.. includecode:: code/docs/akka/typed/IntroSpec.scala#chatroom-protocol
|
||||
|
||||
Initially the client Actors only get access to an ``ActorRef[GetSession]``
|
||||
which allows them to make the first step. Once a client’s session has been
|
||||
established it gets a :class:`SessionGranted` message that contains a handle to
|
||||
unlock the next protocol step, posting messages. The :class:`PostMessage`
|
||||
command will need to be sent to this particular address that represents the
|
||||
session that has been added to the chat room. The other aspect of a session is
|
||||
that the client has revealed its own address so that subsequent
|
||||
:class:`MessagePosted` events can be sent to it.
|
||||
|
||||
This illustrates how Actors can express more than just the equivalent of method
|
||||
calls on Java objects. The declared message types and their contents describe a
|
||||
full protocol that can involve multiple Actors and that can evolve over
|
||||
multiple steps. The implementation of the chat room protocol would be as simple
|
||||
as the following:
|
||||
|
||||
.. includecode:: code/docs/akka/typed/IntroSpec.scala#chatroom-behavior
|
||||
|
||||
The core of this behavior is again static, the chat room itself does not change
|
||||
into something else when sessions are established, but we introduce a variable
|
||||
that tracks the opened sessions. When a new :class:`GetSession` command comes
|
||||
in we add that client to the list and then we need to create the session’s
|
||||
:class:`ActorRef` that will be used to post messages. In this case we want to
|
||||
create a very simple Actor that just repackages the :class:`PostMessage`
|
||||
command into a :class:`PostSessionMessage` command which also includes the
|
||||
screen name. Such a wrapper Actor can be created by using the
|
||||
:meth:`spawnAdapter` method on the :class:`ActorContext`, so that we can then
|
||||
go on to reply to the client with the :class:`SessionGranted` result.
|
||||
|
||||
The behavior that we declare here can handle both subtypes of :class:`Command`.
|
||||
:class:`GetSession` has been explained already and the
|
||||
:class:`PostSessionMessage` commands coming from the wrapper Actors will
|
||||
trigger the dissemination of the contained chat room message to all connected
|
||||
clients. But we do not want to give the ability to send
|
||||
:class:`PostSessionMessage` commands to arbitrary clients, we reserve that
|
||||
right to the wrappers we create—otherwise clients could pose as complete
|
||||
different screen names (imagine the :class:`GetSession` protocol to include
|
||||
authentication information to further secure this). Therefore we narrow the
|
||||
behavior down to only accepting :class:`GetSession` commands before exposing it
|
||||
to the world, hence the type of the ``behavior`` value is
|
||||
:class:`Behavior[GetSession]` instead of :class:`Behavior[Command]`.
|
||||
|
||||
Narrowing the type of a behavior is always a safe operation since it only
|
||||
restricts what clients can do. If we were to widen the type then clients could
|
||||
send other messages that were not foreseen while writing the source code for
|
||||
the behavior.
|
||||
|
||||
If we did not care about securing the correspondence between a session and a
|
||||
screen name then we could change the protocol such that :class:`PostMessage` is
|
||||
removed and all clients just get an :class:`ActorRef[PostSessionMessage]` to
|
||||
send to. In this case no wrapper would be needed and we could just use
|
||||
``ctx.self``. The type-checks work out in that case because
|
||||
:class:`ActorRef[-T]` is contravariant in its type parameter, meaning that we
|
||||
can use a :class:`ActorRef[Command]` wherever an
|
||||
:class:`ActorRef[PostSessionMessage]` is needed—this makes sense because the
|
||||
former simply speaks more languages than the latter. The opposite would be
|
||||
problematic, so passing an :class:`ActorRef[PostSessionMessage]` where
|
||||
:class:`ActorRef[Command]` is required will lead to a type error.
|
||||
|
||||
The final piece of this behavior definition is the :class:`ContextAware`
|
||||
decorator that we use in order to obtain access to the :class:`ActorContext`
|
||||
within the :class:`Static` behavior definition. This decorator invokes the
|
||||
provided function when the first message is received and thereby creates the
|
||||
real behavior that will be used going forward—the decorator is discarded after
|
||||
it has done its job.
|
||||
|
||||
Trying it out
|
||||
-------------
|
||||
|
||||
In order to see this chat room in action we need to write a client Actor that can use it:
|
||||
|
||||
.. includecode:: code/docs/akka/typed/IntroSpec.scala#chatroom-gabbler
|
||||
|
||||
From this behavior we can create an Actor that will accept a chat room session,
|
||||
post a message, wait to see it published, and then terminate. The last step
|
||||
requires the ability to change behavior, we need to transition from the normal
|
||||
running behavior into the terminated state. This is why this Actor uses a
|
||||
different behavior constructor named :class:`Total`. This constructor takes as
|
||||
argument a function from the handled message type, in this case
|
||||
:class:`SessionEvent`, to the next behavior. That next behavior must again be
|
||||
of the same type as we discussed in the theory section above. Here we either
|
||||
stay in the very same behavior or we terminate, and both of these cases are so
|
||||
common that there are special values ``Same`` and ``Stopped`` that can be used.
|
||||
The behavior is named “total” (as opposed to “partial”) because the declared
|
||||
function must handle all values of its input type. Since :class:`SessionEvent`
|
||||
is a sealed trait the Scala compiler will warn us if we forget to handle one of
|
||||
the subtypes; in this case it reminded us that alternatively to
|
||||
:class:`SessionGranted` we may also receive a :class:`SessionDenied` event.
|
||||
|
||||
Now to try things out we must start both a chat room and a gabbler and of
|
||||
course we do this inside an Actor system. Since there can be only one guardian
|
||||
supervisor we could either start the chat room from the gabbler (which we don’t
|
||||
want—it complicates its logic) or the gabbler from the chat room (which is
|
||||
nonsensical) or we start both of them from a third Actor—our only sensible
|
||||
choice:
|
||||
|
||||
.. includecode:: code/docs/akka/typed/IntroSpec.scala#chatroom-main
|
||||
|
||||
In good tradition we call the ``main`` Actor what it is, it directly
|
||||
corresponds to the ``main`` method in a traditional Java application. This
|
||||
Actor will perform its job on its own accord, we do not need to send messages
|
||||
from the outside, so we declare it to be of type ``Unit``. Actors receive not
|
||||
only external messages, they also are notified of certain system events,
|
||||
so-called Signals. In order to get access to those we choose to implement this
|
||||
particular one using the :class:`Full` behavior decorator. The name stems from
|
||||
the fact that within this we have full access to all aspects of the Actor. The
|
||||
provided function will be invoked for signals (wrapped in :class:`Sig`) or user
|
||||
messages (wrapped in :class:`Msg`) and the wrapper also contains a reference to
|
||||
the :class:`ActorContext`.
|
||||
|
||||
This particular main Actor reacts to two signals: when it is started it will
|
||||
first receive the :class:`PreStart` signal, upon which the chat room and the
|
||||
gabbler are created and the session between them is initiated, and when the
|
||||
gabbler is finished we will receive the :class:`Terminated` event due to having
|
||||
called ``ctx.watch`` for it. This allows us to shut down the Actor system: when
|
||||
the main Actor terminates there is nothing more to do.
|
||||
|
||||
Therefore after creating the Actor system with the ``main`` Actor’s
|
||||
:class:`Props` we just await its termination.
|
||||
|
||||
Status of this Project and Relation to Akka Actors
|
||||
==================================================
|
||||
|
||||
Akka Typed is the result of many years of research and previous attempts
|
||||
(including Typed Channels in the 2.2.x series) and it is on its way to
|
||||
stabilization, but maturing such a profound change to the core concept of Akka
|
||||
will take a long time. We expect that this module will stay experimental for
|
||||
multiple major releases of Akka and the plain ``akka.actor.Actor`` will not be
|
||||
deprecated or go away anytime soon.
|
||||
|
||||
Being a research project also entails that the reference documentation is not
|
||||
as detailed as it will be for a final version, please refer to the API
|
||||
documentation for greater depth and finer detail.
|
||||
|
||||
Main Differences
|
||||
----------------
|
||||
|
||||
The most prominent difference is the removal of the ``sender()`` functionality.
|
||||
This turned out to be the Achilles heel of the Typed Channels project, it is
|
||||
the feature that makes its type signatures and macros too complex to be viable.
|
||||
The solution chosen in Akka Typed is to explicitly include the properly typed
|
||||
reply-to address in the message, which both burdens the user with this task but
|
||||
also places this aspect of protocol design where it belongs.
|
||||
|
||||
The other prominent difference is the removal of the :class:`Actor` trait. In
|
||||
order to avoid closing over instable references from different execution
|
||||
contexts (e.g. Future transformations) we turned all remaining methods that
|
||||
were on this trait into messages: the behavior receives the
|
||||
:class:`ActorContext` as an argument during processing and the lifecycle hooks
|
||||
have been converted into Signals.
|
||||
|
||||
A side-effect of this is that behaviors can now be tested in isolation without
|
||||
having to be packaged into an Actor, tests can run fully synchronously without
|
||||
having to worry about timeouts and spurious failures. Another side-effect is
|
||||
that behaviors can nicely be composed and decorated, see the :class:`And`,
|
||||
:class:`Or`, :class:`Widened`, :class:`ContextAware` combinators; nothing about
|
||||
these is special or internal, new combinators can be written as external
|
||||
libraries or tailor-made for each project.
|
||||
|
||||
|
|
@ -298,7 +298,7 @@ private[transport] class ThrottlerManager(wrappedTransport: Transport) extends A
|
|||
if (target.isTerminated) Future successful SetThrottleAck
|
||||
else {
|
||||
val internalTarget = target.asInstanceOf[InternalActorRef]
|
||||
val ref = PromiseActorRef(internalTarget.provider, timeout, target, mode)
|
||||
val ref = PromiseActorRef(internalTarget.provider, timeout, target, mode.getClass.getName)
|
||||
internalTarget.sendSystemMessage(Watch(internalTarget, ref))
|
||||
target.tell(mode, ref)
|
||||
ref.result.future.transform({
|
||||
|
|
|
|||
|
|
@ -86,10 +86,19 @@ abstract class EventFilter(occurrences: Int) {
|
|||
}
|
||||
|
||||
def awaitDone(max: Duration): Boolean = {
|
||||
if (todo != Int.MaxValue && todo > 0) TestKit.awaitCond(todo == 0, max, noThrow = true)
|
||||
if (todo != Int.MaxValue && todo > 0) TestKit.awaitCond(todo <= 0, max, noThrow = true)
|
||||
todo == Int.MaxValue || todo == 0
|
||||
}
|
||||
|
||||
/**
|
||||
* Assert that this filter has matched as often as requested by its
|
||||
* `occurrences` parameter specifies.
|
||||
*/
|
||||
def assertDone(max: Duration): Unit =
|
||||
assert(awaitDone(max),
|
||||
if (todo > 0) s"$todo messages outstanding on $this"
|
||||
else s"received ${-todo} excess messages on $this")
|
||||
|
||||
/**
|
||||
* Apply this filter while executing the given code block. Care is taken to
|
||||
* remove the filter when the block is finished or aborted.
|
||||
|
|
@ -101,9 +110,9 @@ abstract class EventFilter(occurrences: Int) {
|
|||
val result = code
|
||||
if (!awaitDone(leeway))
|
||||
if (todo > 0)
|
||||
throw new AssertionError("Timeout (" + leeway + ") waiting for " + todo + " messages on " + this)
|
||||
throw new AssertionError(s"timeout ($leeway) waiting for $todo messages on $this")
|
||||
else
|
||||
throw new AssertionError("Received " + (-todo) + " messages too many on " + this)
|
||||
throw new AssertionError(s"received ${-todo} excess messages on $this")
|
||||
result
|
||||
} finally system.eventStream publish TestEvent.UnMute(this)
|
||||
}
|
||||
|
|
|
|||
13
akka-typed/build.sbt
Normal file
13
akka-typed/build.sbt
Normal file
|
|
@ -0,0 +1,13 @@
|
|||
import akka.{ AkkaBuild, Formatting, OSGi, Unidoc, Dependencies }
|
||||
import com.typesafe.tools.mima.plugin.MimaKeys
|
||||
|
||||
AkkaBuild.defaultSettings
|
||||
|
||||
AkkaBuild.experimentalSettings
|
||||
|
||||
Formatting.formatSettings
|
||||
|
||||
Unidoc.scaladocSettings
|
||||
|
||||
Unidoc.javadocSettings
|
||||
|
||||
246
akka-typed/src/main/scala/akka/typed/ActorContext.scala
Normal file
246
akka-typed/src/main/scala/akka/typed/ActorContext.scala
Normal file
|
|
@ -0,0 +1,246 @@
|
|||
/**
|
||||
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.typed
|
||||
|
||||
import scala.concurrent.duration.Duration
|
||||
import scala.collection.immutable
|
||||
import scala.collection.immutable.TreeSet
|
||||
import scala.collection.immutable.TreeMap
|
||||
import akka.util.Helpers
|
||||
import akka.{ actor ⇒ untyped }
|
||||
import scala.reflect.ClassTag
|
||||
import scala.reflect.classTag
|
||||
import java.util.concurrent.ConcurrentLinkedQueue
|
||||
import scala.annotation.tailrec
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import scala.concurrent.ExecutionContextExecutor
|
||||
import akka.typed.Behavior.stoppedBehavior
|
||||
|
||||
/**
|
||||
* An Actor is given by the combination of a [[Behavior]] and a context in
|
||||
* which this behavior is executed. As per the Actor Model an Actor can perform
|
||||
* the following actions when processing a message:
|
||||
*
|
||||
* - send a finite number of messages to other Actors it knows
|
||||
* - create a finite number of Actors
|
||||
* - designate the behavior for the next message
|
||||
*
|
||||
* In Akka the first capability is accessed by using the `!` or `tell` method
|
||||
* on an [[ActorRef]], the second is provided by [[ActorContext#spawn]]
|
||||
* and the third is implicit in the signature of [[Behavior]] in that the next
|
||||
* behavior is always returned from the message processing logic.
|
||||
*
|
||||
* An `ActorContext` in addition provides access to the Actor’s own identity (“`self`”),
|
||||
* the [[ActorSystem]] it is part of, methods for querying the list of child Actors it
|
||||
* created, access to [[Terminated DeathWatch]] and timed message scheduling.
|
||||
*/
|
||||
trait ActorContext[T] {
|
||||
|
||||
/**
|
||||
* The identity of this Actor, bound to the lifecycle of this Actor instance.
|
||||
* An Actor with the same name that lives before or after this instance will
|
||||
* have a different [[ActorRef]].
|
||||
*/
|
||||
def self: ActorRef[T]
|
||||
|
||||
/**
|
||||
* The [[Props]] from which this Actor was created.
|
||||
*/
|
||||
def props: Props[T]
|
||||
|
||||
/**
|
||||
* The [[ActorSystem]] to which this Actor belongs.
|
||||
*/
|
||||
def system: ActorSystem[Nothing]
|
||||
|
||||
/**
|
||||
* The list of child Actors created by this Actor during its lifetime that
|
||||
* are still alive, in no particular order.
|
||||
*/
|
||||
def children: Iterable[ActorRef[Nothing]]
|
||||
|
||||
/**
|
||||
* The named child Actor if it is alive.
|
||||
*/
|
||||
def child(name: String): Option[ActorRef[Nothing]]
|
||||
|
||||
/**
|
||||
* Create a child Actor from the given [[Props]] under a randomly chosen name.
|
||||
* It is good practice to name Actors wherever practical.
|
||||
*/
|
||||
def spawnAnonymous[U](props: Props[U]): ActorRef[U]
|
||||
|
||||
/**
|
||||
* Create a child Actor from the given [[Props]] and with the given name.
|
||||
*/
|
||||
def spawn[U](props: Props[U], name: String): ActorRef[U]
|
||||
|
||||
/**
|
||||
* Create an untyped child Actor from the given [[akka.actor.Props]] under a randomly chosen name.
|
||||
* It is good practice to name Actors wherever practical.
|
||||
*/
|
||||
def actorOf(props: untyped.Props): untyped.ActorRef
|
||||
|
||||
/**
|
||||
* Create an untyped child Actor from the given [[akka.actor.Props]] and with the given name.
|
||||
*/
|
||||
def actorOf(props: untyped.Props, name: String): untyped.ActorRef
|
||||
|
||||
/**
|
||||
* Force the child Actor under the given name to terminate after it finishes
|
||||
* processing its current message. Nothing happens if the ActorRef does not
|
||||
* refer to a current child actor.
|
||||
*
|
||||
* @return whether the passed-in [[ActorRef]] points to a current child Actor
|
||||
*/
|
||||
def stop(child: ActorRef[Nothing]): Boolean
|
||||
|
||||
/**
|
||||
* Register for [[Terminated]] notification once the Actor identified by the
|
||||
* given [[ActorRef]] terminates. This notification is also generated when the
|
||||
* [[ActorSystem]] to which the referenced Actor belongs is declared as
|
||||
* failed (e.g. in reaction to being unreachable).
|
||||
*/
|
||||
def watch[U](other: ActorRef[U]): ActorRef[U]
|
||||
|
||||
/**
|
||||
* Register for [[Terminated]] notification once the Actor identified by the
|
||||
* given [[akka.actor.ActorRef]] terminates. This notification is also generated when the
|
||||
* [[ActorSystem]] to which the referenced Actor belongs is declared as
|
||||
* failed (e.g. in reaction to being unreachable).
|
||||
*/
|
||||
def watch(other: akka.actor.ActorRef): akka.actor.ActorRef
|
||||
|
||||
/**
|
||||
* Revoke the registration established by [[#watch[U]* watch]]. A [[Terminated]]
|
||||
* notification will not subsequently be received for the referenced Actor.
|
||||
*/
|
||||
def unwatch[U](other: ActorRef[U]): ActorRef[U]
|
||||
|
||||
/**
|
||||
* Revoke the registration established by [[#watch(* watch]]. A [[Terminated]]
|
||||
* notification will not subsequently be received for the referenced Actor.
|
||||
*/
|
||||
def unwatch(other: akka.actor.ActorRef): akka.actor.ActorRef
|
||||
|
||||
/**
|
||||
* Schedule the sending of a [[ReceiveTimeout]] notification in case no other
|
||||
* message is received during the given period of time. The timeout starts anew
|
||||
* with each received message. Provide `Duration.Undefined` to switch off this
|
||||
* mechanism.
|
||||
*/
|
||||
def setReceiveTimeout(d: Duration): Unit
|
||||
|
||||
/**
|
||||
* Schedule the sending of the given message to the given target Actor after
|
||||
* the given time period has elapsed. The scheduled action can be cancelled
|
||||
* by invoking [[akka.actor.Cancellable!.cancel* cancel]] on the returned
|
||||
* handle.
|
||||
*/
|
||||
def schedule[U](delay: FiniteDuration, target: ActorRef[U], msg: U): untyped.Cancellable
|
||||
|
||||
/**
|
||||
* This Actor’s execution context. It can be used to run asynchronous tasks
|
||||
* like [[scala.concurrent.Future]] combinators.
|
||||
*/
|
||||
implicit def executionContext: ExecutionContextExecutor
|
||||
|
||||
/**
|
||||
* Create a child actor that will wrap messages such that other Actor’s
|
||||
* protocols can be ingested by this Actor. You are strongly advised to cache
|
||||
* these ActorRefs or to stop them when no longer needed.
|
||||
*/
|
||||
def spawnAdapter[U](f: U ⇒ T): ActorRef[U]
|
||||
}
|
||||
|
||||
/**
|
||||
* An [[ActorContext]] for synchronous execution of a [[Behavior]] that
|
||||
* provides only stubs for the effects an Actor can perform and replaces
|
||||
* created child Actors by [[Inbox$.sync* a synchronous Inbox]].
|
||||
*
|
||||
* @see [[EffectfulActorContext]] for more advanced uses.
|
||||
*/
|
||||
class StubbedActorContext[T](
|
||||
val name: String,
|
||||
override val props: Props[T])(
|
||||
override implicit val system: ActorSystem[Nothing]) extends ActorContext[T] {
|
||||
|
||||
val inbox = Inbox.sync[T](name)
|
||||
override val self = inbox.ref
|
||||
|
||||
private var _children = TreeMap.empty[String, Inbox.SyncInbox[_]]
|
||||
private val childName = Iterator from 1 map (Helpers.base64(_))
|
||||
|
||||
override def children: Iterable[ActorRef[Nothing]] = _children.values map (_.ref)
|
||||
override def child(name: String): Option[ActorRef[Nothing]] = _children get name map (_.ref)
|
||||
override def spawnAnonymous[U](props: Props[U]): ActorRef[U] = {
|
||||
val i = Inbox.sync[U](childName.next())
|
||||
_children += i.ref.untypedRef.path.name -> i
|
||||
i.ref
|
||||
}
|
||||
override def spawn[U](props: Props[U], name: String): ActorRef[U] =
|
||||
_children get name match {
|
||||
case Some(_) ⇒ throw new untyped.InvalidActorNameException(s"actor name $name is already taken")
|
||||
case None ⇒
|
||||
val i = Inbox.sync[U](name)
|
||||
_children += name -> i
|
||||
i.ref
|
||||
}
|
||||
override def actorOf(props: untyped.Props): untyped.ActorRef = {
|
||||
val i = Inbox.sync[Any](childName.next())
|
||||
_children += i.ref.untypedRef.path.name -> i
|
||||
i.ref.untypedRef
|
||||
}
|
||||
override def actorOf(props: untyped.Props, name: String): untyped.ActorRef =
|
||||
_children get name match {
|
||||
case Some(_) ⇒ throw new untyped.InvalidActorNameException(s"actor name $name is already taken")
|
||||
case None ⇒
|
||||
val i = Inbox.sync[Any](name)
|
||||
_children += name -> i
|
||||
i.ref.untypedRef
|
||||
}
|
||||
override def stop(child: ActorRef[Nothing]): Boolean = {
|
||||
// removal is asynchronous, so don’t do it here; explicit removeInbox needed from outside
|
||||
_children.get(child.path.name) match {
|
||||
case None ⇒ false
|
||||
case Some(inbox) ⇒ inbox.ref == child
|
||||
}
|
||||
}
|
||||
def watch[U](other: ActorRef[U]): ActorRef[U] = other
|
||||
def watch(other: akka.actor.ActorRef): other.type = other
|
||||
def unwatch[U](other: ActorRef[U]): ActorRef[U] = other
|
||||
def unwatch(other: akka.actor.ActorRef): other.type = other
|
||||
def setReceiveTimeout(d: Duration): Unit = ()
|
||||
|
||||
def schedule[U](delay: FiniteDuration, target: ActorRef[U], msg: U): untyped.Cancellable = new untyped.Cancellable {
|
||||
def cancel() = false
|
||||
def isCancelled = true
|
||||
}
|
||||
implicit def executionContext: ExecutionContextExecutor = system.executionContext
|
||||
def spawnAdapter[U](f: U ⇒ T): ActorRef[U] = ???
|
||||
|
||||
def getInbox[U](name: String): Inbox.SyncInbox[U] = _children(name).asInstanceOf[Inbox.SyncInbox[U]]
|
||||
def removeInbox(name: String): Unit = _children -= name
|
||||
}
|
||||
|
||||
/*
|
||||
* TODO
|
||||
*
|
||||
* Currently running a behavior requires that the context stays the same, since
|
||||
* the behavior may well close over it and thus a change might not be effective
|
||||
* at all. Another issue is that there is genuine state within the context that
|
||||
* is coupled to the behavior’s state: if child actors were created then
|
||||
* migrating a behavior into a new context will not work.
|
||||
*
|
||||
* This note is about remembering the reasons behind this restriction and
|
||||
* proposes an ActorContextProxy as a (broken) half-solution. Another avenue
|
||||
* by which a solution may be explored is for Pure behaviors in that they
|
||||
* may be forced to never remember anything that is immobile.
|
||||
*/
|
||||
//class MobileActorContext[T](_name: String, _props: Props[T], _system: ActorSystem[Nothing])
|
||||
// extends EffectfulActorContext[T](_name, _props, _system) {
|
||||
//
|
||||
//}
|
||||
//
|
||||
//class ActorContextProxy[T](var d: ActorContext[T]) extends ActorContext[T]
|
||||
77
akka-typed/src/main/scala/akka/typed/ActorRef.scala
Normal file
77
akka-typed/src/main/scala/akka/typed/ActorRef.scala
Normal file
|
|
@ -0,0 +1,77 @@
|
|||
/**
|
||||
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.typed
|
||||
|
||||
import akka.actor.ActorPath
|
||||
import scala.annotation.unchecked.uncheckedVariance
|
||||
import language.implicitConversions
|
||||
|
||||
/**
|
||||
* An ActorRef is the identity or address of an Actor instance. It is valid
|
||||
* only during the Actor’s lifetime and allows messages to be sent to that
|
||||
* Actor instance. Sending a message to an Actor that has terminated before
|
||||
* receiving the message will lead to that message being discarded; such
|
||||
* messages are delivered to the [[akka.actor.DeadLetter]] channel of the
|
||||
* [[akka.actor.ActorSystem!.eventStream EventStream]] on a best effort basis
|
||||
* (i.e. this delivery is not reliable).
|
||||
*/
|
||||
abstract class ActorRef[-T] extends java.lang.Comparable[ActorRef[_]] { this: ScalaActorRef[T] ⇒
|
||||
/**
|
||||
* INTERNAL API.
|
||||
*
|
||||
* Implementation detail. The underlying untyped [[akka.actor.ActorRef]]
|
||||
* of this typed ActorRef.
|
||||
*/
|
||||
private[akka] def untypedRef: akka.actor.ActorRef
|
||||
|
||||
/**
|
||||
* Send a message to the Actor referenced by this ActorRef using *at-most-once*
|
||||
* messaging semantics.
|
||||
*/
|
||||
def tell(msg: T): Unit = untypedRef ! msg
|
||||
|
||||
/**
|
||||
* Unsafe utility method for widening the type accepted by this ActorRef;
|
||||
* provided to avoid having to use `asInstanceOf` on the full reference type,
|
||||
* which would unfortunately also work on non-ActorRefs.
|
||||
*/
|
||||
def upcast[U >: T @uncheckedVariance]: ActorRef[U] = this.asInstanceOf[ActorRef[U]]
|
||||
|
||||
/**
|
||||
* The hierarchical path name of the referenced Actor. The lifecycle of the
|
||||
* ActorRef is fully contained within the lifecycle of the [[akka.actor.ActorPath]]
|
||||
* and more than one Actor instance can exist with the same path at different
|
||||
* points in time, but not concurrently.
|
||||
*/
|
||||
def path: ActorPath = untypedRef.path
|
||||
|
||||
override def toString = untypedRef.toString
|
||||
override def equals(other: Any) = other match {
|
||||
case a: ActorRef[_] ⇒ a.untypedRef == untypedRef
|
||||
case _ ⇒ false
|
||||
}
|
||||
override def hashCode = untypedRef.hashCode
|
||||
override def compareTo(other: ActorRef[_]) = untypedRef.compareTo(other.untypedRef)
|
||||
}
|
||||
|
||||
/**
|
||||
* This trait is used to hide the `!` method from Java code.
|
||||
*/
|
||||
trait ScalaActorRef[-T] { this: ActorRef[T] ⇒
|
||||
def !(msg: T): Unit = tell(msg)
|
||||
}
|
||||
|
||||
object ActorRef {
|
||||
private class Combined[T](val untypedRef: akka.actor.ActorRef) extends ActorRef[T] with ScalaActorRef[T]
|
||||
|
||||
implicit def toScalaActorRef[T](ref: ActorRef[T]): ScalaActorRef[T] = ref.asInstanceOf[ScalaActorRef[T]]
|
||||
|
||||
/**
|
||||
* Construct a typed ActorRef from an untyped one and a protocol definition
|
||||
* (i.e. a recipient message type). This can be used to properly represent
|
||||
* untyped Actors within the typed world, given that they implement the assumed
|
||||
* protocol.
|
||||
*/
|
||||
def apply[T](ref: akka.actor.ActorRef): ActorRef[T] = new Combined[T](ref)
|
||||
}
|
||||
144
akka-typed/src/main/scala/akka/typed/ActorSystem.scala
Normal file
144
akka-typed/src/main/scala/akka/typed/ActorSystem.scala
Normal file
|
|
@ -0,0 +1,144 @@
|
|||
/**
|
||||
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.typed
|
||||
|
||||
import akka.event.EventStream
|
||||
import akka.actor.Scheduler
|
||||
import scala.concurrent.ExecutionContext
|
||||
import java.util.concurrent.Executor
|
||||
import scala.concurrent.duration.Duration
|
||||
import akka.actor.Extension
|
||||
import akka.actor.ExtensionId
|
||||
import akka.actor.ActorRefProvider
|
||||
import java.util.concurrent.ThreadFactory
|
||||
import akka.actor.DynamicAccess
|
||||
import akka.actor.ActorSystemImpl
|
||||
import com.typesafe.config.Config
|
||||
import akka.actor.ExtendedActorSystem
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import scala.concurrent.ExecutionContextExecutor
|
||||
import scala.concurrent.Future
|
||||
import akka.dispatch.Dispatchers
|
||||
|
||||
/**
|
||||
* An ActorSystem is home to a hierarchy of Actors. It is created using
|
||||
* [[ActorSystem$.apply[T]*]] from a [[Props]] object that describes the root
|
||||
* Actor of this hierarchy and which will create all other Actors beneath it.
|
||||
* A system also implements the [[ActorRef]] type, and sending a message to
|
||||
* the system directs that message to the root Actor.
|
||||
*/
|
||||
abstract class ActorSystem[-T](_name: String) extends ActorRef[T] { this: ScalaActorRef[T] ⇒
|
||||
|
||||
/**
|
||||
* INTERNAL API.
|
||||
*
|
||||
* Access to the underlying (untyped) ActorSystem.
|
||||
*/
|
||||
private[akka] val untyped: ExtendedActorSystem
|
||||
|
||||
/**
|
||||
* The name of this actor system, used to distinguish multiple ones within
|
||||
* the same JVM & class loader.
|
||||
*/
|
||||
def name: String = _name
|
||||
|
||||
/**
|
||||
* The core settings extracted from the supplied configuration.
|
||||
*/
|
||||
def settings: akka.actor.ActorSystem.Settings = untyped.settings
|
||||
|
||||
/**
|
||||
* Log the configuration.
|
||||
*/
|
||||
def logConfiguration(): Unit = untyped.logConfiguration()
|
||||
|
||||
/**
|
||||
* Start-up time in milliseconds since the epoch.
|
||||
*/
|
||||
def startTime: Long = untyped.startTime
|
||||
|
||||
/**
|
||||
* Up-time of this actor system in seconds.
|
||||
*/
|
||||
def uptime: Long = untyped.uptime
|
||||
|
||||
/**
|
||||
* Helper object for looking up configured dispatchers.
|
||||
*/
|
||||
def dispatchers: Dispatchers = untyped.dispatchers
|
||||
|
||||
/**
|
||||
* A ThreadFactory that can be used if the transport needs to create any Threads
|
||||
*/
|
||||
def threadFactory: ThreadFactory = untyped.threadFactory
|
||||
|
||||
/**
|
||||
* ClassLoader wrapper which is used for reflective accesses internally. This is set
|
||||
* to use the context class loader, if one is set, or the class loader which
|
||||
* loaded the ActorSystem implementation. The context class loader is also
|
||||
* set on all threads created by the ActorSystem, if one was set during
|
||||
* creation.
|
||||
*/
|
||||
def dynamicAccess: DynamicAccess = untyped.dynamicAccess
|
||||
|
||||
/**
|
||||
* The ActorRefProvider is the only entity which creates all actor references within this actor system.
|
||||
*/
|
||||
def provider: ActorRefProvider = untyped.provider
|
||||
|
||||
/**
|
||||
* The user guardian’s untyped [[akka.actor.ActorRef]].
|
||||
*/
|
||||
private[akka] override def untypedRef: akka.actor.ActorRef = untyped.provider.guardian
|
||||
|
||||
/**
|
||||
* Main event bus of this actor system, used for example for logging.
|
||||
*/
|
||||
def eventStream: EventStream = untyped.eventStream
|
||||
|
||||
/**
|
||||
* The default thread pool of this ActorSystem, configured with settings in `akka.actor.default-dispatcher`.
|
||||
*/
|
||||
implicit def executionContext: ExecutionContextExecutor = untyped.dispatcher
|
||||
|
||||
/**
|
||||
* Terminates this actor system. This will stop the guardian actor, which in turn
|
||||
* will recursively stop all its child actors, then the system guardian
|
||||
* (below which the logging actors reside).
|
||||
*/
|
||||
def terminate(): Future[Terminated] = untyped.terminate().map(t ⇒ Terminated(ActorRef(t.actor)))
|
||||
|
||||
/**
|
||||
* Returns a Future which will be completed after the ActorSystem has been terminated
|
||||
* and termination hooks have been executed.
|
||||
*/
|
||||
def whenTerminated: Future[Terminated] = untyped.whenTerminated.map(t ⇒ Terminated(ActorRef(t.actor)))
|
||||
|
||||
/**
|
||||
* The deadLetter address is a destination that will accept (and discard)
|
||||
* every message sent to it.
|
||||
*/
|
||||
def deadLetters[U]: ActorRef[U] = deadLetterRef
|
||||
lazy private val deadLetterRef = ActorRef[Any](untyped.deadLetters)
|
||||
}
|
||||
|
||||
object ActorSystem {
|
||||
private class Impl[T](_name: String, _config: Config, _cl: ClassLoader, _ec: Option[ExecutionContext], _p: Props[T])
|
||||
extends ActorSystem[T](_name) with ScalaActorRef[T] {
|
||||
override private[akka] val untyped: ExtendedActorSystem = new ActorSystemImpl(_name, _config, _cl, _ec, Some(Props.untyped(_p))).start()
|
||||
}
|
||||
|
||||
private class Wrapper(val untyped: ExtendedActorSystem) extends ActorSystem[Nothing](untyped.name) with ScalaActorRef[Nothing]
|
||||
|
||||
def apply[T](name: String, guardianProps: Props[T],
|
||||
config: Option[Config] = None,
|
||||
classLoader: Option[ClassLoader] = None,
|
||||
executionContext: Option[ExecutionContext] = None): ActorSystem[T] = {
|
||||
val cl = classLoader.getOrElse(akka.actor.ActorSystem.findClassLoader())
|
||||
val appConfig = config.getOrElse(ConfigFactory.load(cl))
|
||||
new Impl(name, appConfig, cl, executionContext, guardianProps)
|
||||
}
|
||||
|
||||
def apply(untyped: akka.actor.ActorSystem): ActorSystem[Nothing] = new Wrapper(untyped.asInstanceOf[ExtendedActorSystem])
|
||||
}
|
||||
71
akka-typed/src/main/scala/akka/typed/Ask.scala
Normal file
71
akka-typed/src/main/scala/akka/typed/Ask.scala
Normal file
|
|
@ -0,0 +1,71 @@
|
|||
/**
|
||||
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.typed
|
||||
|
||||
import scala.concurrent.Future
|
||||
import akka.util.Timeout
|
||||
import akka.actor.InternalActorRef
|
||||
import akka.pattern.AskTimeoutException
|
||||
import akka.pattern.PromiseActorRef
|
||||
import java.lang.IllegalArgumentException
|
||||
import scala.util.Success
|
||||
import scala.concurrent.ExecutionContext
|
||||
import scala.util.Failure
|
||||
import akka.actor.Status
|
||||
|
||||
/**
|
||||
* The ask-pattern implements the initiator side of a request–reply protocol.
|
||||
* The party that asks may be within or without an Actor, since the
|
||||
* implementation will fabricate a (hidden) [[ActorRef]] that is bound to a
|
||||
* [[scala.concurrent.Promise]]. This ActorRef will need to be injected in the
|
||||
* message that is sent to the target Actor in order to function as a reply-to
|
||||
* address, therefore the argument to the [[AskPattern$.Askable ask
|
||||
* operator]] is not the message itself but a function that given the reply-to
|
||||
* address will create the message.
|
||||
*
|
||||
* {{{
|
||||
* case class Request(msg: String, replyTo: ActorRef[Reply])
|
||||
* case class Reply(msg: String)
|
||||
*
|
||||
* implicit val timeout = Timeout(3.seconds)
|
||||
* val target: ActorRef[Request] = ...
|
||||
* val f: Future[Reply] = target ? (Request("hello", _))
|
||||
* }}}
|
||||
*/
|
||||
object AskPattern {
|
||||
implicit class Askable[T](val ref: ActorRef[T]) extends AnyVal {
|
||||
def ?[U](f: ActorRef[U] ⇒ T)(implicit timeout: Timeout): Future[U] = ask(ref, timeout, f)
|
||||
}
|
||||
|
||||
private class PromiseRef[U](actorRef: ActorRef[_], timeout: Timeout) {
|
||||
val (ref: ActorRef[U], future: Future[U], promiseRef: PromiseActorRef) = actorRef.untypedRef match {
|
||||
case ref: InternalActorRef if ref.isTerminated ⇒
|
||||
(ActorRef[U](ref.provider.deadLetters),
|
||||
Future.failed[U](new AskTimeoutException(s"Recipient[$actorRef] had already been terminated.")))
|
||||
case ref: InternalActorRef ⇒
|
||||
if (timeout.duration.length <= 0)
|
||||
(ActorRef[U](ref.provider.deadLetters),
|
||||
Future.failed[U](new IllegalArgumentException(s"Timeout length must not be negative, question not sent to [$actorRef]")))
|
||||
else {
|
||||
val a = PromiseActorRef(ref.provider, timeout, actorRef, "unknown")
|
||||
val b = ActorRef[U](a)
|
||||
(b, a.result.future.asInstanceOf[Future[U]], a)
|
||||
}
|
||||
case _ ⇒ throw new IllegalArgumentException(s"cannot create PromiseRef for non-Akka ActorRef (${actorRef.getClass})")
|
||||
}
|
||||
}
|
||||
|
||||
private object PromiseRef {
|
||||
def apply[U](actorRef: ActorRef[_])(implicit timeout: Timeout) = new PromiseRef[U](actorRef, timeout)
|
||||
}
|
||||
|
||||
private[typed] def ask[T, U](actorRef: ActorRef[T], timeout: Timeout, f: ActorRef[U] ⇒ T): Future[U] = {
|
||||
val p = PromiseRef[U](actorRef)(timeout)
|
||||
val m = f(p.ref)
|
||||
p.promiseRef.messageClassName = m.getClass.getName
|
||||
actorRef ! m
|
||||
p.future
|
||||
}
|
||||
|
||||
}
|
||||
285
akka-typed/src/main/scala/akka/typed/Behavior.scala
Normal file
285
akka-typed/src/main/scala/akka/typed/Behavior.scala
Normal file
|
|
@ -0,0 +1,285 @@
|
|||
/**
|
||||
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.typed
|
||||
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import scala.reflect.ClassTag
|
||||
import akka.actor.OneForOneStrategy
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.immutable
|
||||
import akka.util.LineNumbers
|
||||
|
||||
/**
|
||||
* The behavior of an actor defines how it reacts to the messages that it
|
||||
* receives. The message may either be of the type that the Actor declares
|
||||
* and which is part of the [[ActorRef]] signature, or it may be a system
|
||||
* [[Signal]] that expresses a lifecycle event of either this actor or one of
|
||||
* its child actors.
|
||||
*
|
||||
* Behaviors can be formulated in a number of different ways, either by
|
||||
* creating a derived class or by employing factory methods like the ones
|
||||
* in the [[ScalaDSL$]] object.
|
||||
*/
|
||||
abstract class Behavior[T] {
|
||||
/**
|
||||
* Process an incoming [[Signal]] and return the next behavior. This means
|
||||
* that all lifecycle hooks, ReceiveTimeout, Terminated and Failed messages
|
||||
* can initiate a behavior change.
|
||||
*
|
||||
* The returned behavior can in addition to normal behaviors be one of the
|
||||
* canned special objects:
|
||||
*
|
||||
* * returning `Stopped` will terminate this Behavior
|
||||
* * returning `Same` designates to reuse the current Behavior
|
||||
* * returning `Unhandled` keeps the same Behavior and signals that the message was not yet handled
|
||||
*
|
||||
* Code calling this method should use [[Behavior$.canonicalize]] to replace
|
||||
* the special objects with real Behaviors.
|
||||
*/
|
||||
def management(ctx: ActorContext[T], msg: Signal): Behavior[T]
|
||||
|
||||
/**
|
||||
* Process an incoming message and return the next behavior.
|
||||
*
|
||||
* The returned behavior can in addition to normal behaviors be one of the
|
||||
* canned special objects:
|
||||
*
|
||||
* * returning `Stopped` will terminate this Behavior
|
||||
* * returning `Same` designates to reuse the current Behavior
|
||||
* * returning `Unhandled` keeps the same Behavior and signals that the message was not yet handled
|
||||
*
|
||||
* Code calling this method should use [[Behavior$.canonicalize]] to replace
|
||||
* the special objects with real Behaviors.
|
||||
*/
|
||||
def message(ctx: ActorContext[T], msg: T): Behavior[T]
|
||||
|
||||
/**
|
||||
* Narrow the type of this Behavior, which is always a safe operation. This
|
||||
* method is necessary to implement the contravariant nature of Behavior
|
||||
* (which cannot be expressed directly due to type inference problems).
|
||||
*/
|
||||
def narrow[U <: T]: Behavior[U] = this.asInstanceOf[Behavior[U]]
|
||||
}
|
||||
|
||||
/*
|
||||
* FIXME
|
||||
*
|
||||
* Closing over ActorContext makes a Behavior immobile: it cannot be moved to
|
||||
* another context and executed there, and therefore it cannot be replicated or
|
||||
* forked either.
|
||||
*/
|
||||
|
||||
/**
|
||||
* System signals are notifications that are generated by the system and
|
||||
* delivered to the Actor behavior in a reliable fashion (i.e. they are
|
||||
* guaranteed to arrive in contrast to the at-most-once semantics of normal
|
||||
* Actor messages).
|
||||
*/
|
||||
sealed trait Signal
|
||||
/**
|
||||
* Lifecycle signal that is fired upon creation of the Actor. This will be the
|
||||
* first message that the actor processes.
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final case object PreStart extends Signal
|
||||
/**
|
||||
* Lifecycle signal that is fired upon restart of the Actor before replacing
|
||||
* the behavior with the fresh one (i.e. this signal is received within the
|
||||
* behavior that failed).
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final case class PreRestart(failure: Throwable) extends Signal
|
||||
/**
|
||||
* Lifecycle signal that is fired upon restart of the Actor after replacing
|
||||
* the behavior with the fresh one (i.e. this signal is received within the
|
||||
* fresh replacement behavior).
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final case class PostRestart(failure: Throwable) extends Signal
|
||||
/**
|
||||
* Lifecycle signal that is fired after this actor and all its child actors
|
||||
* (transitively) have terminated. The [[Terminated]] signal is only sent to
|
||||
* registered watchers after this signal has been processed.
|
||||
*
|
||||
* <b>IMPORTANT NOTE:</b> if the actor terminated by switching to the
|
||||
* [[ScalaDSL$.Stopped]] behavior then this signal will be ignored (i.e. the
|
||||
* Stopped behvavior will do nothing in reaction to it).
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final case object PostStop extends Signal
|
||||
/**
|
||||
* Lifecycle signal that is fired when a direct child actor fails. The child
|
||||
* actor will be suspended until its fate has been decided. The decision is
|
||||
* communicated by calling the [[Failed#decide]] method. If this is not
|
||||
* done then the default behavior is to escalate the failure, which amounts to
|
||||
* failing this actor with the same exception that the child actor failed with.
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final case class Failed(cause: Throwable, child: ActorRef[Nothing]) extends Signal {
|
||||
import Failed._
|
||||
|
||||
private[this] var _decision: Decision = _
|
||||
def decide(decision: Decision): Unit = _decision = decision
|
||||
def getDecision: Decision = _decision match {
|
||||
case null ⇒ NoFailureResponse
|
||||
case x ⇒ x
|
||||
}
|
||||
}
|
||||
/**
|
||||
* The actor can register for a notification in case no message is received
|
||||
* within a given time window, and the signal that is raised in this case is
|
||||
* this one. See also [[ActorContext#setReceiveTimeout]].
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final case object ReceiveTimeout extends Signal
|
||||
/**
|
||||
* Lifecycle signal that is fired when an Actor that was watched has terminated.
|
||||
* Watching is performed by invoking the
|
||||
* [[akka.typed.ActorContext!.watch[U]* watch]] method. The DeathWatch service is
|
||||
* idempotent, meaning that registering twice has the same effect as registering
|
||||
* once. Registration does not need to happen before the Actor terminates, a
|
||||
* notification is guaranteed to arrive after both registration and termination
|
||||
* have occurred. Termination of a remote Actor can also be effected by declaring
|
||||
* the Actor’s home system as failed (e.g. as a result of being unreachable).
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final case class Terminated(ref: ActorRef[Nothing]) extends Signal
|
||||
|
||||
/**
|
||||
* The parent of an actor decides upon the fate of a failed child actor by
|
||||
* encapsulating its next behavior in one of the four wrappers defined within
|
||||
* this class.
|
||||
*
|
||||
* Failure responses have an associated precedence that ranks them, which is in
|
||||
* descending importance:
|
||||
*
|
||||
* - Escalate
|
||||
* - Stop
|
||||
* - Restart
|
||||
* - Resume
|
||||
*/
|
||||
object Failed {
|
||||
|
||||
/**
|
||||
* Failure responses are in some cases communicated by using the companion
|
||||
* objects of the wrapper behaviors, see the [[StepWise]] behavior for an
|
||||
* example.
|
||||
*/
|
||||
sealed trait Decision
|
||||
|
||||
@SerialVersionUID(1L)
|
||||
case object NoFailureResponse extends Decision
|
||||
|
||||
/**
|
||||
* Resuming the child actor means that the result of processing the message
|
||||
* on which it failed is just ignored, the previous state will be used to
|
||||
* process the next message. The message that triggered the failure will not
|
||||
* be processed again.
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
case object Resume extends Decision
|
||||
|
||||
/**
|
||||
* Restarting the child actor means resetting its behavior to the initial
|
||||
* one that was provided during its creation (i.e. the one which was passed
|
||||
* into the [[Props]] constructor). The previously failed behavior will
|
||||
* receive a [[PreRestart]] signal before this happens and the replacement
|
||||
* behavior will receive a [[PostRestart]] signal afterwards.
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
case object Restart extends Decision
|
||||
|
||||
/**
|
||||
* Stopping the child actor will free its resources and eventually
|
||||
* (asynchronously) unregister its name from the parent. Completion of this
|
||||
* process can be observed by watching the child actor and reacting to its
|
||||
* [[Terminated]] signal.
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
case object Stop extends Decision
|
||||
|
||||
/**
|
||||
* The default response to a failure in a child actor is to escalate the
|
||||
* failure, entailing that the parent actor fails as well. This is equivalent
|
||||
* to an exception unwinding the call stack, but it applies to the supervision
|
||||
* hierarchy instead.
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
case object Escalate extends Decision
|
||||
|
||||
}
|
||||
|
||||
object Behavior {
|
||||
|
||||
/**
|
||||
* INTERNAL API.
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
private[akka] object emptyBehavior extends Behavior[Any] {
|
||||
override def management(ctx: ActorContext[Any], msg: Signal): Behavior[Any] = ScalaDSL.Unhandled
|
||||
override def message(ctx: ActorContext[Any], msg: Any): Behavior[Any] = ScalaDSL.Unhandled
|
||||
override def toString = "Empty"
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API.
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
private[akka] object ignoreBehavior extends Behavior[Any] {
|
||||
override def management(ctx: ActorContext[Any], msg: Signal): Behavior[Any] = ScalaDSL.Same
|
||||
override def message(ctx: ActorContext[Any], msg: Any): Behavior[Any] = ScalaDSL.Same
|
||||
override def toString = "Ignore"
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API.
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
private[akka] object unhandledBehavior extends Behavior[Nothing] {
|
||||
override def management(ctx: ActorContext[Nothing], msg: Signal): Behavior[Nothing] = throw new UnsupportedOperationException("Not Implemented")
|
||||
override def message(ctx: ActorContext[Nothing], msg: Nothing): Behavior[Nothing] = throw new UnsupportedOperationException("Not Implemented")
|
||||
override def toString = "Unhandled"
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API.
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
private[akka] object sameBehavior extends Behavior[Nothing] {
|
||||
override def management(ctx: ActorContext[Nothing], msg: Signal): Behavior[Nothing] = throw new UnsupportedOperationException("Not Implemented")
|
||||
override def message(ctx: ActorContext[Nothing], msg: Nothing): Behavior[Nothing] = throw new UnsupportedOperationException("Not Implemented")
|
||||
override def toString = "Same"
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API.
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
private[akka] object stoppedBehavior extends Behavior[Nothing] {
|
||||
override def management(ctx: ActorContext[Nothing], msg: Signal): Behavior[Nothing] = {
|
||||
assert(msg == PostStop, s"stoppedBehavior received $msg (only PostStop is expected)")
|
||||
this
|
||||
}
|
||||
override def message(ctx: ActorContext[Nothing], msg: Nothing): Behavior[Nothing] = throw new UnsupportedOperationException("Not Implemented")
|
||||
override def toString = "Stopped"
|
||||
}
|
||||
|
||||
/**
|
||||
* Given a possibly special behavior (same or unhandled) and a
|
||||
* “current” behavior (which defines the meaning of encountering a [[ScalaDSL$.Same]]
|
||||
* behavior) this method unwraps the behavior such that the innermost behavior
|
||||
* is returned, i.e. it removes the decorations.
|
||||
*/
|
||||
def canonicalize[T](ctx: ActorContext[T], behavior: Behavior[T], current: Behavior[T]): Behavior[T] =
|
||||
behavior match {
|
||||
case `sameBehavior` ⇒ current
|
||||
case `unhandledBehavior` ⇒ current
|
||||
case other ⇒ other
|
||||
}
|
||||
|
||||
def isAlive[T](behavior: Behavior[T]): Boolean = behavior ne stoppedBehavior
|
||||
|
||||
def isUnhandled[T](behavior: Behavior[T]): Boolean = behavior eq unhandledBehavior
|
||||
}
|
||||
|
||||
106
akka-typed/src/main/scala/akka/typed/Effects.scala
Normal file
106
akka-typed/src/main/scala/akka/typed/Effects.scala
Normal file
|
|
@ -0,0 +1,106 @@
|
|||
/**
|
||||
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.typed
|
||||
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import scala.concurrent.duration.Duration
|
||||
import java.util.concurrent.ConcurrentLinkedQueue
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.immutable
|
||||
|
||||
/**
|
||||
* All tracked effects must extend implement this type. It is deliberately
|
||||
* not sealed in order to allow extensions.
|
||||
*/
|
||||
abstract class Effect
|
||||
|
||||
object Effect {
|
||||
@SerialVersionUID(1L) final case class Spawned(childName: String) extends Effect
|
||||
@SerialVersionUID(1L) final case class Stopped(childName: String) extends Effect
|
||||
@SerialVersionUID(1L) final case class Watched[T](other: ActorRef[T]) extends Effect
|
||||
@SerialVersionUID(1L) final case class Unwatched[T](other: ActorRef[T]) extends Effect
|
||||
@SerialVersionUID(1L) final case class ReceiveTimeoutSet(d: Duration) extends Effect
|
||||
@SerialVersionUID(1L) final case class Messaged[U](other: ActorRef[U], msg: U) extends Effect
|
||||
@SerialVersionUID(1L) final case class Scheduled[U](delay: FiniteDuration, target: ActorRef[U], msg: U) extends Effect
|
||||
@SerialVersionUID(1L) case object EmptyEffect extends Effect
|
||||
}
|
||||
|
||||
/**
|
||||
* An [[ActorContext]] for testing purposes that records the effects performed
|
||||
* on it and otherwise stubs them out like a [[StubbedActorContext]].
|
||||
*/
|
||||
class EffectfulActorContext[T](_name: String, _props: Props[T], _system: ActorSystem[Nothing])
|
||||
extends StubbedActorContext[T](_name, _props)(_system) {
|
||||
import akka.{ actor ⇒ a }
|
||||
import Effect._
|
||||
|
||||
private val effectQueue = new ConcurrentLinkedQueue[Effect]
|
||||
def getEffect(): Effect = effectQueue.poll() match {
|
||||
case null ⇒ throw new NoSuchElementException(s"polling on an empty effect queue: $name")
|
||||
case x ⇒ x
|
||||
}
|
||||
def getAllEffects(): immutable.Seq[Effect] = {
|
||||
@tailrec def rec(acc: List[Effect]): List[Effect] = effectQueue.poll() match {
|
||||
case null ⇒ acc.reverse
|
||||
case x ⇒ rec(x :: acc)
|
||||
}
|
||||
rec(Nil)
|
||||
}
|
||||
def hasEffects: Boolean = effectQueue.peek() != null
|
||||
|
||||
private var current = props.creator()
|
||||
signal(PreStart)
|
||||
|
||||
def currentBehavior: Behavior[T] = current
|
||||
|
||||
def run(msg: T): Unit = current = Behavior.canonicalize(this, current.message(this, msg), current)
|
||||
def signal(signal: Signal): Unit = current = Behavior.canonicalize(this, current.management(this, signal), current)
|
||||
|
||||
override def spawnAnonymous[U](props: Props[U]): ActorRef[U] = {
|
||||
val ref = super.spawnAnonymous(props)
|
||||
effectQueue.offer(Spawned(ref.untypedRef.path.name))
|
||||
ref
|
||||
}
|
||||
override def spawn[U](props: Props[U], name: String): ActorRef[U] = {
|
||||
effectQueue.offer(Spawned(name))
|
||||
super.spawn(props, name)
|
||||
}
|
||||
override def actorOf(props: a.Props): a.ActorRef = {
|
||||
val ref = super.actorOf(props)
|
||||
effectQueue.offer(Spawned(ref.path.name))
|
||||
ref
|
||||
}
|
||||
override def actorOf(props: a.Props, name: String): a.ActorRef = {
|
||||
effectQueue.offer(Spawned(name))
|
||||
super.actorOf(props, name)
|
||||
}
|
||||
override def stop(child: ActorRef[Nothing]): Boolean = {
|
||||
effectQueue.offer(Stopped(child.path.name))
|
||||
super.stop(child)
|
||||
}
|
||||
override def watch[U](other: ActorRef[U]): ActorRef[U] = {
|
||||
effectQueue.offer(Watched(other))
|
||||
super.watch(other)
|
||||
}
|
||||
override def unwatch[U](other: ActorRef[U]): ActorRef[U] = {
|
||||
effectQueue.offer(Unwatched(other))
|
||||
super.unwatch(other)
|
||||
}
|
||||
override def watch(other: akka.actor.ActorRef): other.type = {
|
||||
effectQueue.offer(Watched(ActorRef[Any](other)))
|
||||
super.watch(other)
|
||||
}
|
||||
override def unwatch(other: akka.actor.ActorRef): other.type = {
|
||||
effectQueue.offer(Unwatched(ActorRef[Any](other)))
|
||||
super.unwatch(other)
|
||||
}
|
||||
override def setReceiveTimeout(d: Duration): Unit = {
|
||||
effectQueue.offer(ReceiveTimeoutSet(d))
|
||||
super.setReceiveTimeout(d)
|
||||
}
|
||||
override def schedule[U](delay: FiniteDuration, target: ActorRef[U], msg: U): a.Cancellable = {
|
||||
effectQueue.offer(Scheduled(delay, target, msg))
|
||||
super.schedule(delay, target, msg)
|
||||
}
|
||||
}
|
||||
99
akka-typed/src/main/scala/akka/typed/Impl.scala
Normal file
99
akka-typed/src/main/scala/akka/typed/Impl.scala
Normal file
|
|
@ -0,0 +1,99 @@
|
|||
/**
|
||||
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.typed
|
||||
|
||||
import akka.{ actor ⇒ a }
|
||||
import scala.concurrent.duration.Duration
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import scala.concurrent.ExecutionContextExecutor
|
||||
import akka.event.LoggingReceive
|
||||
|
||||
/**
|
||||
* INTERNAL API. Mapping the execution of a [[Behavior]] onto a good old untyped
|
||||
* [[akka.actor.Actor]].
|
||||
*/
|
||||
private[typed] class ActorAdapter[T](_initialBehavior: () ⇒ Behavior[T]) extends akka.actor.Actor {
|
||||
import Behavior._
|
||||
|
||||
var behavior = _initialBehavior()
|
||||
val ctx = new ActorContextAdapter[T](context)
|
||||
|
||||
def receive = LoggingReceive {
|
||||
case akka.actor.Terminated(ref) ⇒ next(behavior.management(ctx, Terminated(ActorRef(ref))))
|
||||
case akka.actor.ReceiveTimeout ⇒ next(behavior.management(ctx, ReceiveTimeout))
|
||||
case msg ⇒ next(behavior.message(ctx, msg.asInstanceOf[T]))
|
||||
}
|
||||
|
||||
private def next(b: Behavior[T]): Unit = {
|
||||
behavior = canonicalize(ctx, b, behavior)
|
||||
if (!isAlive(behavior)) {
|
||||
context.stop(self)
|
||||
}
|
||||
}
|
||||
|
||||
override val supervisorStrategy = a.OneForOneStrategy() {
|
||||
case ex ⇒
|
||||
import Failed._
|
||||
import akka.actor.{ SupervisorStrategy ⇒ s }
|
||||
val f = Failed(ex, ActorRef(sender()))
|
||||
next(behavior.management(ctx, f))
|
||||
f.getDecision match {
|
||||
case Resume ⇒ s.Resume
|
||||
case Restart ⇒ s.Restart
|
||||
case Stop ⇒ s.Stop
|
||||
case _ ⇒ s.Escalate
|
||||
}
|
||||
}
|
||||
|
||||
override def preStart(): Unit =
|
||||
next(behavior.management(ctx, PreStart))
|
||||
override def preRestart(reason: Throwable, message: Option[Any]): Unit =
|
||||
next(behavior.management(ctx, PreRestart(reason)))
|
||||
override def postRestart(reason: Throwable): Unit =
|
||||
next(behavior.management(ctx, PostRestart(reason)))
|
||||
override def postStop(): Unit =
|
||||
next(behavior.management(ctx, PostStop))
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API. Wrapping an [[akka.actor.ActorContext]] as an [[ActorContext]].
|
||||
*/
|
||||
private[typed] class ActorContextAdapter[T](ctx: akka.actor.ActorContext) extends ActorContext[T] {
|
||||
import Ops._
|
||||
def self = ActorRef(ctx.self)
|
||||
def props = Props(ctx.props)
|
||||
val system = ActorSystem(ctx.system)
|
||||
def children = ctx.children.map(ActorRef(_))
|
||||
def child(name: String) = ctx.child(name).map(ActorRef(_))
|
||||
def spawnAnonymous[U](props: Props[U]) = ctx.spawn(props)
|
||||
def spawn[U](props: Props[U], name: String) = ctx.spawn(props, name)
|
||||
def actorOf(props: a.Props) = ctx.actorOf(props)
|
||||
def actorOf(props: a.Props, name: String) = ctx.actorOf(props, name)
|
||||
def stop(child: ActorRef[Nothing]) = ctx.child(child.path.name) match {
|
||||
case Some(ref) if ref == child.untypedRef ⇒
|
||||
ctx.stop(child.untypedRef)
|
||||
true
|
||||
case _ ⇒ false // none of our business
|
||||
}
|
||||
def watch[U](other: ActorRef[U]) = { ctx.watch(other.untypedRef); other }
|
||||
def watch(other: a.ActorRef) = { ctx.watch(other); other }
|
||||
def unwatch[U](other: ActorRef[U]) = { ctx.unwatch(other.untypedRef); other }
|
||||
def unwatch(other: a.ActorRef) = { ctx.unwatch(other); other }
|
||||
def setReceiveTimeout(d: Duration) = ctx.setReceiveTimeout(d)
|
||||
def executionContext: ExecutionContextExecutor = ctx.dispatcher
|
||||
def schedule[U](delay: FiniteDuration, target: ActorRef[U], msg: U): a.Cancellable = {
|
||||
import ctx.dispatcher
|
||||
ctx.system.scheduler.scheduleOnce(delay, target.untypedRef, msg)
|
||||
}
|
||||
def spawnAdapter[U](f: U ⇒ T) = ActorRef[U](ctx.actorOf(akka.actor.Props(classOf[MessageWrapper], f)))
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API. A small Actor that translates between message protocols.
|
||||
*/
|
||||
private[typed] class MessageWrapper(f: Any ⇒ Any) extends akka.actor.Actor {
|
||||
def receive = {
|
||||
case msg ⇒ context.parent ! f(msg)
|
||||
}
|
||||
}
|
||||
54
akka-typed/src/main/scala/akka/typed/Inbox.scala
Normal file
54
akka-typed/src/main/scala/akka/typed/Inbox.scala
Normal file
|
|
@ -0,0 +1,54 @@
|
|||
/**
|
||||
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.typed
|
||||
|
||||
import java.util.concurrent.TimeoutException
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import scala.collection.immutable.TreeSet
|
||||
import scala.collection.mutable.Queue
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration.{ Deadline, Duration, DurationInt, FiniteDuration }
|
||||
import akka.{ actor ⇒ a }
|
||||
import akka.pattern.ask
|
||||
import akka.util.Helpers.ConfigOps
|
||||
import akka.util.Timeout
|
||||
import scala.concurrent.Future
|
||||
import akka.actor.MinimalActorRef
|
||||
import java.util.concurrent.ConcurrentLinkedQueue
|
||||
import akka.actor.ActorPath
|
||||
import akka.actor.RootActorPath
|
||||
import akka.actor.Address
|
||||
import scala.reflect.ClassTag
|
||||
import scala.collection.immutable
|
||||
import scala.annotation.tailrec
|
||||
import akka.actor.ActorRefProvider
|
||||
import scala.concurrent.ExecutionContext
|
||||
|
||||
object Inbox {
|
||||
|
||||
def sync[T](name: String): SyncInbox[T] = new SyncInbox(name)
|
||||
|
||||
class SyncInbox[T](name: String) {
|
||||
private val q = new ConcurrentLinkedQueue[T]
|
||||
private val r = new akka.actor.MinimalActorRef {
|
||||
override def provider: ActorRefProvider = ???
|
||||
override val path: ActorPath = RootActorPath(Address("akka", "SyncInbox")) / name
|
||||
override def !(msg: Any)(implicit sender: akka.actor.ActorRef) = q.offer(msg.asInstanceOf[T])
|
||||
}
|
||||
|
||||
val ref: ActorRef[T] = ActorRef(r)
|
||||
def receiveMsg(): T = q.poll() match {
|
||||
case null ⇒ throw new NoSuchElementException(s"polling on an empty inbox: $name")
|
||||
case x ⇒ x
|
||||
}
|
||||
def receiveAll(): immutable.Seq[T] = {
|
||||
@tailrec def rec(acc: List[T]): List[T] = q.poll() match {
|
||||
case null ⇒ acc.reverse
|
||||
case x ⇒ rec(x :: acc)
|
||||
}
|
||||
rec(Nil)
|
||||
}
|
||||
def hasMessages: Boolean = q.peek() != null
|
||||
}
|
||||
}
|
||||
31
akka-typed/src/main/scala/akka/typed/Ops.scala
Normal file
31
akka-typed/src/main/scala/akka/typed/Ops.scala
Normal file
|
|
@ -0,0 +1,31 @@
|
|||
/**
|
||||
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.typed
|
||||
|
||||
import language.implicitConversions
|
||||
|
||||
/**
|
||||
* Import the contents of this object to retrofit the typed APIs onto the
|
||||
* untyped [[akka.actor.ActorSystem]], [[akka.actor.ActorContext]] and
|
||||
* [[akka.actor.ActorRef]].
|
||||
*/
|
||||
object Ops {
|
||||
|
||||
implicit class ActorSystemOps(val sys: akka.actor.ActorSystem) extends AnyVal {
|
||||
def spawn[T](props: Props[T]): ActorRef[T] =
|
||||
ActorRef(sys.actorOf(Props.untyped(props)))
|
||||
def spawn[T](props: Props[T], name: String): ActorRef[T] =
|
||||
ActorRef(sys.actorOf(Props.untyped(props), name))
|
||||
}
|
||||
|
||||
implicit class ActorContextOps(val ctx: akka.actor.ActorContext) extends AnyVal {
|
||||
def spawn[T](props: Props[T]): ActorRef[T] =
|
||||
ActorRef(ctx.actorOf(Props.untyped(props)))
|
||||
def spawn[T](props: Props[T], name: String): ActorRef[T] =
|
||||
ActorRef(ctx.actorOf(Props.untyped(props), name))
|
||||
}
|
||||
|
||||
implicit def actorRefAdapter(ref: akka.actor.ActorRef): ActorRef[Any] = ActorRef(ref)
|
||||
|
||||
}
|
||||
55
akka-typed/src/main/scala/akka/typed/Props.scala
Normal file
55
akka-typed/src/main/scala/akka/typed/Props.scala
Normal file
|
|
@ -0,0 +1,55 @@
|
|||
/**
|
||||
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.typed
|
||||
|
||||
import akka.actor.Deploy
|
||||
import akka.routing.RouterConfig
|
||||
|
||||
/**
|
||||
* Props describe how to dress up a [[Behavior]] so that it can become an Actor.
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final case class Props[T](creator: () ⇒ Behavior[T], deploy: Deploy) {
|
||||
def withDispatcher(d: String) = copy(deploy = deploy.copy(dispatcher = d))
|
||||
def withMailbox(m: String) = copy(deploy = deploy.copy(mailbox = m))
|
||||
def withRouter(r: RouterConfig) = copy(deploy = deploy.copy(routerConfig = r))
|
||||
def withDeploy(d: Deploy) = copy(deploy = d)
|
||||
}
|
||||
|
||||
/**
|
||||
* Props describe how to dress up a [[Behavior]] so that it can become an Actor.
|
||||
*/
|
||||
object Props {
|
||||
/**
|
||||
* Create a Props instance from a block of code that creates a [[Behavior]].
|
||||
*
|
||||
* FIXME: investigate the pros and cons of making this take an explicit
|
||||
* function instead of a by-name argument
|
||||
*/
|
||||
def apply[T](block: ⇒ Behavior[T]): Props[T] = Props(() ⇒ block, akka.actor.Props.defaultDeploy)
|
||||
|
||||
/**
|
||||
* Props for a Behavior that just ignores all messages.
|
||||
*/
|
||||
def empty[T]: Props[T] = _empty.asInstanceOf[Props[T]]
|
||||
private val _empty: Props[Any] = Props(ScalaDSL.Static[Any] { case _ ⇒ ScalaDSL.Unhandled })
|
||||
|
||||
/**
|
||||
* INTERNAL API.
|
||||
*/
|
||||
private[typed] def untyped[T](p: Props[T]): akka.actor.Props =
|
||||
new akka.actor.Props(p.deploy, classOf[ActorAdapter[_]], p.creator :: Nil)
|
||||
|
||||
/**
|
||||
* INTERNAL API.
|
||||
*/
|
||||
private[typed] def apply[T](p: akka.actor.Props): Props[T] = {
|
||||
assert(p.clazz == classOf[ActorAdapter[_]], "typed.Actor must have typed.Props")
|
||||
p.args match {
|
||||
case (creator: Function0[_]) :: Nil ⇒
|
||||
Props(creator.asInstanceOf[Function0[Behavior[T]]], p.deploy)
|
||||
case _ ⇒ throw new AssertionError("typed.Actor args must be right")
|
||||
}
|
||||
}
|
||||
}
|
||||
445
akka-typed/src/main/scala/akka/typed/ScalaDSL.scala
Normal file
445
akka-typed/src/main/scala/akka/typed/ScalaDSL.scala
Normal file
|
|
@ -0,0 +1,445 @@
|
|||
/**
|
||||
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.typed
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import Behavior._
|
||||
import akka.util.LineNumbers
|
||||
|
||||
/**
|
||||
* This object holds several behavior factories and combinators that can be
|
||||
* used to construct Behavior instances.
|
||||
*/
|
||||
object ScalaDSL {
|
||||
|
||||
// FIXME check that all behaviors can cope with not getting PreStart as first message
|
||||
|
||||
implicit class BehaviorDecorators[T](val behavior: Behavior[T]) extends AnyVal {
|
||||
/**
|
||||
* Widen the type of this Behavior by providing a filter function that permits
|
||||
* only a subtype of the widened set of messages.
|
||||
*/
|
||||
def widen[U >: T](matcher: PartialFunction[U, T]): Behavior[U] = Widened(behavior, matcher)
|
||||
/**
|
||||
* Combine the two behaviors such that incoming messages are distributed
|
||||
* to both of them, each one evolving its state independently.
|
||||
*/
|
||||
def &&(other: Behavior[T]): Behavior[T] = And(behavior, other)
|
||||
/**
|
||||
* Combine the two behaviors such that incoming messages are given first to
|
||||
* the left behavior and are then only passed on to the right behavior if
|
||||
* the left one returned Unhandled.
|
||||
*/
|
||||
def ||(other: Behavior[T]): Behavior[T] = Or(behavior, other)
|
||||
}
|
||||
|
||||
/**
|
||||
* Widen the wrapped Behavior by placing a funnel in front of it: the supplied
|
||||
* PartialFunction decides which message to pull in (those that it is defined
|
||||
* at) and may transform the incoming message to place them into the wrapped
|
||||
* Behavior’s type hierarchy. Signals are not transformed.
|
||||
*/
|
||||
final case class Widened[T, U >: T](behavior: Behavior[T], matcher: PartialFunction[U, T]) extends Behavior[U] {
|
||||
private def postProcess(ctx: ActorContext[U], behv: Behavior[T]): Behavior[U] =
|
||||
if (isUnhandled(behv)) Unhandled
|
||||
else if (isAlive(behv)) {
|
||||
val next = canonicalize(ctx.asInstanceOf[ActorContext[T]], behv, behavior)
|
||||
if (next eq behavior) Same else Widened(next, matcher)
|
||||
} else Stopped
|
||||
|
||||
override def management(ctx: ActorContext[U], msg: Signal): Behavior[U] =
|
||||
postProcess(ctx, behavior.management(ctx.asInstanceOf[ActorContext[T]], msg))
|
||||
|
||||
override def message(ctx: ActorContext[U], msg: U): Behavior[U] =
|
||||
if (matcher.isDefinedAt(msg))
|
||||
postProcess(ctx, behavior.message(ctx.asInstanceOf[ActorContext[T]], matcher(msg)))
|
||||
else Unhandled
|
||||
|
||||
override def toString: String = s"${behavior.toString}.widen(${LineNumbers(matcher)})"
|
||||
}
|
||||
|
||||
/**
|
||||
* Return this behavior from message processing in order to advise the
|
||||
* system to reuse the previous behavior. This is provided in order to
|
||||
* avoid the allocation overhead of recreating the current behavior where
|
||||
* that is not necessary.
|
||||
*/
|
||||
def Same[T]: Behavior[T] = sameBehavior.asInstanceOf[Behavior[T]]
|
||||
|
||||
/**
|
||||
* Return this behavior from message processing in order to advise the
|
||||
* system to reuse the previous behavior, including the hint that the
|
||||
* message has not been handled. This hint may be used by composite
|
||||
* behaviors that delegate (partial) handling to other behaviors.
|
||||
*/
|
||||
def Unhandled[T]: Behavior[T] = unhandledBehavior.asInstanceOf[Behavior[T]]
|
||||
|
||||
/*
|
||||
* TODO write a Behavior that waits for all child actors to stop and then
|
||||
* runs some cleanup before stopping. The factory for this behavior should
|
||||
* stop and watch all children to get the process started.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Return this behavior from message processing to signal that this actor
|
||||
* shall terminate voluntarily. If this actor has created child actors then
|
||||
* these will be stopped as part of the shutdown procedure. The PostStop
|
||||
* signal that results from stopping this actor will NOT be passed to the
|
||||
* current behavior, it will be effectively ignored.
|
||||
*/
|
||||
def Stopped[T]: Behavior[T] = stoppedBehavior.asInstanceOf[Behavior[T]]
|
||||
|
||||
/**
|
||||
* This behavior does not handle any inputs, it is completely inert.
|
||||
*/
|
||||
def Empty[T]: Behavior[T] = emptyBehavior.asInstanceOf[Behavior[T]]
|
||||
|
||||
/**
|
||||
* This behavior does not handle any inputs, it is completely inert.
|
||||
*/
|
||||
def Ignore[T]: Behavior[T] = ignoreBehavior.asInstanceOf[Behavior[T]]
|
||||
|
||||
/**
|
||||
* Algebraic Data Type modeling either a [[ScalaDSL$.Msg message]] or a
|
||||
* [[ScalaDSL$.Sig signal]], including the [[ActorContext]]. This type is
|
||||
* used by several of the behaviors defined in this DSL, see for example
|
||||
* [[ScalaDSL$.Full]].
|
||||
*/
|
||||
sealed trait MessageOrSignal[T]
|
||||
/**
|
||||
* A message bundled together with the current [[ActorContext]].
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final case class Msg[T](ctx: ActorContext[T], msg: T) extends MessageOrSignal[T]
|
||||
/**
|
||||
* A signal bundled together with the current [[ActorContext]].
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final case class Sig[T](ctx: ActorContext[T], signal: Signal) extends MessageOrSignal[T]
|
||||
|
||||
/**
|
||||
* This type of behavior allows to handle all incoming messages within
|
||||
* the same user-provided partial function, be that a user message or a system
|
||||
* signal. For messages that do not match the partial function the same
|
||||
* behavior is emitted without change. This does entail that unhandled
|
||||
* failures of child actors will lead to a failure in this actor.
|
||||
*
|
||||
* For the lifecycle notifications pertaining to the actor itself this
|
||||
* behavior includes a fallback mechanism: an unhandled [[PreRestart]] signal
|
||||
* will terminate all child actors (transitively) and then emit a [[PostStop]]
|
||||
* signal in addition, whereas an unhandled [[PostRestart]] signal will emit
|
||||
* an additional [[PreStart]] signal.
|
||||
*/
|
||||
final case class Full[T](behavior: PartialFunction[MessageOrSignal[T], Behavior[T]]) extends Behavior[T] {
|
||||
override def management(ctx: ActorContext[T], msg: Signal): Behavior[T] = {
|
||||
lazy val fallback: (MessageOrSignal[T]) ⇒ Behavior[T] = {
|
||||
case Sig(context, PreRestart(_)) ⇒
|
||||
context.children foreach { child ⇒
|
||||
context.unwatch(child.untypedRef)
|
||||
context.stop(child)
|
||||
}
|
||||
behavior.applyOrElse(Sig(context, PostStop), fallback)
|
||||
case Sig(context, PostRestart(_)) ⇒ behavior.applyOrElse(Sig(context, PreStart), fallback)
|
||||
case _ ⇒ Unhandled
|
||||
}
|
||||
behavior.applyOrElse(Sig(ctx, msg), fallback)
|
||||
}
|
||||
override def message(ctx: ActorContext[T], msg: T): Behavior[T] = {
|
||||
behavior.applyOrElse(Msg(ctx, msg), unhandledFunction)
|
||||
}
|
||||
override def toString = s"Full(${LineNumbers(behavior)})"
|
||||
}
|
||||
|
||||
/**
|
||||
* This type of behavior expects a total function that describes the actor’s
|
||||
* reaction to all system signals or user messages, without providing a
|
||||
* fallback mechanism for either. If you use partial function literal syntax
|
||||
* to create the supplied function then any message not matching the list of
|
||||
* cases will fail this actor with a [[scala.MatchError]].
|
||||
*/
|
||||
final case class FullTotal[T](behavior: MessageOrSignal[T] ⇒ Behavior[T]) extends Behavior[T] {
|
||||
override def management(ctx: ActorContext[T], msg: Signal) = behavior(Sig(ctx, msg))
|
||||
override def message(ctx: ActorContext[T], msg: T) = behavior(Msg(ctx, msg))
|
||||
override def toString = s"FullTotal(${LineNumbers(behavior)})"
|
||||
}
|
||||
|
||||
/**
|
||||
* This type of behavior is created from a total function from the declared
|
||||
* message type to the next behavior, which means that all possible incoming
|
||||
* messages for the given type must be handled. All system signals are
|
||||
* ignored by this behavior, which implies that a failure of a child actor
|
||||
* will be escalated unconditionally.
|
||||
*
|
||||
* This behavior type is most useful for leaf actors that do not create child
|
||||
* actors themselves.
|
||||
*/
|
||||
final case class Total[T](behavior: T ⇒ Behavior[T]) extends Behavior[T] {
|
||||
override def management(ctx: ActorContext[T], msg: Signal): Behavior[T] = msg match {
|
||||
case _ ⇒ Unhandled
|
||||
}
|
||||
override def message(ctx: ActorContext[T], msg: T): Behavior[T] = behavior(msg)
|
||||
override def toString = s"Total(${LineNumbers(behavior)})"
|
||||
}
|
||||
|
||||
/**
|
||||
* This type of Behavior is created from a partial function from the declared
|
||||
* message type to the next behavior, flagging all unmatched messages as
|
||||
* [[ScalaDSL$.Unhandled]]. All system signals are
|
||||
* ignored by this behavior, which implies that a failure of a child actor
|
||||
* will be escalated unconditionally.
|
||||
*
|
||||
* This behavior type is most useful for leaf actors that do not create child
|
||||
* actors themselves.
|
||||
*/
|
||||
final case class Partial[T](behavior: PartialFunction[T, Behavior[T]]) extends Behavior[T] {
|
||||
override def management(ctx: ActorContext[T], msg: Signal): Behavior[T] = msg match {
|
||||
case _ ⇒ Unhandled
|
||||
}
|
||||
override def message(ctx: ActorContext[T], msg: T): Behavior[T] = behavior.applyOrElse(msg, unhandledFunction)
|
||||
override def toString = s"Partial(${LineNumbers(behavior)})"
|
||||
}
|
||||
|
||||
/**
|
||||
* This type of Behavior wraps another Behavior while allowing you to perform
|
||||
* some action upon each received message or signal. It is most commonly used
|
||||
* for logging or tracing what a certain Actor does.
|
||||
*/
|
||||
final case class Tap[T](f: PartialFunction[MessageOrSignal[T], Unit], behavior: Behavior[T]) extends Behavior[T] {
|
||||
private def canonical(behv: Behavior[T]): Behavior[T] =
|
||||
if (isUnhandled(behv)) Unhandled
|
||||
else if (behv eq sameBehavior) Same
|
||||
else if (isAlive(behv)) Tap(f, behv)
|
||||
else Stopped
|
||||
override def management(ctx: ActorContext[T], msg: Signal): Behavior[T] = {
|
||||
f.applyOrElse(Sig(ctx, msg), unitFunction)
|
||||
canonical(behavior.management(ctx, msg))
|
||||
}
|
||||
override def message(ctx: ActorContext[T], msg: T): Behavior[T] = {
|
||||
f.applyOrElse(Msg(ctx, msg), unitFunction)
|
||||
canonical(behavior.message(ctx, msg))
|
||||
}
|
||||
override def toString = s"Tap(${LineNumbers(f)},$behavior)"
|
||||
}
|
||||
object Tap {
|
||||
def monitor[T](monitor: ActorRef[T], behavior: Behavior[T]): Tap[T] = Tap({ case Msg(_, msg) ⇒ monitor ! msg }, behavior)
|
||||
}
|
||||
|
||||
/**
|
||||
* This type of behavior is a variant of [[ScalaDSL$.Total]] that does not
|
||||
* allow the actor to change behavior. It is an efficient choice for stateless
|
||||
* actors, possibly entering such a behavior after finishing its
|
||||
* initialization (which may be modeled using any of the other behavior types).
|
||||
*
|
||||
* This behavior type is most useful for leaf actors that do not create child
|
||||
* actors themselves.
|
||||
*/
|
||||
final case class Static[T](behavior: T ⇒ Unit) extends Behavior[T] {
|
||||
override def management(ctx: ActorContext[T], msg: Signal): Behavior[T] = Unhandled
|
||||
override def message(ctx: ActorContext[T], msg: T): Behavior[T] = {
|
||||
behavior(msg)
|
||||
this
|
||||
}
|
||||
override def toString = s"Static(${LineNumbers(behavior)})"
|
||||
}
|
||||
|
||||
/**
|
||||
* This behavior allows sending messages to itself without going through the
|
||||
* Actor’s mailbox. A message sent like this will be processed before the next
|
||||
* message is taken out of the mailbox. In case of Actor failures outstanding
|
||||
* messages that were sent to the synchronous self reference will be lost.
|
||||
*
|
||||
* This decorator is useful for passing messages between the left and right
|
||||
* sides of [[ScalaDSL$.And]] and [[ScalaDSL$.Or]] combinators.
|
||||
*/
|
||||
final case class SynchronousSelf[T](f: ActorRef[T] ⇒ Behavior[T]) extends Behavior[T] {
|
||||
private val inbox = Inbox.sync[T]("syncbox")
|
||||
private var _behavior = f(inbox.ref)
|
||||
private def behavior = _behavior
|
||||
private def setBehavior(ctx: ActorContext[T], b: Behavior[T]): Unit =
|
||||
_behavior = canonicalize(ctx, b, _behavior)
|
||||
|
||||
// FIXME should we protect against infinite loops?
|
||||
@tailrec private def run(ctx: ActorContext[T], next: Behavior[T]): Behavior[T] = {
|
||||
setBehavior(ctx, next)
|
||||
if (inbox.hasMessages) run(ctx, behavior.message(ctx, inbox.receiveMsg()))
|
||||
else if (isUnhandled(next)) Unhandled
|
||||
else if (isAlive(next)) this
|
||||
else Stopped
|
||||
}
|
||||
|
||||
override def management(ctx: ActorContext[T], msg: Signal): Behavior[T] =
|
||||
run(ctx, behavior.management(ctx, msg))
|
||||
override def message(ctx: ActorContext[T], msg: T): Behavior[T] =
|
||||
run(ctx, behavior.message(ctx, msg))
|
||||
|
||||
override def toString: String = s"SynchronousSelf($behavior)"
|
||||
}
|
||||
|
||||
/**
|
||||
* A behavior combinator that feeds incoming messages and signals both into
|
||||
* the left and right sub-behavior and allows them to evolve independently of
|
||||
* each other. When one of the sub-behaviors terminates the other takes over
|
||||
* exclusively. When both sub-behaviors respond to a [[Failed]] signal, the
|
||||
* response with the higher precedence is chosen (see [[Failed$]]).
|
||||
*/
|
||||
final case class And[T](left: Behavior[T], right: Behavior[T]) extends Behavior[T] {
|
||||
|
||||
override def management(ctx: ActorContext[T], msg: Signal): Behavior[T] = {
|
||||
val l = left.management(ctx, msg)
|
||||
val r = right.management(ctx, msg)
|
||||
if (isUnhandled(l) && isUnhandled(r)) Unhandled
|
||||
else {
|
||||
val nextLeft = canonicalize(ctx, l, left)
|
||||
val nextRight = canonicalize(ctx, r, right)
|
||||
val leftAlive = isAlive(nextLeft)
|
||||
val rightAlive = isAlive(nextRight)
|
||||
|
||||
if (leftAlive && rightAlive) And(nextLeft, nextRight)
|
||||
else if (leftAlive) nextLeft
|
||||
else if (rightAlive) nextRight
|
||||
else Stopped
|
||||
}
|
||||
}
|
||||
|
||||
override def message(ctx: ActorContext[T], msg: T): Behavior[T] = {
|
||||
val l = left.message(ctx, msg)
|
||||
val r = right.message(ctx, msg)
|
||||
if (isUnhandled(l) && isUnhandled(r)) Unhandled
|
||||
else {
|
||||
val nextLeft = canonicalize(ctx, l, left)
|
||||
val nextRight = canonicalize(ctx, r, right)
|
||||
val leftAlive = isAlive(nextLeft)
|
||||
val rightAlive = isAlive(nextRight)
|
||||
|
||||
if (leftAlive && rightAlive) And(nextLeft, nextRight)
|
||||
else if (leftAlive) nextLeft
|
||||
else if (rightAlive) nextRight
|
||||
else Stopped
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A behavior combinator that feeds incoming messages and signals either into
|
||||
* the left or right sub-behavior and allows them to evolve independently of
|
||||
* each other. The message or signal is passed first into the left sub-behavior
|
||||
* and only if that results in [[ScalaDSL$.Unhandled]] is it passed to the right
|
||||
* sub-behavior. When one of the sub-behaviors terminates the other takes over
|
||||
* exclusively. When both sub-behaviors respond to a [[Failed]] signal, the
|
||||
* response with the higher precedence is chosen (see [[Failed$]]).
|
||||
*/
|
||||
final case class Or[T](left: Behavior[T], right: Behavior[T]) extends Behavior[T] {
|
||||
|
||||
override def management(ctx: ActorContext[T], msg: Signal): Behavior[T] =
|
||||
left.management(ctx, msg) match {
|
||||
case b if isUnhandled(b) ⇒
|
||||
val r = right.management(ctx, msg)
|
||||
if (isUnhandled(r)) Unhandled
|
||||
else {
|
||||
val nr = canonicalize(ctx, r, right)
|
||||
if (isAlive(nr)) Or(left, nr) else left
|
||||
}
|
||||
case nl ⇒
|
||||
val next = canonicalize(ctx, nl, left)
|
||||
if (isAlive(next)) Or(next, right) else right
|
||||
}
|
||||
|
||||
override def message(ctx: ActorContext[T], msg: T): Behavior[T] =
|
||||
left.message(ctx, msg) match {
|
||||
case b if isUnhandled(b) ⇒
|
||||
val r = right.message(ctx, msg)
|
||||
if (isUnhandled(r)) Unhandled
|
||||
else {
|
||||
val nr = canonicalize(ctx, r, right)
|
||||
if (isAlive(nr)) Or(left, nr) else left
|
||||
}
|
||||
case nl ⇒
|
||||
val next = canonicalize(ctx, nl, left)
|
||||
if (isAlive(next)) Or(next, right) else right
|
||||
}
|
||||
}
|
||||
|
||||
// TODO
|
||||
// final case class Selective[T](timeout: FiniteDuration, selector: PartialFunction[T, Behavior[T]], onTimeout: () ⇒ Behavior[T])
|
||||
|
||||
/**
|
||||
* A behavior decorator that extracts the self [[ActorRef]] while receiving the
|
||||
* the first signal or message and uses that to construct the real behavior
|
||||
* (which will then also receive that signal or message).
|
||||
*
|
||||
* Example:
|
||||
* {{{
|
||||
* SelfAware[MyCommand] { self =>
|
||||
* Simple {
|
||||
* case cmd =>
|
||||
* }
|
||||
* }
|
||||
* }}}
|
||||
*
|
||||
* This can also be used together with implicitly sender-capturing message
|
||||
* types:
|
||||
* {{{
|
||||
* final case class OtherMsg(msg: String)(implicit val replyTo: ActorRef[Reply])
|
||||
*
|
||||
* SelfAware[MyCommand] { implicit self =>
|
||||
* Simple {
|
||||
* case cmd =>
|
||||
* other ! OtherMsg("hello") // assuming Reply <: MyCommand
|
||||
* }
|
||||
* }
|
||||
* }}}
|
||||
*/
|
||||
def SelfAware[T](behavior: ActorRef[T] ⇒ Behavior[T]): Behavior[T] =
|
||||
FullTotal {
|
||||
case Sig(ctx, signal) ⇒
|
||||
val behv = behavior(ctx.self)
|
||||
canonicalize(ctx, behv.management(ctx, signal), behv)
|
||||
case Msg(ctx, msg) ⇒
|
||||
val behv = behavior(ctx.self)
|
||||
canonicalize(ctx, behv.message(ctx, msg), behv)
|
||||
}
|
||||
|
||||
/**
|
||||
* A behavior decorator that extracts the [[ActorContext]] while receiving the
|
||||
* the first signal or message and uses that to construct the real behavior
|
||||
* (which will then also receive that signal or message).
|
||||
*
|
||||
* Example:
|
||||
* {{{
|
||||
* ContextAware[MyCommand] { ctx => Simple {
|
||||
* case cmd =>
|
||||
* ...
|
||||
* }
|
||||
* }
|
||||
* }}}
|
||||
*/
|
||||
def ContextAware[T](behavior: ActorContext[T] ⇒ Behavior[T]): Behavior[T] =
|
||||
FullTotal {
|
||||
case Sig(ctx, signal) ⇒
|
||||
val behv = behavior(ctx)
|
||||
canonicalize(ctx, behv.management(ctx, signal), behv)
|
||||
case Msg(ctx, msg) ⇒
|
||||
val behv = behavior(ctx)
|
||||
canonicalize(ctx, behv.message(ctx, msg), behv)
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API.
|
||||
*/
|
||||
private[akka] val _unhandledFunction = (_: Any) ⇒ Unhandled[Nothing]
|
||||
/**
|
||||
* INTERNAL API.
|
||||
*/
|
||||
private[akka] def unhandledFunction[T, U] = _unhandledFunction.asInstanceOf[(T ⇒ Behavior[U])]
|
||||
|
||||
/**
|
||||
* INTERNAL API.
|
||||
*/
|
||||
private[akka] val _unitFunction = (_: Any) ⇒ ()
|
||||
/**
|
||||
* INTERNAL API.
|
||||
*/
|
||||
private[akka] def unitFunction[T, U] = _unhandledFunction.asInstanceOf[(T ⇒ Behavior[U])]
|
||||
|
||||
}
|
||||
175
akka-typed/src/main/scala/akka/typed/StepWise.scala
Normal file
175
akka-typed/src/main/scala/akka/typed/StepWise.scala
Normal file
|
|
@ -0,0 +1,175 @@
|
|||
/**
|
||||
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.typed
|
||||
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import scala.annotation.tailrec
|
||||
import java.util.concurrent.TimeoutException
|
||||
import scala.concurrent.duration.Deadline
|
||||
import scala.util.control.NonFatal
|
||||
|
||||
/**
|
||||
* This object contains tools for building step-wise behaviors for formulating
|
||||
* a linearly progressing protocol.
|
||||
*
|
||||
* Example:
|
||||
* {{{
|
||||
* import scala.concurrent.duration._
|
||||
*
|
||||
* StepWise[Command] { (ctx, startWith) =>
|
||||
* startWith {
|
||||
* val child = ctx.spawn(...)
|
||||
* child ! msg
|
||||
* child
|
||||
* }.expectMessage(100.millis) { (reply, child) =>
|
||||
* target ! GotReply(reply)
|
||||
* }
|
||||
* }
|
||||
* }}}
|
||||
*
|
||||
* State can be passed from one step to the next by returning it as is
|
||||
* demonstrated with the `child` ActorRef in the example.
|
||||
*
|
||||
* This way of writing Actors can be very useful when writing Actor-based
|
||||
* test procedures for actor systems, hence also the possibility to expect
|
||||
* failures (see [[StepWise$.Steps!.expectFailure]]).
|
||||
*/
|
||||
object StepWise {
|
||||
import Behavior._
|
||||
import ScalaDSL._
|
||||
|
||||
sealed trait AST
|
||||
private final case class Thunk(f: () ⇒ Any) extends AST
|
||||
private final case class ThunkV(f: Any ⇒ Any) extends AST
|
||||
private final case class Message(timeout: FiniteDuration, f: (Any, Any) ⇒ Any, trace: Trace) extends AST
|
||||
private final case class MultiMessage(timeout: FiniteDuration, count: Int, f: (Seq[Any], Any) ⇒ Any, trace: Trace) extends AST
|
||||
private final case class Failure(timeout: FiniteDuration, f: (Failed, Any) ⇒ (Failed.Decision, Any), trace: Trace) extends AST
|
||||
private final case class Termination(timeout: FiniteDuration, f: (Terminated, Any) ⇒ Any, trace: Trace) extends AST
|
||||
|
||||
private sealed trait Trace {
|
||||
def getStackTrace: Array[StackTraceElement]
|
||||
protected def getFrames: Array[StackTraceElement] =
|
||||
Thread.currentThread.getStackTrace.dropWhile { elem ⇒
|
||||
val name = elem.getClassName
|
||||
name.startsWith("java.lang.Thread") || name.startsWith("akka.typed.StepWise")
|
||||
}
|
||||
}
|
||||
private class WithTrace extends Trace {
|
||||
private val trace = getFrames
|
||||
def getStackTrace = trace
|
||||
}
|
||||
private object WithoutTrace extends Trace {
|
||||
def getStackTrace = getFrames
|
||||
}
|
||||
|
||||
final case class Steps[T, U](ops: List[AST], keepTraces: Boolean) {
|
||||
private def getTrace(): Trace =
|
||||
if (keepTraces) new WithTrace
|
||||
else WithoutTrace
|
||||
|
||||
def apply[V](thunk: U ⇒ V): Steps[T, V] =
|
||||
copy(ops = ThunkV(thunk.asInstanceOf[Any ⇒ Any]) :: ops)
|
||||
|
||||
def keep(thunk: U ⇒ Unit): Steps[T, U] =
|
||||
copy(ops = ThunkV(value ⇒ { thunk.asInstanceOf[Any ⇒ Any](value); value }) :: ops)
|
||||
|
||||
def expectMessage[V](timeout: FiniteDuration)(f: (T, U) ⇒ V): Steps[T, V] =
|
||||
copy(ops = Message(timeout, f.asInstanceOf[(Any, Any) ⇒ Any], getTrace()) :: ops)
|
||||
|
||||
def expectMultipleMessages[V](timeout: FiniteDuration, count: Int)(f: (Seq[T], U) ⇒ V): Steps[T, V] =
|
||||
copy(ops = MultiMessage(timeout, count, f.asInstanceOf[(Seq[Any], Any) ⇒ Any], getTrace()) :: ops)
|
||||
|
||||
def expectFailure[V](timeout: FiniteDuration)(f: (Failed, U) ⇒ (Failed.Decision, V)): Steps[T, V] =
|
||||
copy(ops = Failure(timeout, f.asInstanceOf[(Failed, Any) ⇒ (Failed.Decision, Any)], getTrace()) :: ops)
|
||||
|
||||
def expectTermination[V](timeout: FiniteDuration)(f: (Terminated, U) ⇒ V): Steps[T, V] =
|
||||
copy(ops = Termination(timeout, f.asInstanceOf[(Terminated, Any) ⇒ Any], getTrace()) :: ops)
|
||||
|
||||
def expectMessageKeep(timeout: FiniteDuration)(f: (T, U) ⇒ Unit): Steps[T, U] =
|
||||
copy(ops = Message(timeout, (msg, value) ⇒ { f.asInstanceOf[(Any, Any) ⇒ Any](msg, value); value }, getTrace()) :: ops)
|
||||
|
||||
def expectMultipleMessagesKeep(timeout: FiniteDuration, count: Int)(f: (Seq[T], U) ⇒ Unit): Steps[T, U] =
|
||||
copy(ops = MultiMessage(timeout, count, (msgs, value) ⇒ { f.asInstanceOf[(Seq[Any], Any) ⇒ Any](msgs, value); value }, getTrace()) :: ops)
|
||||
|
||||
def expectFailureKeep(timeout: FiniteDuration)(f: (Failed, U) ⇒ Failed.Decision): Steps[T, U] =
|
||||
copy(ops = Failure(timeout, (failed, value) ⇒ f.asInstanceOf[(Failed, Any) ⇒ Failed.Decision](failed, value) -> value, getTrace()) :: ops)
|
||||
|
||||
def expectTerminationKeep(timeout: FiniteDuration)(f: (Terminated, U) ⇒ Unit): Steps[T, U] =
|
||||
copy(ops = Termination(timeout, (t, value) ⇒ { f.asInstanceOf[(Terminated, Any) ⇒ Any](t, value); value }, getTrace()) :: ops)
|
||||
|
||||
def withKeepTraces(b: Boolean): Steps[T, U] = copy(keepTraces = b)
|
||||
}
|
||||
|
||||
class StartWith[T](keepTraces: Boolean) {
|
||||
def apply[U](thunk: ⇒ U): Steps[T, U] = Steps(Thunk(() ⇒ thunk) :: Nil, keepTraces)
|
||||
def withKeepTraces(b: Boolean): StartWith[T] = new StartWith(b)
|
||||
}
|
||||
|
||||
def apply[T](f: (ActorContext[T], StartWith[T]) ⇒ Steps[T, _]): Behavior[T] =
|
||||
Full {
|
||||
case Sig(ctx, PreStart) ⇒ run(ctx, f(ctx, new StartWith(keepTraces = false)).ops.reverse, ())
|
||||
}
|
||||
|
||||
private def throwTimeout(trace: Trace, message: String): Nothing =
|
||||
throw new TimeoutException(message) {
|
||||
override def fillInStackTrace(): Throwable = {
|
||||
setStackTrace(trace.getStackTrace)
|
||||
this
|
||||
}
|
||||
}
|
||||
|
||||
private def throwIllegalState(trace: Trace, message: String): Nothing =
|
||||
throw new IllegalStateException(message) {
|
||||
override def fillInStackTrace(): Throwable = {
|
||||
setStackTrace(trace.getStackTrace)
|
||||
this
|
||||
}
|
||||
}
|
||||
|
||||
private def run[T](ctx: ActorContext[T], ops: List[AST], value: Any): Behavior[T] =
|
||||
ops match {
|
||||
case Thunk(f) :: tail ⇒ run(ctx, tail, f())
|
||||
case ThunkV(f) :: tail ⇒ run(ctx, tail, f(value))
|
||||
case Message(t, f, trace) :: tail ⇒
|
||||
ctx.setReceiveTimeout(t)
|
||||
Full {
|
||||
case Sig(_, ReceiveTimeout) ⇒ throwTimeout(trace, s"timeout of $t expired while waiting for a message")
|
||||
case Msg(_, msg) ⇒ run(ctx, tail, f(msg, value))
|
||||
case Sig(_, other) ⇒ throwIllegalState(trace, s"unexpected $other while waiting for a message")
|
||||
}
|
||||
case MultiMessage(t, c, f, trace) :: tail ⇒
|
||||
val deadline = Deadline.now + t
|
||||
def behavior(count: Int, acc: List[Any]): Behavior[T] = {
|
||||
ctx.setReceiveTimeout(deadline.timeLeft)
|
||||
Full {
|
||||
case Sig(_, ReceiveTimeout) ⇒ throwTimeout(trace, s"timeout of $t expired while waiting for $c messages (got only $count)")
|
||||
case Msg(_, msg) ⇒
|
||||
val nextCount = count + 1
|
||||
if (nextCount == c) {
|
||||
run(ctx, tail, f((msg :: acc).reverse, value))
|
||||
} else behavior(nextCount, msg :: acc)
|
||||
case Sig(_, other) ⇒ throwIllegalState(trace, s"unexpected $other while waiting for $c messages (got $count valid ones)")
|
||||
}
|
||||
}
|
||||
behavior(0, Nil)
|
||||
case Failure(t, f, trace) :: tail ⇒
|
||||
ctx.setReceiveTimeout(t)
|
||||
Full {
|
||||
case Sig(_, ReceiveTimeout) ⇒ throwTimeout(trace, s"timeout of $t expired while waiting for a failure")
|
||||
case Sig(_, failure: Failed) ⇒
|
||||
val (response, v) = f(failure, value)
|
||||
failure.decide(response)
|
||||
run(ctx, tail, v)
|
||||
case other ⇒ throwIllegalState(trace, s"unexpected $other while waiting for a message")
|
||||
}
|
||||
case Termination(t, f, trace) :: tail ⇒
|
||||
ctx.setReceiveTimeout(t)
|
||||
Full {
|
||||
case Sig(_, ReceiveTimeout) ⇒ throwTimeout(trace, s"timeout of $t expired while waiting for termination")
|
||||
case Sig(_, t: Terminated) ⇒ run(ctx, tail, f(t, value))
|
||||
case other ⇒ throwIllegalState(trace, s"unexpected $other while waiting for termination")
|
||||
}
|
||||
case Nil ⇒ Stopped
|
||||
}
|
||||
}
|
||||
122
akka-typed/src/main/scala/akka/typed/patterns/Receiver.scala
Normal file
122
akka-typed/src/main/scala/akka/typed/patterns/Receiver.scala
Normal file
|
|
@ -0,0 +1,122 @@
|
|||
/**
|
||||
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.typed.patterns
|
||||
|
||||
import scala.concurrent.duration._
|
||||
import akka.typed.ActorRef
|
||||
import scala.collection.immutable
|
||||
import akka.typed.Behavior
|
||||
import scala.concurrent.duration.Deadline
|
||||
import akka.typed.ActorContext
|
||||
import java.util.LinkedList
|
||||
import scala.collection.JavaConverters._
|
||||
import akka.typed.ReceiveTimeout
|
||||
import scala.collection.immutable.Queue
|
||||
|
||||
object Receiver {
|
||||
import akka.typed.ScalaDSL._
|
||||
|
||||
sealed trait Command[T]
|
||||
|
||||
/**
|
||||
* Retrieve one message from the Receiver, waiting at most for the given duration.
|
||||
*/
|
||||
final case class GetOne[T](timeout: FiniteDuration)(val replyTo: ActorRef[GetOneResult[T]]) extends Command[T]
|
||||
/**
|
||||
* Retrieve all messages from the Receiver that it has queued after the given
|
||||
* duration has elapsed.
|
||||
*/
|
||||
final case class GetAll[T](timeout: FiniteDuration)(val replyTo: ActorRef[GetAllResult[T]]) extends Command[T]
|
||||
/**
|
||||
* Retrieve the external address of this Receiver (i.e. the side at which it
|
||||
* takes in the messages of type T.
|
||||
*/
|
||||
final case class ExternalAddress[T](replyTo: ActorRef[ActorRef[T]]) extends Command[T]
|
||||
|
||||
sealed trait Replies[T]
|
||||
final case class GetOneResult[T](receiver: ActorRef[Command[T]], msg: Option[T]) extends Replies[T]
|
||||
final case class GetAllResult[T](receiver: ActorRef[Command[T]], msgs: immutable.Seq[T]) extends Replies[T]
|
||||
|
||||
private final case class Enqueue[T](msg: T) extends Command[T]
|
||||
|
||||
def behavior[T]: Behavior[Command[T]] =
|
||||
ContextAware[Any] { ctx ⇒
|
||||
SynchronousSelf { syncself ⇒
|
||||
Or(
|
||||
empty(ctx).widen { case c: Command[t] ⇒ c.asInstanceOf[Command[T]] },
|
||||
Static[Any] {
|
||||
case msg ⇒ syncself ! Enqueue(msg)
|
||||
})
|
||||
}
|
||||
}.narrow
|
||||
|
||||
private def empty[T](ctx: ActorContext[Any]): Behavior[Command[T]] =
|
||||
Total {
|
||||
case ExternalAddress(replyTo) ⇒ { replyTo ! ctx.self; Same }
|
||||
case g @ GetOne(d) if d <= Duration.Zero ⇒ { g.replyTo ! GetOneResult(ctx.self, None); Same }
|
||||
case g @ GetOne(d) ⇒ asked(ctx, Queue(Asked(g.replyTo, Deadline.now + d)))
|
||||
case g @ GetAll(d) if d <= Duration.Zero ⇒ { g.replyTo ! GetAllResult(ctx.self, Nil); Same }
|
||||
case g @ GetAll(d) ⇒ { ctx.schedule(d, ctx.self, GetAll(Duration.Zero)(g.replyTo)); Same }
|
||||
case Enqueue(msg) ⇒ queued(ctx, msg)
|
||||
}
|
||||
|
||||
private def queued[T](ctx: ActorContext[Any], t: T): Behavior[Command[T]] = {
|
||||
val queue = new LinkedList[T]
|
||||
queue.add(t)
|
||||
Total {
|
||||
case ExternalAddress(replyTo) ⇒
|
||||
replyTo ! ctx.self
|
||||
Same
|
||||
case g: GetOne[t] ⇒
|
||||
g.replyTo ! GetOneResult(ctx.self, Some(queue.remove()))
|
||||
if (queue.isEmpty) empty(ctx) else Same
|
||||
case g @ GetAll(d) if d <= Duration.Zero ⇒
|
||||
g.replyTo ! GetAllResult(ctx.self, queue.iterator.asScala.toVector)
|
||||
empty(ctx)
|
||||
case g @ GetAll(d) ⇒
|
||||
ctx.schedule(d, ctx.self, GetAll(Duration.Zero)(g.replyTo))
|
||||
Same
|
||||
case Enqueue(msg) ⇒
|
||||
queue.add(msg)
|
||||
Same
|
||||
}
|
||||
}
|
||||
|
||||
private case class Asked[T](replyTo: ActorRef[GetOneResult[T]], deadline: Deadline)
|
||||
private def asked[T](ctx: ActorContext[Any], queue: Queue[Asked[T]]): Behavior[Command[T]] = {
|
||||
ctx.setReceiveTimeout(queue.map(_.deadline).min.timeLeft)
|
||||
|
||||
Full {
|
||||
case Sig(_, ReceiveTimeout) ⇒
|
||||
val (overdue, remaining) = queue partition (_.deadline.isOverdue)
|
||||
overdue foreach (a ⇒ a.replyTo ! GetOneResult(ctx.self, None))
|
||||
if (remaining.isEmpty) {
|
||||
ctx.setReceiveTimeout(Duration.Undefined)
|
||||
empty(ctx)
|
||||
} else asked(ctx, remaining)
|
||||
case Msg(_, msg) ⇒
|
||||
msg match {
|
||||
case ExternalAddress(replyTo) ⇒ { replyTo ! ctx.self; Same }
|
||||
case g @ GetOne(d) if d <= Duration.Zero ⇒
|
||||
g.replyTo ! GetOneResult(ctx.self, None)
|
||||
asked(ctx, queue)
|
||||
case g @ GetOne(d) ⇒
|
||||
asked(ctx, queue enqueue Asked(g.replyTo, Deadline.now + d))
|
||||
case g @ GetAll(d) if d <= Duration.Zero ⇒
|
||||
g.replyTo ! GetAllResult(ctx.self, Nil)
|
||||
asked(ctx, queue)
|
||||
case g @ GetAll(d) ⇒
|
||||
ctx.schedule(d, ctx.self, GetAll(Duration.Zero)(g.replyTo))
|
||||
asked(ctx, queue)
|
||||
case Enqueue(msg) ⇒
|
||||
val (ask, q) = queue.dequeue
|
||||
ask.replyTo ! GetOneResult(ctx.self, Some(msg))
|
||||
if (q.isEmpty) {
|
||||
ctx.setReceiveTimeout(Duration.Undefined)
|
||||
empty(ctx)
|
||||
} else asked(ctx, q)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
109
akka-typed/src/main/scala/akka/typed/patterns/Receptionist.scala
Normal file
109
akka-typed/src/main/scala/akka/typed/patterns/Receptionist.scala
Normal file
|
|
@ -0,0 +1,109 @@
|
|||
/**
|
||||
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.typed.patterns
|
||||
|
||||
import akka.typed.ScalaDSL._
|
||||
import akka.typed.ActorRef
|
||||
import akka.typed.Behavior
|
||||
import akka.typed.Terminated
|
||||
|
||||
/**
|
||||
* A Receptionist is an entry point into an Actor hierarchy where select Actors
|
||||
* publish their identity together with the protocols that they implement. Other
|
||||
* Actors need only know the Receptionist’s identity in order to be able to use
|
||||
* the services of the registered Actors.
|
||||
*/
|
||||
object Receptionist {
|
||||
|
||||
/**
|
||||
* A service key is an object that implements this trait for a given protocol
|
||||
* T, meaning that it signifies that the type T is the entry point into the
|
||||
* protocol spoken by that service (think of it as the set of first messages
|
||||
* that a client could send).
|
||||
*/
|
||||
trait ServiceKey[T]
|
||||
|
||||
/**
|
||||
* The set of commands accepted by a Receptionist.
|
||||
*/
|
||||
sealed trait Command
|
||||
/**
|
||||
* Associate the given [[ActorRef]] with the given [[ServiceKey]]. Multiple
|
||||
* registrations can be made for the same key. Unregistration is implied by
|
||||
* the end of the referenced Actor’s lifecycle.
|
||||
*/
|
||||
final case class Register[T](key: ServiceKey[T], address: ActorRef[T])(val replyTo: ActorRef[Registered[T]]) extends Command
|
||||
/**
|
||||
* Query the Receptionist for a list of all Actors implementing the given
|
||||
* protocol.
|
||||
*/
|
||||
final case class Find[T](key: ServiceKey[T])(val replyTo: ActorRef[Listing[T]]) extends Command
|
||||
|
||||
/**
|
||||
* Confirmtion that the given [[ActorRef]] has been associated with the [[ServiceKey]].
|
||||
*/
|
||||
final case class Registered[T](key: ServiceKey[T], address: ActorRef[T])
|
||||
/**
|
||||
* Current listing of all Actors that implement the protocol given by the [[ServiceKey]].
|
||||
*/
|
||||
final case class Listing[T](key: ServiceKey[T], addresses: Set[ActorRef[T]])
|
||||
|
||||
/**
|
||||
* Initial behavior of a receptionist, used to create a new receptionist like in the following:
|
||||
*
|
||||
* {{{
|
||||
* val receptionist: ActorRef[Receptionist.Command] = ctx.spawn(Props(Receptionist.behavior), "receptionist")
|
||||
* }}}
|
||||
*/
|
||||
val behavior: Behavior[Command] = behavior(Map.empty)
|
||||
|
||||
/*
|
||||
* These wrappers are just there to get around the type madness that would otherwise ensue
|
||||
* by declaring a Map[ServiceKey[_], Set[ActorRef[_]]] (and actually trying to use it).
|
||||
*/
|
||||
private class Key(val key: ServiceKey[_]) {
|
||||
override def equals(other: Any) = other match {
|
||||
case k: Key ⇒ key == k.key
|
||||
case _ ⇒ false
|
||||
}
|
||||
override def hashCode = key.hashCode
|
||||
}
|
||||
private object Key {
|
||||
def apply(r: Register[_]) = new Key(r.key)
|
||||
def apply(f: Find[_]) = new Key(f.key)
|
||||
}
|
||||
|
||||
private class Address(val address: ActorRef[_]) {
|
||||
def extract[T]: ActorRef[T] = address.asInstanceOf[ActorRef[T]]
|
||||
override def equals(other: Any) = other match {
|
||||
case a: Address ⇒ address == a.address
|
||||
case _ ⇒ false
|
||||
}
|
||||
override def hashCode = address.hashCode
|
||||
}
|
||||
private object Address {
|
||||
def apply(r: Register[_]) = new Address(r.address)
|
||||
def apply(r: ActorRef[_]) = new Address(r)
|
||||
}
|
||||
|
||||
private def behavior(map: Map[Key, Set[Address]]): Behavior[Command] = Full {
|
||||
case Msg(ctx, r: Register[t]) ⇒
|
||||
ctx.watch(r.address)
|
||||
val key = Key(r)
|
||||
val set = map get key match {
|
||||
case Some(old) ⇒ old + Address(r)
|
||||
case None ⇒ Set(Address(r))
|
||||
}
|
||||
r.replyTo ! Registered(r.key, r.address)
|
||||
behavior(map.updated(key, set))
|
||||
case Msg(ctx, f: Find[t]) ⇒
|
||||
val set = map get Key(f) getOrElse Set.empty
|
||||
f.replyTo ! Listing(f.key, set.map(_.extract[t]))
|
||||
Same
|
||||
case Sig(ctx, Terminated(ref)) ⇒
|
||||
val addr = Address(ref)
|
||||
// this is not at all optimized
|
||||
behavior(map.map { case (k, v) ⇒ k -> (v - addr) })
|
||||
}
|
||||
}
|
||||
18
akka-typed/src/test/resources/reference.conf
Normal file
18
akka-typed/src/test/resources/reference.conf
Normal file
|
|
@ -0,0 +1,18 @@
|
|||
dispatcher-1 {
|
||||
fork-join-executor {
|
||||
parallelism-min=1
|
||||
parallelism-max=1
|
||||
}
|
||||
}
|
||||
dispatcher-2 {
|
||||
fork-join-executor {
|
||||
parallelism-min=2
|
||||
parallelism-max=2
|
||||
}
|
||||
}
|
||||
dispatcher-8 {
|
||||
fork-join-executor {
|
||||
parallelism-min=8
|
||||
parallelism-max=8
|
||||
}
|
||||
}
|
||||
533
akka-typed/src/test/scala/akka/typed/ActorContextSpec.scala
Normal file
533
akka-typed/src/test/scala/akka/typed/ActorContextSpec.scala
Normal file
|
|
@ -0,0 +1,533 @@
|
|||
package akka.typed
|
||||
|
||||
import scala.concurrent.duration._
|
||||
import scala.concurrent.Future
|
||||
import org.scalautils.ConversionCheckedTripleEquals
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.actor.DeadLetterSuppression
|
||||
|
||||
object ActorContextSpec {
|
||||
import ScalaDSL._
|
||||
|
||||
sealed trait Command
|
||||
sealed trait Event
|
||||
|
||||
final case class GotSignal(signal: Signal) extends Event with DeadLetterSuppression
|
||||
|
||||
final case class Ping(replyTo: ActorRef[Pong]) extends Command
|
||||
sealed trait Pong extends Event
|
||||
case object Pong1 extends Pong
|
||||
case object Pong2 extends Pong
|
||||
|
||||
final case class Miss(replyTo: ActorRef[Missed.type]) extends Command
|
||||
case object Missed extends Event
|
||||
|
||||
final case class Renew(replyTo: ActorRef[Renewed.type]) extends Command
|
||||
case object Renewed extends Event
|
||||
|
||||
final case class Throw(ex: Exception) extends Command
|
||||
|
||||
final case class MkChild(name: Option[String], monitor: ActorRef[GotSignal], replyTo: ActorRef[Created]) extends Command
|
||||
final case class Created(ref: ActorRef[Command]) extends Event
|
||||
|
||||
final case class SetTimeout(duration: FiniteDuration, replyTo: ActorRef[TimeoutSet.type]) extends Command
|
||||
case object TimeoutSet extends Event
|
||||
|
||||
final case class Schedule[T](delay: FiniteDuration, target: ActorRef[T], msg: T, replyTo: ActorRef[Scheduled.type]) extends Command
|
||||
case object Scheduled extends Event
|
||||
|
||||
case object Stop extends Command
|
||||
|
||||
final case class Kill(ref: ActorRef[Nothing], replyTo: ActorRef[KillResult]) extends Command
|
||||
sealed trait KillResult extends Event
|
||||
case object Killed extends KillResult
|
||||
case object NotKilled extends KillResult
|
||||
|
||||
final case class Watch(ref: ActorRef[Nothing], replyTo: ActorRef[Watched.type]) extends Command
|
||||
case object Watched extends Event
|
||||
|
||||
final case class Unwatch(ref: ActorRef[Nothing], replyTo: ActorRef[Unwatched.type]) extends Command
|
||||
case object Unwatched extends Event
|
||||
|
||||
final case class GetInfo(replyTo: ActorRef[Info]) extends Command
|
||||
final case class Info(self: ActorRef[Command], props: Props[Command], system: ActorSystem[Nothing]) extends Event
|
||||
|
||||
final case class GetChild(name: String, replyTo: ActorRef[Child]) extends Command
|
||||
final case class Child(c: Option[ActorRef[Nothing]]) extends Event
|
||||
|
||||
final case class GetChildren(replyTo: ActorRef[Children]) extends Command
|
||||
final case class Children(c: Set[ActorRef[Nothing]]) extends Event
|
||||
|
||||
final case class ChildEvent(event: Event) extends Event
|
||||
|
||||
final case class BecomeInert(replyTo: ActorRef[BecameInert.type]) extends Command
|
||||
case object BecameInert extends Event
|
||||
|
||||
def subject(monitor: ActorRef[GotSignal]): Behavior[Command] =
|
||||
FullTotal {
|
||||
case Sig(ctx, signal) ⇒
|
||||
monitor ! GotSignal(signal)
|
||||
signal match {
|
||||
case f: Failed ⇒ f.decide(Failed.Restart)
|
||||
case _ ⇒
|
||||
}
|
||||
Same
|
||||
case Msg(ctx, message) ⇒ message match {
|
||||
case Ping(replyTo) ⇒
|
||||
replyTo ! Pong1
|
||||
Same
|
||||
case Miss(replyTo) ⇒
|
||||
replyTo ! Missed
|
||||
Unhandled
|
||||
case Renew(replyTo) ⇒
|
||||
replyTo ! Renewed
|
||||
subject(monitor)
|
||||
case Throw(ex) ⇒
|
||||
throw ex
|
||||
case MkChild(name, mon, replyTo) ⇒
|
||||
val child = name match {
|
||||
case None ⇒ ctx.spawnAnonymous(Props(subject(mon)))
|
||||
case Some(n) ⇒ ctx.spawn(Props(subject(mon)), n)
|
||||
}
|
||||
replyTo ! Created(child)
|
||||
Same
|
||||
case SetTimeout(d, replyTo) ⇒
|
||||
ctx.setReceiveTimeout(d)
|
||||
replyTo ! TimeoutSet
|
||||
Same
|
||||
case Schedule(delay, target, msg, replyTo) ⇒
|
||||
ctx.schedule(delay, target, msg)
|
||||
replyTo ! Scheduled
|
||||
Same
|
||||
case Stop ⇒ Stopped
|
||||
case Kill(ref, replyTo) ⇒
|
||||
if (ctx.stop(ref)) replyTo ! Killed
|
||||
else replyTo ! NotKilled
|
||||
Same
|
||||
case Watch(ref, replyTo) ⇒
|
||||
ctx.watch[Nothing](ref)
|
||||
replyTo ! Watched
|
||||
Same
|
||||
case Unwatch(ref, replyTo) ⇒
|
||||
ctx.unwatch[Nothing](ref)
|
||||
replyTo ! Unwatched
|
||||
Same
|
||||
case GetInfo(replyTo) ⇒
|
||||
replyTo ! Info(ctx.self, ctx.props, ctx.system)
|
||||
Same
|
||||
case GetChild(name, replyTo) ⇒
|
||||
replyTo ! Child(ctx.child(name))
|
||||
Same
|
||||
case GetChildren(replyTo) ⇒
|
||||
replyTo ! Children(ctx.children.toSet)
|
||||
Same
|
||||
case BecomeInert(replyTo) ⇒
|
||||
replyTo ! BecameInert
|
||||
Full {
|
||||
case Msg(_, Ping(replyTo)) ⇒
|
||||
replyTo ! Pong2
|
||||
Same
|
||||
case Msg(_, Throw(ex)) ⇒
|
||||
throw ex
|
||||
case _ ⇒ Same
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class ActorContextSpec extends TypedSpec(ConfigFactory.parseString(
|
||||
"""|akka {
|
||||
| loglevel = WARNING
|
||||
| actor.debug {
|
||||
| lifecycle = off
|
||||
| autoreceive = off
|
||||
| }
|
||||
|}""".stripMargin)) with ConversionCheckedTripleEquals {
|
||||
import ActorContextSpec._
|
||||
import ScalaDSL._
|
||||
|
||||
trait Tests {
|
||||
/**
|
||||
* The name for the set of tests to be instantiated, used for keeping the test case actors’ names unique.
|
||||
*/
|
||||
def suite: String
|
||||
|
||||
/**
|
||||
* The behavior against which to run all the tests.
|
||||
*/
|
||||
def behavior(ctx: ActorContext[Event]): Behavior[Command]
|
||||
|
||||
def setup(name: String)(proc: (ActorContext[Event], StepWise.Steps[Event, ActorRef[Command]]) ⇒ StepWise.Steps[Event, _]): Future[TypedSpec.Status] =
|
||||
runTest(s"$suite-$name")(StepWise[Event] { (ctx, startWith) ⇒
|
||||
val steps =
|
||||
startWith.withKeepTraces(true)(ctx.spawn(Props(behavior(ctx)), "subject"))
|
||||
.expectMessage(500.millis) { (msg, ref) ⇒
|
||||
msg should ===(GotSignal(PreStart))
|
||||
ref
|
||||
}
|
||||
proc(ctx, steps)
|
||||
})
|
||||
|
||||
private implicit class MkC(val startWith: StepWise.Steps[Event, ActorRef[Command]]) {
|
||||
/**
|
||||
* Ask the subject to create a child actor, setting its behavior to “inert” if requested.
|
||||
* The latter is very useful in order to avoid disturbances with GotSignal(PostStop) in
|
||||
* test procedures that stop this child.
|
||||
*/
|
||||
def mkChild(name: Option[String],
|
||||
monitor: ActorRef[Event],
|
||||
self: ActorRef[Event],
|
||||
inert: Boolean = false): StepWise.Steps[Event, (ActorRef[Command], ActorRef[Command])] = {
|
||||
val s =
|
||||
startWith.keep { subj ⇒
|
||||
subj ! MkChild(name, monitor, self)
|
||||
}.expectMultipleMessages(500.millis, 2) { (msgs, subj) ⇒
|
||||
val child = msgs match {
|
||||
case Created(child) :: ChildEvent(GotSignal(PreStart)) :: Nil ⇒ child
|
||||
case ChildEvent(GotSignal(PreStart)) :: Created(child) :: Nil ⇒ child
|
||||
}
|
||||
(subj, child)
|
||||
}
|
||||
|
||||
if (!inert) s
|
||||
else
|
||||
s.keep {
|
||||
case (subj, child) ⇒
|
||||
child ! BecomeInert(self)
|
||||
}.expectMessageKeep(500.millis) { (msg, _) ⇒
|
||||
msg should ===(BecameInert)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private implicit class MessageStep[T](val startWith: StepWise.Steps[Event, T]) {
|
||||
def stimulate(f: T ⇒ Unit, ev: T ⇒ Event, timeout: FiniteDuration = 500.millis): StepWise.Steps[Event, T] =
|
||||
startWith.keep(f).expectMessageKeep(timeout) { (msg, v) ⇒
|
||||
msg should ===(ev(v))
|
||||
}
|
||||
}
|
||||
|
||||
protected def stop(ref: ActorRef[Command]) = ref ! Stop
|
||||
|
||||
def `00 must canonicalize behaviors`(): Unit = sync(setup("ctx00") { (ctx, startWith) ⇒
|
||||
val self = ctx.self
|
||||
startWith.keep { subj ⇒
|
||||
subj ! Ping(self)
|
||||
}.expectMessageKeep(500.millis) { (msg, subj) ⇒
|
||||
msg should ===(Pong1)
|
||||
subj ! Miss(self)
|
||||
}.expectMessageKeep(500.millis) { (msg, subj) ⇒
|
||||
msg should ===(Missed)
|
||||
subj ! Renew(self)
|
||||
}.expectMessage(500.millis) { (msg, subj) ⇒
|
||||
msg should ===(Renewed)
|
||||
subj ! Ping(self)
|
||||
}.expectMessage(500.millis) { (msg, _) ⇒
|
||||
msg should ===(Pong1)
|
||||
}
|
||||
})
|
||||
|
||||
def `01 must correctly wire the lifecycle hooks`(): Unit = sync(setup("ctx01") { (ctx, startWith) ⇒
|
||||
val self = ctx.self
|
||||
val ex = new Exception("KABOOM1")
|
||||
startWith { subj ⇒
|
||||
val log = muteExpectedException[Exception]("KABOOM1", occurrences = 1)
|
||||
subj ! Throw(ex)
|
||||
(subj, log)
|
||||
}.expectFailureKeep(500.millis) {
|
||||
case (f, (subj, _)) ⇒
|
||||
f.cause should ===(ex)
|
||||
f.child should ===(subj)
|
||||
Failed.Restart
|
||||
}.expectMessage(500.millis) {
|
||||
case (msg, (subj, log)) ⇒
|
||||
msg should ===(GotSignal(PreRestart(ex)))
|
||||
log.assertDone(500.millis)
|
||||
subj
|
||||
}.expectMessage(500.millis) { (msg, subj) ⇒
|
||||
msg should ===(GotSignal(PostRestart(ex)))
|
||||
ctx.stop(subj)
|
||||
}.expectMessage(500.millis) { (msg, _) ⇒
|
||||
msg should ===(GotSignal(PostStop))
|
||||
}
|
||||
})
|
||||
|
||||
def `02 must not signal PostStop after voluntary termination`(): Unit = sync(setup("ctx02") { (ctx, startWith) ⇒
|
||||
startWith.keep { subj ⇒
|
||||
ctx.watch(subj)
|
||||
stop(subj)
|
||||
}.expectTermination(500.millis) { (t, subj) ⇒
|
||||
t.ref should ===(subj)
|
||||
}
|
||||
})
|
||||
|
||||
def `03 must restart and stop a child actor`(): Unit = sync(setup("ctx03") { (ctx, startWith) ⇒
|
||||
val self = ctx.self
|
||||
val ex = new Exception("KABOOM2")
|
||||
startWith.mkChild(None, ctx.spawnAdapter(ChildEvent), self) {
|
||||
case (subj, child) ⇒
|
||||
val log = muteExpectedException[Exception]("KABOOM2", occurrences = 1)
|
||||
child ! Throw(ex)
|
||||
(subj, child, log)
|
||||
}.expectMultipleMessages(500.millis, 3) {
|
||||
case (msgs, (subj, child, log)) ⇒
|
||||
msgs should ===(
|
||||
GotSignal(Failed(`ex`, `child`)) ::
|
||||
ChildEvent(GotSignal(PreRestart(`ex`))) ::
|
||||
ChildEvent(GotSignal(PostRestart(`ex`))) :: Nil)
|
||||
log.assertDone(500.millis)
|
||||
child ! BecomeInert(self) // necessary to avoid PostStop/Terminated interference
|
||||
(subj, child)
|
||||
}.expectMessageKeep(500.millis) {
|
||||
case (msg, (subj, child)) ⇒
|
||||
msg should ===(BecameInert)
|
||||
stop(subj)
|
||||
ctx.watch(child)
|
||||
ctx.watch(subj)
|
||||
}.expectTermination(500.millis) {
|
||||
case (t, (subj, child)) ⇒
|
||||
if (t.ref === child) subj
|
||||
else if (t.ref === subj) child
|
||||
else fail(s"expected termination of either $subj or $child but got $t")
|
||||
}.expectTermination(500.millis) { (t, subj) ⇒
|
||||
t.ref should ===(subj)
|
||||
}
|
||||
})
|
||||
|
||||
def `04 must stop a child actor`(): Unit = sync(setup("ctx04") { (ctx, startWith) ⇒
|
||||
val self = ctx.self
|
||||
startWith.mkChild(Some("A"), ctx.spawnAdapter(ChildEvent), self, inert = true) {
|
||||
case (subj, child) ⇒
|
||||
subj ! Kill(child, self)
|
||||
child
|
||||
}.expectMessageKeep(500.millis) { (msg, child) ⇒
|
||||
msg should ===(Killed)
|
||||
ctx.watch(child)
|
||||
}.expectTermination(500.millis) { (t, child) ⇒
|
||||
t.ref should ===(child)
|
||||
}
|
||||
})
|
||||
|
||||
def `05 must reset behavior upon Restart`(): Unit = sync(setup("ctx05") { (ctx, startWith) ⇒
|
||||
val self = ctx.self
|
||||
val ex = new Exception("KABOOM05")
|
||||
startWith
|
||||
.stimulate(_ ! BecomeInert(self), _ ⇒ BecameInert)
|
||||
.stimulate(_ ! Ping(self), _ ⇒ Pong2) { subj ⇒
|
||||
val log = muteExpectedException[Exception]("KABOOM05")
|
||||
subj ! Throw(ex)
|
||||
(subj, log)
|
||||
}.expectFailureKeep(500.millis) {
|
||||
case (f, (subj, log)) ⇒
|
||||
f.child should ===(subj)
|
||||
f.cause should ===(ex)
|
||||
Failed.Restart
|
||||
}.expectMessage(500.millis) {
|
||||
case (msg, (subj, log)) ⇒
|
||||
msg should ===(GotSignal(PostRestart(ex)))
|
||||
log.assertDone(500.millis)
|
||||
subj
|
||||
}.stimulate(_ ! Ping(self), _ ⇒ Pong1)
|
||||
})
|
||||
|
||||
def `06 must not reset behavior upon Resume`(): Unit = sync(setup("ctx06") { (ctx, startWith) ⇒
|
||||
val self = ctx.self
|
||||
val ex = new Exception("KABOOM05")
|
||||
startWith
|
||||
.stimulate(_ ! BecomeInert(self), _ ⇒ BecameInert)
|
||||
.stimulate(_ ! Ping(self), _ ⇒ Pong2).keep { subj ⇒
|
||||
subj ! Throw(ex)
|
||||
}.expectFailureKeep(500.millis) { (f, subj) ⇒
|
||||
f.child should ===(subj)
|
||||
f.cause should ===(ex)
|
||||
Failed.Resume
|
||||
}.stimulate(_ ! Ping(self), _ ⇒ Pong2)
|
||||
})
|
||||
|
||||
def `07 must stop upon Stop`(): Unit = sync(setup("ctx07") { (ctx, startWith) ⇒
|
||||
val self = ctx.self
|
||||
val ex = new Exception("KABOOM05")
|
||||
startWith
|
||||
.stimulate(_ ! Ping(self), _ ⇒ Pong1).keep { subj ⇒
|
||||
subj ! Throw(ex)
|
||||
ctx.watch(subj)
|
||||
}.expectFailureKeep(500.millis) { (f, subj) ⇒
|
||||
f.child should ===(subj)
|
||||
f.cause should ===(ex)
|
||||
Failed.Stop
|
||||
}.expectMessageKeep(500.millis) { (msg, _) ⇒
|
||||
msg should ===(GotSignal(PostStop))
|
||||
}.expectTermination(500.millis) { (t, subj) ⇒
|
||||
t.ref should ===(subj)
|
||||
}
|
||||
})
|
||||
|
||||
def `08 must not stop non-child actor`(): Unit = sync(setup("ctx08") { (ctx, startWith) ⇒
|
||||
val self = ctx.self
|
||||
startWith.mkChild(Some("A"), ctx.spawnAdapter(ChildEvent), self) { pair ⇒
|
||||
(pair._1, pair._2, ctx.spawn(Props(behavior(ctx)), "A"))
|
||||
}.expectMessage(500.millis) {
|
||||
case (msg, (subj, child, other)) ⇒
|
||||
msg should ===(GotSignal(PreStart))
|
||||
subj ! Kill(other, ctx.self)
|
||||
child
|
||||
}.expectMessageKeep(500.millis) { (msg, _) ⇒
|
||||
msg should ===(NotKilled)
|
||||
}.stimulate(_ ! Ping(self), _ ⇒ Pong1)
|
||||
})
|
||||
|
||||
def `10 must watch a child actor before its termination`(): Unit = sync(setup("ctx10") { (ctx, startWith) ⇒
|
||||
val self = ctx.self
|
||||
startWith.mkChild(None, ctx.spawnAdapter(ChildEvent), self) {
|
||||
case (subj, child) ⇒
|
||||
subj ! Watch(child, self)
|
||||
child
|
||||
}.expectMessageKeep(500.millis) { (msg, child) ⇒
|
||||
msg should ===(Watched)
|
||||
child ! Stop
|
||||
}.expectMessage(500.millis) { (msg, child) ⇒
|
||||
msg should ===(GotSignal(Terminated(child)))
|
||||
}
|
||||
})
|
||||
|
||||
def `11 must watch a child actor after its termination`(): Unit = sync(setup("ctx11") { (ctx, startWith) ⇒
|
||||
val self = ctx.self
|
||||
startWith.mkChild(None, ctx.spawnAdapter(ChildEvent), self).keep {
|
||||
case (subj, child) ⇒
|
||||
ctx.watch(child)
|
||||
child ! Stop
|
||||
}.expectTermination(500.millis) {
|
||||
case (t, (subj, child)) ⇒
|
||||
t should ===(Terminated(child))
|
||||
subj ! Watch(child, blackhole)
|
||||
child
|
||||
}.expectMessage(500.millis) { (msg, child) ⇒
|
||||
msg should ===(GotSignal(Terminated(child)))
|
||||
}
|
||||
})
|
||||
|
||||
def `12 must unwatch a child actor before its termination`(): Unit = sync(setup("ctx12") { (ctx, startWith) ⇒
|
||||
val self = ctx.self
|
||||
startWith.mkChild(None, ctx.spawnAdapter(ChildEvent), self).keep {
|
||||
case (subj, child) ⇒
|
||||
subj ! Watch(child, self)
|
||||
}.expectMessageKeep(500.millis) {
|
||||
case (msg, (subj, child)) ⇒
|
||||
msg should ===(Watched)
|
||||
subj ! Unwatch(child, self)
|
||||
}.expectMessage(500.millis) {
|
||||
case (msg, (subj, child)) ⇒
|
||||
msg should ===(Unwatched)
|
||||
ctx.watch(child)
|
||||
child ! Stop
|
||||
child
|
||||
}.expectTermination(500.millis) { (t, child) ⇒
|
||||
t should ===(Terminated(child))
|
||||
}
|
||||
})
|
||||
|
||||
def `20 must return the right context info`(): Unit = sync(setup("ctx20") { (ctx, startWith) ⇒
|
||||
startWith.keep(_ ! GetInfo(ctx.self))
|
||||
.expectMessage(500.millis) {
|
||||
case (msg: Info, subj) ⇒
|
||||
msg.self should ===(subj)
|
||||
msg.system should ===(system)
|
||||
case (other, _) ⇒
|
||||
fail(s"$other was not an Info(...)")
|
||||
}
|
||||
})
|
||||
|
||||
def `21 must return right info about children`(): Unit = sync(setup("ctx21") { (ctx, startWith) ⇒
|
||||
val self = ctx.self
|
||||
startWith
|
||||
.mkChild(Some("B"), ctx.spawnAdapter(ChildEvent), self)
|
||||
.stimulate(_._1 ! GetChild("A", self), _ ⇒ Child(None))
|
||||
.stimulate(_._1 ! GetChild("B", self), x ⇒ Child(Some(x._2)))
|
||||
.stimulate(_._1 ! GetChildren(self), x ⇒ Children(Set(x._2)))
|
||||
})
|
||||
|
||||
def `30 must set small receive timeout`(): Unit = sync(setup("ctx30") { (ctx, startWith) ⇒
|
||||
val self = ctx.self
|
||||
startWith
|
||||
.stimulate(_ ! SetTimeout(1.nano, self), _ ⇒ TimeoutSet)
|
||||
.expectMessage(500.millis) { (msg, _) ⇒
|
||||
msg should ===(GotSignal(ReceiveTimeout))
|
||||
}
|
||||
})
|
||||
|
||||
def `31 must set large receive timeout`(): Unit = sync(setup("ctx31") { (ctx, startWith) ⇒
|
||||
val self = ctx.self
|
||||
startWith
|
||||
.stimulate(_ ! SetTimeout(1.minute, self), _ ⇒ TimeoutSet)
|
||||
.stimulate(_ ⇒ ctx.schedule(1.second, self, Pong2), _ ⇒ Pong2, 1.5.seconds)
|
||||
.stimulate(_ ! Ping(self), _ ⇒ Pong1)
|
||||
})
|
||||
|
||||
def `32 must schedule a message`(): Unit = sync(setup("ctx32") { (ctx, startWith) ⇒
|
||||
startWith(_ ! Schedule(1.nano, ctx.self, Pong2, ctx.self))
|
||||
.expectMultipleMessages(500.millis, 2) { (msgs, _) ⇒
|
||||
msgs should ===(Scheduled :: Pong2 :: Nil)
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
object `An ActorContext` extends Tests {
|
||||
override def suite = "basic"
|
||||
override def behavior(ctx: ActorContext[Event]): Behavior[Command] = subject(ctx.self)
|
||||
}
|
||||
|
||||
object `An ActorContext with widened Behavior` extends Tests {
|
||||
override def suite = "widened"
|
||||
override def behavior(ctx: ActorContext[Event]): Behavior[Command] = subject(ctx.self).widen { case x ⇒ x }
|
||||
}
|
||||
|
||||
object `An ActorContext with SynchronousSelf` extends Tests {
|
||||
override def suite = "synchronous"
|
||||
override def behavior(ctx: ActorContext[Event]): Behavior[Command] = SynchronousSelf(self ⇒ subject(ctx.self))
|
||||
}
|
||||
|
||||
object `An ActorContext with non-matching Tap` extends Tests {
|
||||
override def suite = "TapNonMatch"
|
||||
override def behavior(ctx: ActorContext[Event]): Behavior[Command] = Tap({ case null ⇒ }, subject(ctx.self))
|
||||
}
|
||||
|
||||
object `An ActorContext with matching Tap` extends Tests {
|
||||
override def suite = "TapMatch"
|
||||
override def behavior(ctx: ActorContext[Event]): Behavior[Command] = Tap({ case _ ⇒ }, subject(ctx.self))
|
||||
}
|
||||
|
||||
private val stoppingBehavior = Full[Command] { case Msg(_, Stop) ⇒ Stopped }
|
||||
|
||||
object `An ActorContext with And (left)` extends Tests {
|
||||
override def suite = "and"
|
||||
override def behavior(ctx: ActorContext[Event]): Behavior[Command] =
|
||||
And(subject(ctx.self), stoppingBehavior)
|
||||
}
|
||||
|
||||
object `An ActorContext with And (right)` extends Tests {
|
||||
override def suite = "and"
|
||||
override def behavior(ctx: ActorContext[Event]): Behavior[Command] =
|
||||
And(stoppingBehavior, subject(ctx.self))
|
||||
}
|
||||
|
||||
object `An ActorContext with Or (left)` extends Tests {
|
||||
override def suite = "basic"
|
||||
override def behavior(ctx: ActorContext[Event]): Behavior[Command] =
|
||||
Or(subject(ctx.self), stoppingBehavior)
|
||||
override def stop(ref: ActorRef[Command]) = {
|
||||
ref ! Stop
|
||||
ref ! Stop
|
||||
}
|
||||
}
|
||||
|
||||
object `An ActorContext with Or (right)` extends Tests {
|
||||
override def suite = "basic"
|
||||
override def behavior(ctx: ActorContext[Event]): Behavior[Command] =
|
||||
Or(stoppingBehavior, subject(ctx.self))
|
||||
override def stop(ref: ActorRef[Command]) = {
|
||||
ref ! Stop
|
||||
ref ! Stop
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
546
akka-typed/src/test/scala/akka/typed/BehaviorSpec.scala
Normal file
546
akka-typed/src/test/scala/akka/typed/BehaviorSpec.scala
Normal file
|
|
@ -0,0 +1,546 @@
|
|||
/**
|
||||
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.typed
|
||||
|
||||
import org.scalautils.ConversionCheckedTripleEquals
|
||||
|
||||
class BehaviorSpec extends TypedSpec with ConversionCheckedTripleEquals {
|
||||
|
||||
sealed trait Command {
|
||||
def expectedResponse(ctx: ActorContext[Command]): Seq[Event] = Nil
|
||||
}
|
||||
case object GetSelf extends Command {
|
||||
override def expectedResponse(ctx: ActorContext[Command]): Seq[Event] = Self(ctx.self) :: Nil
|
||||
}
|
||||
// Behavior under test must return Unhandled
|
||||
case object Miss extends Command {
|
||||
override def expectedResponse(ctx: ActorContext[Command]): Seq[Event] = Missed :: Nil
|
||||
}
|
||||
// Behavior under test must return Same
|
||||
case object Ignore extends Command {
|
||||
override def expectedResponse(ctx: ActorContext[Command]): Seq[Event] = Ignored :: Nil
|
||||
}
|
||||
case object Ping extends Command {
|
||||
override def expectedResponse(ctx: ActorContext[Command]): Seq[Event] = Pong :: Nil
|
||||
}
|
||||
case object Swap extends Command {
|
||||
override def expectedResponse(ctx: ActorContext[Command]): Seq[Event] = Swapped :: Nil
|
||||
}
|
||||
case class GetState(replyTo: ActorRef[State]) extends Command
|
||||
object GetState {
|
||||
def apply()(implicit inbox: Inbox.SyncInbox[State]): GetState = GetState(inbox.ref)
|
||||
}
|
||||
case class AuxPing(id: Int) extends Command {
|
||||
override def expectedResponse(ctx: ActorContext[Command]): Seq[Event] = Pong :: Nil
|
||||
}
|
||||
case object Stop extends Command
|
||||
|
||||
sealed trait Event
|
||||
case class GotSignal(signal: Signal) extends Event
|
||||
case class Self(self: ActorRef[Command]) extends Event
|
||||
case object Missed extends Event
|
||||
case object Ignored extends Event
|
||||
case object Pong extends Event
|
||||
case object Swapped extends Event
|
||||
|
||||
trait State { def next: State }
|
||||
val StateA: State = new State { override def toString = "StateA"; override def next = StateB }
|
||||
val StateB: State = new State { override def toString = "StateB"; override def next = StateA }
|
||||
|
||||
trait Common {
|
||||
def behavior(monitor: ActorRef[Event]): Behavior[Command]
|
||||
|
||||
case class Setup(ctx: EffectfulActorContext[Command], inbox: Inbox.SyncInbox[Event])
|
||||
|
||||
protected def mkCtx(requirePreStart: Boolean = false, factory: (ActorRef[Event]) ⇒ Behavior[Command] = behavior) = {
|
||||
val inbox = Inbox.sync[Event]("evt")
|
||||
val ctx = new EffectfulActorContext("ctx", Props(factory(inbox.ref)), system)
|
||||
val msgs = inbox.receiveAll()
|
||||
if (requirePreStart)
|
||||
msgs should ===(GotSignal(PreStart) :: Nil)
|
||||
Setup(ctx, inbox)
|
||||
}
|
||||
|
||||
protected implicit class Check(val setup: Setup) {
|
||||
def check(signal: Signal): Setup = {
|
||||
setup.ctx.signal(signal)
|
||||
setup.inbox.receiveAll() should ===(GotSignal(signal) :: Nil)
|
||||
setup
|
||||
}
|
||||
def check(command: Command): Setup = {
|
||||
setup.ctx.run(command)
|
||||
setup.inbox.receiveAll() should ===(command.expectedResponse(setup.ctx))
|
||||
setup
|
||||
}
|
||||
def check[T](command: Command, aux: T*)(implicit inbox: Inbox.SyncInbox[T]): Setup = {
|
||||
setup.ctx.run(command)
|
||||
setup.inbox.receiveAll() should ===(command.expectedResponse(setup.ctx))
|
||||
inbox.receiveAll() should ===(aux)
|
||||
setup
|
||||
}
|
||||
def check2(command: Command): Setup = {
|
||||
setup.ctx.run(command)
|
||||
val expected = command.expectedResponse(setup.ctx)
|
||||
setup.inbox.receiveAll() should ===(expected ++ expected)
|
||||
setup
|
||||
}
|
||||
def check2[T](command: Command, aux: T*)(implicit inbox: Inbox.SyncInbox[T]): Setup = {
|
||||
setup.ctx.run(command)
|
||||
val expected = command.expectedResponse(setup.ctx)
|
||||
setup.inbox.receiveAll() should ===(expected ++ expected)
|
||||
inbox.receiveAll() should ===(aux ++ aux)
|
||||
setup
|
||||
}
|
||||
}
|
||||
|
||||
protected val ex = new Exception("mine!")
|
||||
}
|
||||
|
||||
trait Lifecycle extends Common {
|
||||
def `must react to PreStart`(): Unit = {
|
||||
mkCtx(requirePreStart = true)
|
||||
}
|
||||
|
||||
def `must react to PostStop`(): Unit = {
|
||||
mkCtx().check(PostStop)
|
||||
}
|
||||
|
||||
def `must react to PostStop after a message`(): Unit = {
|
||||
mkCtx().check(GetSelf).check(PostStop)
|
||||
}
|
||||
|
||||
def `must react to PreRestart`(): Unit = {
|
||||
mkCtx().check(PreRestart(ex))
|
||||
}
|
||||
|
||||
def `must react to PreRestart after a message`(): Unit = {
|
||||
mkCtx().check(GetSelf).check(PreRestart(ex))
|
||||
}
|
||||
|
||||
def `must react to PostRestart`(): Unit = {
|
||||
mkCtx().check(PostRestart(ex))
|
||||
}
|
||||
|
||||
def `must react to a message after PostRestart`(): Unit = {
|
||||
mkCtx().check(PostRestart(ex)).check(GetSelf)
|
||||
}
|
||||
|
||||
def `must react to Failed`(): Unit = {
|
||||
val setup @ Setup(ctx, inbox) = mkCtx()
|
||||
val f = Failed(ex, inbox.ref)
|
||||
setup.check(f)
|
||||
f.getDecision should ===(Failed.Restart)
|
||||
}
|
||||
|
||||
def `must react to Failed after a message`(): Unit = {
|
||||
val setup @ Setup(ctx, inbox) = mkCtx().check(GetSelf)
|
||||
val f = Failed(ex, inbox.ref)
|
||||
setup.check(f)
|
||||
f.getDecision should ===(Failed.Restart)
|
||||
}
|
||||
|
||||
def `must react to a message after Failed`(): Unit = {
|
||||
val setup @ Setup(ctx, inbox) = mkCtx()
|
||||
val f = Failed(ex, inbox.ref)
|
||||
setup.check(f)
|
||||
f.getDecision should ===(Failed.Restart)
|
||||
setup.check(GetSelf)
|
||||
}
|
||||
|
||||
def `must react to ReceiveTimeout`(): Unit = {
|
||||
mkCtx().check(ReceiveTimeout)
|
||||
}
|
||||
|
||||
def `must react to ReceiveTimeout after a message`(): Unit = {
|
||||
mkCtx().check(GetSelf).check(ReceiveTimeout)
|
||||
}
|
||||
|
||||
def `must react to a message after ReceiveTimeout`(): Unit = {
|
||||
mkCtx().check(ReceiveTimeout).check(GetSelf)
|
||||
}
|
||||
|
||||
def `must react to Terminated`(): Unit = {
|
||||
mkCtx().check(Terminated(Inbox.sync("x").ref))
|
||||
}
|
||||
|
||||
def `must react to Terminated after a message`(): Unit = {
|
||||
mkCtx().check(GetSelf).check(Terminated(Inbox.sync("x").ref))
|
||||
}
|
||||
|
||||
def `must react to a message after Terminated`(): Unit = {
|
||||
mkCtx().check(Terminated(Inbox.sync("x").ref)).check(GetSelf)
|
||||
}
|
||||
}
|
||||
|
||||
trait Messages extends Common {
|
||||
def `must react to two messages`(): Unit = {
|
||||
mkCtx().check(Ping).check(Ping)
|
||||
}
|
||||
|
||||
def `must react to a message after missing one`(): Unit = {
|
||||
mkCtx().check(Miss).check(Ping)
|
||||
}
|
||||
|
||||
def `must react to a message after ignoring one`(): Unit = {
|
||||
mkCtx().check(Ignore).check(Ping)
|
||||
}
|
||||
}
|
||||
|
||||
trait Unhandled extends Common {
|
||||
def `must return Unhandled`(): Unit = {
|
||||
val Setup(ctx, inbox) = mkCtx()
|
||||
ctx.currentBehavior.message(ctx, Miss) should ===(ScalaDSL.Unhandled[Command])
|
||||
inbox.receiveAll() should ===(Missed :: Nil)
|
||||
}
|
||||
}
|
||||
|
||||
trait Stoppable extends Common {
|
||||
def `must stop`(): Unit = {
|
||||
val Setup(ctx, inbox) = mkCtx()
|
||||
ctx.run(Stop)
|
||||
ctx.currentBehavior should ===(ScalaDSL.Stopped[Command])
|
||||
}
|
||||
}
|
||||
|
||||
trait Become extends Common with Unhandled {
|
||||
private implicit val inbox = Inbox.sync[State]("state")
|
||||
|
||||
def `must be in state A`(): Unit = {
|
||||
mkCtx().check(GetState(), StateA)
|
||||
}
|
||||
|
||||
def `must switch to state B`(): Unit = {
|
||||
mkCtx().check(Swap).check(GetState(), StateB)
|
||||
}
|
||||
|
||||
def `must switch back to state A`(): Unit = {
|
||||
mkCtx().check(Swap).check(Swap).check(GetState(), StateA)
|
||||
}
|
||||
}
|
||||
|
||||
trait BecomeWithLifecycle extends Become with Lifecycle {
|
||||
def `must react to PostStop after swap`(): Unit = {
|
||||
mkCtx().check(Swap).check(PostStop)
|
||||
}
|
||||
|
||||
def `must react to PostStop after a message after swap`(): Unit = {
|
||||
mkCtx().check(Swap).check(GetSelf).check(PostStop)
|
||||
}
|
||||
|
||||
def `must react to PreRestart after swap`(): Unit = {
|
||||
mkCtx().check(Swap).check(PreRestart(ex))
|
||||
}
|
||||
|
||||
def `must react to PreRestart after a message after swap`(): Unit = {
|
||||
mkCtx().check(Swap).check(GetSelf).check(PreRestart(ex))
|
||||
}
|
||||
|
||||
def `must react to a message after PostRestart after swap`(): Unit = {
|
||||
mkCtx().check(PostRestart(ex)).check(Swap).check(GetSelf)
|
||||
}
|
||||
|
||||
def `must react to Failed after swap`(): Unit = {
|
||||
val setup @ Setup(ctx, inbox) = mkCtx().check(Swap)
|
||||
val f = Failed(ex, inbox.ref)
|
||||
setup.check(f)
|
||||
f.getDecision should ===(Failed.Restart)
|
||||
}
|
||||
|
||||
def `must react to Failed after a message after swap`(): Unit = {
|
||||
val setup @ Setup(ctx, inbox) = mkCtx().check(Swap).check(GetSelf)
|
||||
val f = Failed(ex, inbox.ref)
|
||||
setup.check(f)
|
||||
f.getDecision should ===(Failed.Restart)
|
||||
}
|
||||
|
||||
def `must react to a message after Failed after swap`(): Unit = {
|
||||
val setup @ Setup(ctx, inbox) = mkCtx().check(Swap)
|
||||
val f = Failed(ex, inbox.ref)
|
||||
setup.check(f)
|
||||
f.getDecision should ===(Failed.Restart)
|
||||
setup.check(GetSelf)
|
||||
}
|
||||
|
||||
def `must react to ReceiveTimeout after swap`(): Unit = {
|
||||
mkCtx().check(Swap).check(ReceiveTimeout)
|
||||
}
|
||||
|
||||
def `must react to ReceiveTimeout after a message after swap`(): Unit = {
|
||||
mkCtx().check(Swap).check(GetSelf).check(ReceiveTimeout)
|
||||
}
|
||||
|
||||
def `must react to a message after ReceiveTimeout after swap`(): Unit = {
|
||||
mkCtx().check(Swap).check(ReceiveTimeout).check(GetSelf)
|
||||
}
|
||||
|
||||
def `must react to Terminated after swap`(): Unit = {
|
||||
mkCtx().check(Swap).check(Terminated(Inbox.sync("x").ref))
|
||||
}
|
||||
|
||||
def `must react to Terminated after a message after swap`(): Unit = {
|
||||
mkCtx().check(Swap).check(GetSelf).check(Terminated(Inbox.sync("x").ref))
|
||||
}
|
||||
|
||||
def `must react to a message after Terminated after swap`(): Unit = {
|
||||
mkCtx().check(Swap).check(Terminated(Inbox.sync("x").ref)).check(GetSelf)
|
||||
}
|
||||
}
|
||||
|
||||
private def mkFull(monitor: ActorRef[Event], state: State = StateA): Behavior[Command] = {
|
||||
import ScalaDSL.{ Full, Msg, Sig, Same, Unhandled, Stopped }
|
||||
Full {
|
||||
case Sig(ctx, signal) ⇒
|
||||
monitor ! GotSignal(signal)
|
||||
signal match {
|
||||
case f: Failed ⇒ f.decide(Failed.Restart)
|
||||
case _ ⇒
|
||||
}
|
||||
Same
|
||||
case Msg(ctx, GetSelf) ⇒
|
||||
monitor ! Self(ctx.self)
|
||||
Same
|
||||
case Msg(ctx, Miss) ⇒
|
||||
monitor ! Missed
|
||||
Unhandled
|
||||
case Msg(ctx, Ignore) ⇒
|
||||
monitor ! Ignored
|
||||
Same
|
||||
case Msg(ctx, Ping) ⇒
|
||||
monitor ! Pong
|
||||
mkFull(monitor, state)
|
||||
case Msg(ctx, Swap) ⇒
|
||||
monitor ! Swapped
|
||||
mkFull(monitor, state.next)
|
||||
case Msg(ctx, GetState(replyTo)) ⇒
|
||||
replyTo ! state
|
||||
Same
|
||||
case Msg(ctx, Stop) ⇒ Stopped
|
||||
}
|
||||
}
|
||||
|
||||
object `A Full Behavior` extends Messages with BecomeWithLifecycle with Stoppable {
|
||||
override def behavior(monitor: ActorRef[Event]): Behavior[Command] = mkFull(monitor)
|
||||
}
|
||||
|
||||
object `A FullTotal Behavior` extends Messages with BecomeWithLifecycle with Stoppable {
|
||||
override def behavior(monitor: ActorRef[Event]): Behavior[Command] = behv(monitor, StateA)
|
||||
private def behv(monitor: ActorRef[Event], state: State): Behavior[Command] = {
|
||||
import ScalaDSL.{ FullTotal, Msg, Sig, Same, Unhandled, Stopped }
|
||||
FullTotal {
|
||||
case Sig(ctx, signal) ⇒
|
||||
monitor ! GotSignal(signal)
|
||||
signal match {
|
||||
case f: Failed ⇒ f.decide(Failed.Restart)
|
||||
case _ ⇒
|
||||
}
|
||||
Same
|
||||
case Msg(ctx, GetSelf) ⇒
|
||||
monitor ! Self(ctx.self)
|
||||
Same
|
||||
case Msg(_, Miss) ⇒
|
||||
monitor ! Missed
|
||||
Unhandled
|
||||
case Msg(_, Ignore) ⇒
|
||||
monitor ! Ignored
|
||||
Same
|
||||
case Msg(_, Ping) ⇒
|
||||
monitor ! Pong
|
||||
behv(monitor, state)
|
||||
case Msg(_, Swap) ⇒
|
||||
monitor ! Swapped
|
||||
behv(monitor, state.next)
|
||||
case Msg(_, GetState(replyTo)) ⇒
|
||||
replyTo ! state
|
||||
Same
|
||||
case Msg(_, Stop) ⇒ Stopped
|
||||
case Msg(_, _: AuxPing) ⇒ Unhandled
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object `A Widened Behavior` extends Messages with BecomeWithLifecycle with Stoppable {
|
||||
override def behavior(monitor: ActorRef[Event]): Behavior[Command] =
|
||||
ScalaDSL.Widened(mkFull(monitor), { case x ⇒ x })
|
||||
}
|
||||
|
||||
object `A ContextAware Behavior` extends Messages with BecomeWithLifecycle with Stoppable {
|
||||
override def behavior(monitor: ActorRef[Event]): Behavior[Command] =
|
||||
ScalaDSL.ContextAware(ctx ⇒ mkFull(monitor))
|
||||
}
|
||||
|
||||
object `A SelfAware Behavior` extends Messages with BecomeWithLifecycle with Stoppable {
|
||||
override def behavior(monitor: ActorRef[Event]): Behavior[Command] =
|
||||
ScalaDSL.SelfAware(self ⇒ mkFull(monitor))
|
||||
}
|
||||
|
||||
object `A non-matching Tap Behavior` extends Messages with BecomeWithLifecycle with Stoppable {
|
||||
override def behavior(monitor: ActorRef[Event]): Behavior[Command] =
|
||||
ScalaDSL.Tap({ case null ⇒ }, mkFull(monitor))
|
||||
}
|
||||
|
||||
object `A matching Tap Behavior` extends Messages with BecomeWithLifecycle with Stoppable {
|
||||
override def behavior(monitor: ActorRef[Event]): Behavior[Command] =
|
||||
ScalaDSL.Tap({ case _ ⇒ }, mkFull(monitor))
|
||||
}
|
||||
|
||||
object `A SynchronousSelf Behavior` extends Messages with BecomeWithLifecycle with Stoppable {
|
||||
import ScalaDSL._
|
||||
|
||||
implicit private val inbox = Inbox.sync[Command]("syncself")
|
||||
|
||||
override def behavior(monitor: ActorRef[Event]): Behavior[Command] =
|
||||
SynchronousSelf(self ⇒ mkFull(monitor))
|
||||
|
||||
private def behavior2(monitor: ActorRef[Event]): Behavior[Command] = {
|
||||
def first(self: ActorRef[Command]) = Tap.monitor(inbox.ref, Partial[Command] {
|
||||
case AuxPing(id) ⇒ { self ! AuxPing(0); second(self) }
|
||||
})
|
||||
def second(self: ActorRef[Command]) = Partial[Command] {
|
||||
case AuxPing(0) ⇒ { self ! AuxPing(1); Same }
|
||||
case AuxPing(1) ⇒ { self ! AuxPing(2); third(self) }
|
||||
}
|
||||
def third(self: ActorRef[Command]) = Partial[Command] {
|
||||
case AuxPing(2) ⇒ { self ! AuxPing(3); Unhandled }
|
||||
case AuxPing(3) ⇒ { self ! Ping; Same }
|
||||
case AuxPing(4) ⇒ { self ! Stop; Stopped }
|
||||
}
|
||||
SynchronousSelf(self ⇒ Or(mkFull(monitor), first(self)))
|
||||
}
|
||||
|
||||
def `must send messages to itself and stop correctly`(): Unit = {
|
||||
val Setup(ctx, _) = mkCtx(factory = behavior2).check[Command](AuxPing(42), Seq(42, 0, 1, 2, 3) map AuxPing: _*)
|
||||
ctx.run(AuxPing(4))
|
||||
inbox.receiveAll() should ===(AuxPing(4) :: Nil)
|
||||
ctx.currentBehavior should ===(Stopped[Command])
|
||||
}
|
||||
}
|
||||
|
||||
trait And extends Common {
|
||||
private implicit val inbox = Inbox.sync[State]("and")
|
||||
|
||||
private def behavior2(monitor: ActorRef[Event]): Behavior[Command] =
|
||||
ScalaDSL.And(mkFull(monitor), mkFull(monitor))
|
||||
|
||||
def `must pass message to both parts`(): Unit = {
|
||||
mkCtx(factory = behavior2).check2(Swap).check2[State](GetState(), StateB)
|
||||
}
|
||||
|
||||
def `must half-terminate`(): Unit = {
|
||||
val Setup(ctx, inbox) = mkCtx()
|
||||
ctx.run(Stop)
|
||||
ctx.currentBehavior should ===(ScalaDSL.Empty[Command])
|
||||
}
|
||||
}
|
||||
|
||||
object `A Behavior combined with And (left)` extends Messages with BecomeWithLifecycle with And {
|
||||
override def behavior(monitor: ActorRef[Event]): Behavior[Command] =
|
||||
ScalaDSL.And(mkFull(monitor), ScalaDSL.Empty)
|
||||
}
|
||||
|
||||
object `A Behavior combined with And (right)` extends Messages with BecomeWithLifecycle with And {
|
||||
override def behavior(monitor: ActorRef[Event]): Behavior[Command] =
|
||||
ScalaDSL.And(ScalaDSL.Empty, mkFull(monitor))
|
||||
}
|
||||
|
||||
trait Or extends Common {
|
||||
private def strange(monitor: ActorRef[Event]): Behavior[Command] =
|
||||
ScalaDSL.Full {
|
||||
case ScalaDSL.Msg(_, Ping | AuxPing(_)) ⇒
|
||||
monitor ! Pong
|
||||
ScalaDSL.Unhandled
|
||||
}
|
||||
|
||||
private def behavior2(monitor: ActorRef[Event]): Behavior[Command] =
|
||||
ScalaDSL.Or(mkFull(monitor), strange(monitor))
|
||||
|
||||
private def behavior3(monitor: ActorRef[Event]): Behavior[Command] =
|
||||
ScalaDSL.Or(strange(monitor), mkFull(monitor))
|
||||
|
||||
def `must pass message only to first interested party`(): Unit = {
|
||||
mkCtx(factory = behavior2).check(Ping).check(AuxPing(0))
|
||||
}
|
||||
|
||||
def `must pass message through both if first is uninterested`(): Unit = {
|
||||
mkCtx(factory = behavior3).check2(Ping).check(AuxPing(0))
|
||||
}
|
||||
|
||||
def `must half-terminate`(): Unit = {
|
||||
val Setup(ctx, inbox) = mkCtx()
|
||||
ctx.run(Stop)
|
||||
ctx.currentBehavior should ===(ScalaDSL.Empty[Command])
|
||||
}
|
||||
}
|
||||
|
||||
object `A Behavior combined with Or (left)` extends Messages with BecomeWithLifecycle with Or {
|
||||
override def behavior(monitor: ActorRef[Event]): Behavior[Command] =
|
||||
ScalaDSL.Or(mkFull(monitor), ScalaDSL.Empty)
|
||||
}
|
||||
|
||||
object `A Behavior combined with Or (right)` extends Messages with BecomeWithLifecycle with Or {
|
||||
override def behavior(monitor: ActorRef[Event]): Behavior[Command] =
|
||||
ScalaDSL.Or(ScalaDSL.Empty, mkFull(monitor))
|
||||
}
|
||||
|
||||
object `A Partial Behavior` extends Messages with Become with Stoppable {
|
||||
override def behavior(monitor: ActorRef[Event]): Behavior[Command] = behv(monitor, StateA)
|
||||
def behv(monitor: ActorRef[Event], state: State): Behavior[Command] =
|
||||
ScalaDSL.Partial {
|
||||
case Ping ⇒
|
||||
monitor ! Pong
|
||||
behv(monitor, state)
|
||||
case Miss ⇒
|
||||
monitor ! Missed
|
||||
ScalaDSL.Unhandled
|
||||
case Ignore ⇒
|
||||
monitor ! Ignored
|
||||
ScalaDSL.Same
|
||||
case Swap ⇒
|
||||
monitor ! Swapped
|
||||
behv(monitor, state.next)
|
||||
case GetState(replyTo) ⇒
|
||||
replyTo ! state
|
||||
ScalaDSL.Same
|
||||
case Stop ⇒ ScalaDSL.Stopped
|
||||
}
|
||||
}
|
||||
|
||||
object `A Total Behavior` extends Messages with Become with Stoppable {
|
||||
override def behavior(monitor: ActorRef[Event]): Behavior[Command] = behv(monitor, StateA)
|
||||
def behv(monitor: ActorRef[Event], state: State): Behavior[Command] =
|
||||
ScalaDSL.Total {
|
||||
case Ping ⇒
|
||||
monitor ! Pong
|
||||
behv(monitor, state)
|
||||
case Miss ⇒
|
||||
monitor ! Missed
|
||||
ScalaDSL.Unhandled
|
||||
case Ignore ⇒
|
||||
monitor ! Ignored
|
||||
ScalaDSL.Same
|
||||
case GetSelf ⇒ ScalaDSL.Unhandled
|
||||
case Swap ⇒
|
||||
monitor ! Swapped
|
||||
behv(monitor, state.next)
|
||||
case GetState(replyTo) ⇒
|
||||
replyTo ! state
|
||||
ScalaDSL.Same
|
||||
case Stop ⇒ ScalaDSL.Stopped
|
||||
case _: AuxPing ⇒ ScalaDSL.Unhandled
|
||||
}
|
||||
}
|
||||
|
||||
object `A Static Behavior` extends Messages {
|
||||
override def behavior(monitor: ActorRef[Event]): Behavior[Command] =
|
||||
ScalaDSL.Static {
|
||||
case Ping ⇒ monitor ! Pong
|
||||
case Miss ⇒ monitor ! Missed
|
||||
case Ignore ⇒ monitor ! Ignored
|
||||
case GetSelf ⇒
|
||||
case Swap ⇒
|
||||
case GetState(_) ⇒
|
||||
case Stop ⇒
|
||||
case _: AuxPing ⇒
|
||||
}
|
||||
}
|
||||
}
|
||||
72
akka-typed/src/test/scala/akka/typed/PerformanceSpec.scala
Normal file
72
akka-typed/src/test/scala/akka/typed/PerformanceSpec.scala
Normal file
|
|
@ -0,0 +1,72 @@
|
|||
/**
|
||||
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.typed
|
||||
|
||||
import ScalaDSL._
|
||||
import Ops._
|
||||
import AskPattern._
|
||||
import StepWise._
|
||||
import scala.concurrent.duration._
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.Future
|
||||
import com.typesafe.config.ConfigFactory
|
||||
|
||||
class PerformanceSpec extends TypedSpec {
|
||||
|
||||
object `A static behavior` {
|
||||
|
||||
object `must be fast` {
|
||||
|
||||
case class Ping(x: Int, pong: ActorRef[Pong], report: ActorRef[Pong])
|
||||
case class Pong(x: Int, ping: ActorRef[Ping], report: ActorRef[Pong])
|
||||
|
||||
def behavior(pairs: Int, pings: Int, count: Int, executor: String) =
|
||||
StepWise[Pong] { (ctx, startWith) ⇒
|
||||
startWith {
|
||||
import ctx.executionContext
|
||||
|
||||
val pinger = Props(SelfAware[Ping](self ⇒ Static { msg ⇒
|
||||
if (msg.x == 0) {
|
||||
msg.report ! Pong(0, self, msg.report)
|
||||
} else msg.pong ! Pong(msg.x - 1, self, msg.report)
|
||||
})).withDispatcher(executor)
|
||||
|
||||
val ponger = Props(SelfAware[Pong](self ⇒ Static { msg ⇒
|
||||
msg.ping ! Ping(msg.x, self, msg.report)
|
||||
})).withDispatcher(executor)
|
||||
|
||||
val actors =
|
||||
for (i ← 1 to pairs)
|
||||
yield (ctx.spawn(pinger, s"pinger-$i"), ctx.spawn(ponger, s"ponger-$i"))
|
||||
|
||||
val start = Deadline.now
|
||||
|
||||
for {
|
||||
(ping, pong) ← actors
|
||||
_ ← 1 to pings
|
||||
} ping ! Ping(count, pong, ctx.self)
|
||||
|
||||
start
|
||||
}.expectMultipleMessages(15.seconds, pairs * pings) { (msgs, start) ⇒
|
||||
val stop = Deadline.now
|
||||
|
||||
val rate = 2L * count * pairs * pings / (stop - start).toMillis
|
||||
info(s"messaging rate was $rate/ms")
|
||||
}
|
||||
}
|
||||
|
||||
def `01 when warming up`(): Unit = sync(runTest("01")(behavior(1, 1, 1000000, "dispatcher-1")))
|
||||
def `02 when using a single message on a single thread`(): Unit = sync(runTest("02")(behavior(1, 1, 1000000, "dispatcher-1")))
|
||||
def `03 when using a 10 messages on a single thread`(): Unit = sync(runTest("03")(behavior(1, 10, 1000000, "dispatcher-1")))
|
||||
def `04 when using a single message on two threads`(): Unit = sync(runTest("04")(behavior(1, 1, 1000000, "dispatcher-2")))
|
||||
def `05 when using a 10 messages on two threads`(): Unit = sync(runTest("05")(behavior(1, 10, 1000000, "dispatcher-2")))
|
||||
def `06 when using 4 pairs with a single message`(): Unit = sync(runTest("06")(behavior(4, 1, 1000000, "dispatcher-8")))
|
||||
def `07 when using 4 pairs with 10 messages`(): Unit = sync(runTest("07")(behavior(4, 10, 1000000, "dispatcher-8")))
|
||||
def `08 when using 8 pairs with a single message`(): Unit = sync(runTest("08")(behavior(8, 1, 1000000, "dispatcher-8")))
|
||||
def `09 when using 8 pairs with 10 messages`(): Unit = sync(runTest("09")(behavior(8, 10, 1000000, "dispatcher-8")))
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
133
akka-typed/src/test/scala/akka/typed/TypedSpec.scala
Normal file
133
akka-typed/src/test/scala/akka/typed/TypedSpec.scala
Normal file
|
|
@ -0,0 +1,133 @@
|
|||
/**
|
||||
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.typed
|
||||
|
||||
import org.scalatest.Spec
|
||||
import org.scalatest.Matchers
|
||||
import org.scalatest.BeforeAndAfterAll
|
||||
import akka.testkit.AkkaSpec
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
import scala.concurrent.Future
|
||||
import com.typesafe.config.Config
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.util.Timeout
|
||||
import scala.reflect.ClassTag
|
||||
import akka.actor.ActorInitializationException
|
||||
import language.existentials
|
||||
import akka.testkit.EventFilter
|
||||
import akka.testkit.TestEvent.Mute
|
||||
|
||||
/**
|
||||
* Helper class for writing tests for typed Actors with ScalaTest.
|
||||
*/
|
||||
class TypedSpec(config: Config) extends Spec with Matchers with BeforeAndAfterAll {
|
||||
import TypedSpec._
|
||||
import AskPattern._
|
||||
|
||||
def this() = this(ConfigFactory.empty)
|
||||
|
||||
implicit val system = ActorSystem(AkkaSpec.getCallerName(classOf[TypedSpec]), Props(guardian()), Some(config withFallback AkkaSpec.testConf))
|
||||
|
||||
implicit def timeout = Timeout(1.minute)
|
||||
|
||||
override def afterAll(): Unit = {
|
||||
Await.result(system ? (Terminate(_)), timeout.duration): Status
|
||||
}
|
||||
|
||||
// TODO remove after basing on ScalaTest 3 with async support
|
||||
def await[T](f: Future[T]): T = Await.result(f, 30.seconds)
|
||||
|
||||
val blackhole = await(system ? Create(Props(ScalaDSL.Full[Any] { case _ ⇒ ScalaDSL.Same }), "blackhole"))
|
||||
|
||||
/**
|
||||
* Run an Actor-based test. The test procedure is most conveniently
|
||||
* formulated using the [[StepWise$]] behavior type.
|
||||
*/
|
||||
def runTest[T: ClassTag](name: String)(behavior: Behavior[T]): Future[Status] =
|
||||
system ? (RunTest(name, Props(behavior), _, timeout.duration))
|
||||
|
||||
// TODO remove after basing on ScalaTest 3 with async support
|
||||
def sync(f: Future[Status]): Unit = {
|
||||
def unwrap(ex: Throwable): Throwable = ex match {
|
||||
case ActorInitializationException(_, _, ex) ⇒ ex
|
||||
case other ⇒ other
|
||||
}
|
||||
|
||||
await(f) match {
|
||||
case Success ⇒ ()
|
||||
case Failed(ex) ⇒ throw unwrap(ex)
|
||||
case Timedout ⇒ fail("test timed out")
|
||||
}
|
||||
}
|
||||
|
||||
def muteExpectedException[T <: Exception: ClassTag](
|
||||
message: String = null,
|
||||
source: String = null,
|
||||
start: String = "",
|
||||
pattern: String = null,
|
||||
occurrences: Int = Int.MaxValue): EventFilter = {
|
||||
val filter = EventFilter(message, source, start, pattern, occurrences)
|
||||
system.eventStream.publish(Mute(filter))
|
||||
filter
|
||||
}
|
||||
|
||||
/**
|
||||
* Group assertion that ensures that the given inboxes are empty.
|
||||
*/
|
||||
def assertEmpty(inboxes: Inbox.SyncInbox[_]*): Unit = {
|
||||
inboxes foreach (i ⇒ withClue(s"inbox $i had messages")(i.hasMessages should be(false)))
|
||||
}
|
||||
}
|
||||
|
||||
object TypedSpec {
|
||||
import ScalaDSL._
|
||||
import akka.{ typed ⇒ t }
|
||||
|
||||
sealed abstract class Start
|
||||
case object Start extends Start
|
||||
|
||||
sealed trait Command
|
||||
case class RunTest[T](name: String, props: Props[T], replyTo: ActorRef[Status], timeout: FiniteDuration) extends Command
|
||||
case class Terminate(reply: ActorRef[Status]) extends Command
|
||||
case class Create[T](props: Props[T], name: String)(val replyTo: ActorRef[ActorRef[T]]) extends Command
|
||||
|
||||
sealed trait Status
|
||||
case object Success extends Status
|
||||
case class Failed(thr: Throwable) extends Status
|
||||
case object Timedout extends Status
|
||||
|
||||
def guardian(outstanding: Map[ActorRef[_], ActorRef[Status]] = Map.empty): Behavior[Command] =
|
||||
FullTotal {
|
||||
case Sig(ctx, f @ t.Failed(ex, test)) ⇒
|
||||
outstanding get test match {
|
||||
case Some(reply) ⇒
|
||||
reply ! Failed(ex)
|
||||
f.decide(t.Failed.Stop)
|
||||
guardian(outstanding - test)
|
||||
case None ⇒
|
||||
f.decide(t.Failed.Stop)
|
||||
Same
|
||||
}
|
||||
case Sig(ctx, Terminated(test)) ⇒
|
||||
outstanding get test match {
|
||||
case Some(reply) ⇒
|
||||
reply ! Success
|
||||
guardian(outstanding - test)
|
||||
case None ⇒ Same
|
||||
}
|
||||
case _: Sig[_] ⇒ Same
|
||||
case Msg(ctx, r: RunTest[t]) ⇒
|
||||
val test = ctx.spawn(r.props, r.name)
|
||||
ctx.schedule(r.timeout, r.replyTo, Timedout)
|
||||
ctx.watch(test)
|
||||
guardian(outstanding + ((test, r.replyTo)))
|
||||
case Msg(_, Terminate(reply)) ⇒
|
||||
reply ! Success
|
||||
Stopped
|
||||
case Msg(ctx, c: Create[t]) ⇒
|
||||
c.replyTo ! ctx.spawn(c.props, c.name)
|
||||
Same
|
||||
}
|
||||
}
|
||||
249
akka-typed/src/test/scala/akka/typed/patterns/ReceiverSpec.scala
Normal file
249
akka-typed/src/test/scala/akka/typed/patterns/ReceiverSpec.scala
Normal file
|
|
@ -0,0 +1,249 @@
|
|||
package akka.typed.patterns
|
||||
|
||||
import akka.typed._
|
||||
import scala.concurrent.duration._
|
||||
import akka.typed.Effect.{ ReceiveTimeoutSet, Scheduled }
|
||||
import scala.util.control.NonFatal
|
||||
import Receiver._
|
||||
|
||||
object ReceiverSpec {
|
||||
case class Msg(x: Int)
|
||||
case class Setup(name: String, creator: ActorContext[Command[Msg]] ⇒ Behavior[Command[Msg]], messages: Int, effects: Int)
|
||||
}
|
||||
|
||||
class ReceiverSpec extends TypedSpec {
|
||||
import ReceiverSpec._
|
||||
|
||||
private val dummyInbox = Inbox.sync[Replies[Msg]]("dummy")
|
||||
|
||||
private val startingPoints: Seq[Setup] = Seq(
|
||||
Setup("initial", ctx ⇒ behavior[Msg], 0, 0),
|
||||
Setup("afterGetOneFirst", afterGetOneFirst, 1, 0),
|
||||
Setup("afterGetOneLater", afterGetOneLater, 1, 2),
|
||||
Setup("afterGetOneTimeout", afterGetOneTimeout, 1, 2),
|
||||
Setup("afterGetAll", afterGetAll, 1, 1),
|
||||
Setup("afterGetAllTimeout", afterGetAllTimeout, 1, 1))
|
||||
|
||||
private def afterGetOneFirst(ctx: ActorContext[Command[Msg]]): Behavior[Command[Msg]] =
|
||||
behavior[Msg]
|
||||
.asInstanceOf[Behavior[Msg]].message(ctx.asInstanceOf[ActorContext[Msg]], Msg(1)).asInstanceOf[Behavior[Command[Msg]]]
|
||||
.message(ctx, GetOne(Duration.Zero)(dummyInbox.ref))
|
||||
|
||||
private def afterGetOneLater(ctx: ActorContext[Command[Msg]]): Behavior[Command[Msg]] =
|
||||
behavior[Msg]
|
||||
.message(ctx, GetOne(1.second)(dummyInbox.ref))
|
||||
.asInstanceOf[Behavior[Msg]].message(ctx.asInstanceOf[ActorContext[Msg]], Msg(1)).asInstanceOf[Behavior[Command[Msg]]]
|
||||
|
||||
private def afterGetOneTimeout(ctx: ActorContext[Command[Msg]]): Behavior[Command[Msg]] =
|
||||
behavior[Msg]
|
||||
.message(ctx, GetOne(1.nano)(dummyInbox.ref))
|
||||
.management(ctx, ReceiveTimeout)
|
||||
|
||||
private def afterGetAll(ctx: ActorContext[Command[Msg]]): Behavior[Command[Msg]] =
|
||||
behavior[Msg]
|
||||
.message(ctx, GetAll(1.nano)(dummyInbox.ref))
|
||||
.asInstanceOf[Behavior[Msg]].message(ctx.asInstanceOf[ActorContext[Msg]], Msg(1)).asInstanceOf[Behavior[Command[Msg]]]
|
||||
.message(ctx, GetAll(Duration.Zero)(dummyInbox.ref))
|
||||
|
||||
private def afterGetAllTimeout(ctx: ActorContext[Command[Msg]]): Behavior[Command[Msg]] =
|
||||
behavior[Msg]
|
||||
.message(ctx, GetAll(1.nano)(dummyInbox.ref))
|
||||
.message(ctx, GetAll(Duration.Zero)(dummyInbox.ref))
|
||||
|
||||
private def setup(name: String, behv: Behavior[Command[Msg]] = behavior[Msg])(
|
||||
proc: (EffectfulActorContext[Command[Msg]], EffectfulActorContext[Msg], Inbox.SyncInbox[Replies[Msg]]) ⇒ Unit): Unit =
|
||||
for (Setup(description, behv, messages, effects) ← startingPoints) {
|
||||
val ctx = new EffectfulActorContext("ctx", Props(ScalaDSL.ContextAware(behv)), system)
|
||||
withClue(s"[running for starting point '$description' (${ctx.currentBehavior})]: ") {
|
||||
dummyInbox.receiveAll() should have size messages
|
||||
ctx.getAllEffects() should have size effects
|
||||
proc(ctx, ctx.asInstanceOf[EffectfulActorContext[Msg]], Inbox.sync[Replies[Msg]](name))
|
||||
}
|
||||
}
|
||||
|
||||
object `A Receiver` {
|
||||
|
||||
/*
|
||||
* This test suite assumes that the Receiver is only one actor with two
|
||||
* sides that share the same ActorRef.
|
||||
*/
|
||||
def `must return "self" as external address`(): Unit =
|
||||
setup("") { (int, ext, _) ⇒
|
||||
val inbox = Inbox.sync[ActorRef[Msg]]("extAddr")
|
||||
int.run(ExternalAddress(inbox.ref))
|
||||
int.hasEffects should be(false)
|
||||
inbox.receiveAll() should be(List(int.self))
|
||||
}
|
||||
|
||||
def `must receive one message which arrived first`(): Unit =
|
||||
setup("getOne") { (int, ext, inbox) ⇒
|
||||
// first with zero timeout
|
||||
ext.run(Msg(1))
|
||||
int.run(GetOne(Duration.Zero)(inbox.ref))
|
||||
int.getAllEffects() should be(Nil)
|
||||
inbox.receiveAll() should be(GetOneResult(int.self, Some(Msg(1))) :: Nil)
|
||||
// then with positive timeout
|
||||
ext.run(Msg(2))
|
||||
int.run(GetOne(1.second)(inbox.ref))
|
||||
int.getAllEffects() should be(Nil)
|
||||
inbox.receiveAll() should be(GetOneResult(int.self, Some(Msg(2))) :: Nil)
|
||||
// then with negative timeout
|
||||
ext.run(Msg(3))
|
||||
int.run(GetOne(-1.second)(inbox.ref))
|
||||
int.getAllEffects() should be(Nil)
|
||||
inbox.receiveAll() should be(GetOneResult(int.self, Some(Msg(3))) :: Nil)
|
||||
}
|
||||
|
||||
def `must receive one message which arrives later`(): Unit =
|
||||
setup("getOneLater") { (int, ext, inbox) ⇒
|
||||
int.run(GetOne(1.second)(inbox.ref))
|
||||
int.getAllEffects() match {
|
||||
case ReceiveTimeoutSet(d) :: Nil ⇒ d > Duration.Zero should be(true)
|
||||
case other ⇒ fail(s"$other was not List(ReceiveTimeoutSet(_))")
|
||||
}
|
||||
inbox.hasMessages should be(false)
|
||||
ext.run(Msg(1))
|
||||
int.getAllEffects() match {
|
||||
case ReceiveTimeoutSet(d) :: Nil ⇒ d should be theSameInstanceAs (Duration.Undefined)
|
||||
}
|
||||
inbox.receiveAll() should be(GetOneResult(int.self, Some(Msg(1))) :: Nil)
|
||||
}
|
||||
|
||||
def `must reply with no message when asked for immediate value`(): Unit =
|
||||
setup("getNone") { (int, ext, inbox) ⇒
|
||||
int.run(GetOne(Duration.Zero)(inbox.ref))
|
||||
int.getAllEffects() should be(Nil)
|
||||
inbox.receiveAll() should be(GetOneResult(int.self, None) :: Nil)
|
||||
int.run(GetOne(-1.second)(inbox.ref))
|
||||
int.getAllEffects() should be(Nil)
|
||||
inbox.receiveAll() should be(GetOneResult(int.self, None) :: Nil)
|
||||
}
|
||||
|
||||
def `must reply with no message after a timeout`(): Unit =
|
||||
setup("getNoneTimeout") { (int, ext, inbox) ⇒
|
||||
int.run(GetOne(1.nano)(inbox.ref))
|
||||
int.getAllEffects() match {
|
||||
case ReceiveTimeoutSet(d) :: Nil ⇒ // okay
|
||||
case other ⇒ fail(s"$other was not List(ReceiveTimeoutSet(_))")
|
||||
}
|
||||
inbox.hasMessages should be(false)
|
||||
// currently this all takes >1ns, but who knows what the future brings
|
||||
Thread.sleep(1)
|
||||
int.signal(ReceiveTimeout)
|
||||
int.getAllEffects() match {
|
||||
case ReceiveTimeoutSet(d) :: Nil ⇒ d should be theSameInstanceAs (Duration.Undefined)
|
||||
case other ⇒ fail(s"$other was not List(ReceiveTimeoutSet(_))")
|
||||
}
|
||||
inbox.receiveAll() should be(GetOneResult(int.self, None) :: Nil)
|
||||
}
|
||||
|
||||
def `must reply with messages which arrived first in the same order (GetOne)`(): Unit =
|
||||
setup("getMoreOrderOne") { (int, ext, inbox) ⇒
|
||||
ext.run(Msg(1))
|
||||
ext.run(Msg(2))
|
||||
ext.run(Msg(3))
|
||||
ext.run(Msg(4))
|
||||
int.run(GetOne(Duration.Zero)(inbox.ref))
|
||||
inbox.receiveAll() should be(GetOneResult(int.self, Some(Msg(1))) :: Nil)
|
||||
int.run(GetOne(Duration.Zero)(inbox.ref))
|
||||
inbox.receiveAll() should be(GetOneResult(int.self, Some(Msg(2))) :: Nil)
|
||||
int.run(GetOne(Duration.Zero)(inbox.ref))
|
||||
int.run(GetOne(Duration.Zero)(inbox.ref))
|
||||
inbox.receiveAll() should be(GetOneResult(int.self, Some(Msg(3))) :: GetOneResult(int.self, Some(Msg(4))) :: Nil)
|
||||
int.hasEffects should be(false)
|
||||
}
|
||||
|
||||
def `must reply with messages which arrived first in the same order (GetAll)`(): Unit =
|
||||
setup("getMoreOrderAll") { (int, ext, inbox) ⇒
|
||||
ext.run(Msg(1))
|
||||
ext.run(Msg(2))
|
||||
int.run(GetAll(Duration.Zero)(inbox.ref))
|
||||
inbox.receiveAll() should be(GetAllResult(int.self, List(Msg(1), Msg(2))) :: Nil)
|
||||
// now with negative timeout
|
||||
ext.run(Msg(3))
|
||||
ext.run(Msg(4))
|
||||
int.run(GetAll(-1.second)(inbox.ref))
|
||||
inbox.receiveAll() should be(GetAllResult(int.self, List(Msg(3), Msg(4))) :: Nil)
|
||||
int.hasEffects should be(false)
|
||||
}
|
||||
|
||||
private def assertScheduled[T, U](s: Scheduled[T], target: ActorRef[U]): U = {
|
||||
s.target should be(target)
|
||||
// unfortunately Scala cannot automatically transfer the hereby established type knowledge
|
||||
s.msg.asInstanceOf[U]
|
||||
}
|
||||
|
||||
def `must reply to GetAll with messages which arrived first`(): Unit =
|
||||
setup("getAllFirst") { (int, ext, inbox) ⇒
|
||||
ext.run(Msg(1))
|
||||
ext.run(Msg(2))
|
||||
int.run(GetAll(1.second)(inbox.ref))
|
||||
val msg = int.getAllEffects() match {
|
||||
case (s: Scheduled[_]) :: Nil ⇒ assertScheduled(s, int.self)
|
||||
}
|
||||
int.run(msg)
|
||||
inbox.receiveAll() should be(GetAllResult(int.self, List(Msg(1), Msg(2))) :: Nil)
|
||||
int.hasEffects should be(false)
|
||||
}
|
||||
|
||||
def `must reply to GetAll with messages which arrived first and later`(): Unit =
|
||||
setup("getAllFirstAndLater") { (int, ext, inbox) ⇒
|
||||
ext.run(Msg(1))
|
||||
ext.run(Msg(2))
|
||||
int.run(GetAll(1.second)(inbox.ref))
|
||||
val msg = int.getAllEffects() match {
|
||||
case (s: Scheduled[_]) :: Nil ⇒ assertScheduled(s, int.self)
|
||||
}
|
||||
inbox.hasMessages should be(false)
|
||||
ext.run(Msg(3))
|
||||
int.run(msg)
|
||||
inbox.receiveAll() should be(GetAllResult(int.self, List(Msg(1), Msg(2), Msg(3))) :: Nil)
|
||||
int.hasEffects should be(false)
|
||||
}
|
||||
|
||||
def `must reply to GetAll with messages which arrived later`(): Unit =
|
||||
setup("getAllLater") { (int, ext, inbox) ⇒
|
||||
int.run(GetAll(1.second)(inbox.ref))
|
||||
val msg = int.getAllEffects() match {
|
||||
case (s: Scheduled[_]) :: Nil ⇒ assertScheduled(s, int.self)
|
||||
}
|
||||
ext.run(Msg(1))
|
||||
ext.run(Msg(2))
|
||||
inbox.hasMessages should be(false)
|
||||
int.run(msg)
|
||||
inbox.receiveAll() should be(GetAllResult(int.self, List(Msg(1), Msg(2))) :: Nil)
|
||||
int.hasEffects should be(false)
|
||||
}
|
||||
|
||||
def `must reply to GetAll immediately while GetOne is pending`(): Unit =
|
||||
setup("getAllWhileGetOne") { (int, ext, inbox) ⇒
|
||||
int.run(GetOne(1.second)(inbox.ref))
|
||||
int.getAllEffects() match {
|
||||
case ReceiveTimeoutSet(d) :: Nil ⇒ // okay
|
||||
case other ⇒ fail(s"$other was not List(ReceiveTimeoutSet(_))")
|
||||
}
|
||||
inbox.hasMessages should be(false)
|
||||
int.run(GetAll(Duration.Zero)(inbox.ref))
|
||||
int.getAllEffects() should have size 1
|
||||
inbox.receiveAll() should be(GetAllResult(int.self, Nil) :: Nil)
|
||||
}
|
||||
|
||||
def `must reply to GetAll later while GetOne is pending`(): Unit =
|
||||
setup("getAllWhileGetOne") { (int, ext, inbox) ⇒
|
||||
int.run(GetOne(1.second)(inbox.ref))
|
||||
int.getAllEffects() match {
|
||||
case ReceiveTimeoutSet(d) :: Nil ⇒ // okay
|
||||
case other ⇒ fail(s"$other was not List(ReceiveTimeoutSet(_))")
|
||||
}
|
||||
inbox.hasMessages should be(false)
|
||||
int.run(GetAll(1.nano)(inbox.ref))
|
||||
val msg = int.getAllEffects() match {
|
||||
case (s: Scheduled[_]) :: ReceiveTimeoutSet(_) :: Nil ⇒ assertScheduled(s, int.self)
|
||||
}
|
||||
inbox.receiveAll() should be(Nil)
|
||||
int.run(msg)
|
||||
inbox.receiveAll() should be(GetAllResult(int.self, Nil) :: Nil)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,129 @@
|
|||
/**
|
||||
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.typed.patterns
|
||||
|
||||
import Receptionist._
|
||||
import akka.typed.ScalaDSL._
|
||||
import akka.typed.AskPattern._
|
||||
import scala.concurrent.Future
|
||||
import scala.concurrent.duration._
|
||||
import akka.typed._
|
||||
|
||||
class ReceptionistSpec extends TypedSpec {
|
||||
|
||||
trait ServiceA
|
||||
case object ServiceKeyA extends ServiceKey[ServiceA]
|
||||
val propsA = Props(Static[ServiceA](msg ⇒ ()))
|
||||
|
||||
trait ServiceB
|
||||
case object ServiceKeyB extends ServiceKey[ServiceB]
|
||||
val propsB = Props(Static[ServiceB](msg ⇒ ()))
|
||||
|
||||
object `A Receptionist` {
|
||||
|
||||
def `must register a service`(): Unit = {
|
||||
val ctx = new EffectfulActorContext("register", Props(behavior), system)
|
||||
val a = Inbox.sync[ServiceA]("a")
|
||||
val r = Inbox.sync[Registered[_]]("r")
|
||||
ctx.run(Register(ServiceKeyA, a.ref)(r.ref))
|
||||
r.receiveMsg() should be(Registered(ServiceKeyA, a.ref))
|
||||
val q = Inbox.sync[Listing[ServiceA]]("q")
|
||||
ctx.run(Find(ServiceKeyA)(q.ref))
|
||||
q.receiveMsg() should be(Listing(ServiceKeyA, Set(a.ref)))
|
||||
assertEmpty(a, r, q)
|
||||
}
|
||||
|
||||
def `must register two services`(): Unit = {
|
||||
val ctx = new EffectfulActorContext("registertwo", Props(behavior), system)
|
||||
val a = Inbox.sync[ServiceA]("a")
|
||||
val r = Inbox.sync[Registered[_]]("r")
|
||||
ctx.run(Register(ServiceKeyA, a.ref)(r.ref))
|
||||
r.receiveMsg() should be(Registered(ServiceKeyA, a.ref))
|
||||
val b = Inbox.sync[ServiceB]("b")
|
||||
ctx.run(Register(ServiceKeyB, b.ref)(r.ref))
|
||||
r.receiveMsg() should be(Registered(ServiceKeyB, b.ref))
|
||||
val q = Inbox.sync[Listing[_]]("q")
|
||||
ctx.run(Find(ServiceKeyA)(q.ref))
|
||||
q.receiveMsg() should be(Listing(ServiceKeyA, Set(a.ref)))
|
||||
ctx.run(Find(ServiceKeyB)(q.ref))
|
||||
q.receiveMsg() should be(Listing(ServiceKeyB, Set(b.ref)))
|
||||
assertEmpty(a, b, r, q)
|
||||
}
|
||||
|
||||
def `must register two services with the same key`(): Unit = {
|
||||
val ctx = new EffectfulActorContext("registertwosame", Props(behavior), system)
|
||||
val a1 = Inbox.sync[ServiceA]("a1")
|
||||
val r = Inbox.sync[Registered[_]]("r")
|
||||
ctx.run(Register(ServiceKeyA, a1.ref)(r.ref))
|
||||
r.receiveMsg() should be(Registered(ServiceKeyA, a1.ref))
|
||||
val a2 = Inbox.sync[ServiceA]("a2")
|
||||
ctx.run(Register(ServiceKeyA, a2.ref)(r.ref))
|
||||
r.receiveMsg() should be(Registered(ServiceKeyA, a2.ref))
|
||||
val q = Inbox.sync[Listing[_]]("q")
|
||||
ctx.run(Find(ServiceKeyA)(q.ref))
|
||||
q.receiveMsg() should be(Listing(ServiceKeyA, Set(a1.ref, a2.ref)))
|
||||
ctx.run(Find(ServiceKeyB)(q.ref))
|
||||
q.receiveMsg() should be(Listing(ServiceKeyB, Set.empty[ActorRef[ServiceB]]))
|
||||
assertEmpty(a1, a2, r, q)
|
||||
}
|
||||
|
||||
def `must unregister services when they terminate`(): Unit = {
|
||||
val ctx = new EffectfulActorContext("registertwosame", Props(behavior), system)
|
||||
val r = Inbox.sync[Registered[_]]("r")
|
||||
val a = Inbox.sync[ServiceA]("a")
|
||||
ctx.run(Register(ServiceKeyA, a.ref)(r.ref))
|
||||
ctx.getEffect() should be(Effect.Watched(a.ref))
|
||||
r.receiveMsg() should be(Registered(ServiceKeyA, a.ref))
|
||||
|
||||
val b = Inbox.sync[ServiceB]("b")
|
||||
ctx.run(Register(ServiceKeyB, b.ref)(r.ref))
|
||||
ctx.getEffect() should be(Effect.Watched(b.ref))
|
||||
r.receiveMsg() should be(Registered(ServiceKeyB, b.ref))
|
||||
|
||||
val c = Inbox.sync[Any]("c")
|
||||
ctx.run(Register(ServiceKeyA, c.ref)(r.ref))
|
||||
ctx.run(Register(ServiceKeyB, c.ref)(r.ref))
|
||||
ctx.getAllEffects() should be(Seq(Effect.Watched(c.ref), Effect.Watched(c.ref)))
|
||||
r.receiveMsg() should be(Registered(ServiceKeyA, c.ref))
|
||||
r.receiveMsg() should be(Registered(ServiceKeyB, c.ref))
|
||||
|
||||
val q = Inbox.sync[Listing[_]]("q")
|
||||
ctx.run(Find(ServiceKeyA)(q.ref))
|
||||
q.receiveMsg() should be(Listing(ServiceKeyA, Set(a.ref, c.ref)))
|
||||
ctx.run(Find(ServiceKeyB)(q.ref))
|
||||
q.receiveMsg() should be(Listing(ServiceKeyB, Set(b.ref, c.ref)))
|
||||
|
||||
ctx.signal(Terminated(c.ref))
|
||||
ctx.run(Find(ServiceKeyA)(q.ref))
|
||||
q.receiveMsg() should be(Listing(ServiceKeyA, Set(a.ref)))
|
||||
ctx.run(Find(ServiceKeyB)(q.ref))
|
||||
q.receiveMsg() should be(Listing(ServiceKeyB, Set(b.ref)))
|
||||
assertEmpty(a, b, c, r, q)
|
||||
}
|
||||
|
||||
def `must work with ask`(): Unit = sync(runTest("Receptionist") {
|
||||
StepWise[Registered[ServiceA]] { (ctx, startWith) ⇒
|
||||
val self = ctx.self
|
||||
import system.executionContext
|
||||
startWith.withKeepTraces(true) {
|
||||
val r = ctx.spawnAnonymous(Props(behavior))
|
||||
val s = ctx.spawnAnonymous(propsA)
|
||||
val f = r ? Register(ServiceKeyA, s)
|
||||
r ! Register(ServiceKeyA, s)(self)
|
||||
(f, s)
|
||||
}.expectMessage(1.second) {
|
||||
case (msg, (f, s)) ⇒
|
||||
msg should be(Registered(ServiceKeyA, s))
|
||||
f foreach (self ! _)
|
||||
s
|
||||
}.expectMessage(1.second) {
|
||||
case (msg, s) ⇒
|
||||
msg should be(Registered(ServiceKeyA, s))
|
||||
}
|
||||
}
|
||||
})
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -77,6 +77,12 @@ object AkkaBuild extends Build {
|
|||
dependencies = Seq(actor)
|
||||
)
|
||||
|
||||
lazy val typed = Project(
|
||||
id = "akka-typed-experimental",
|
||||
base = file("akka-typed"),
|
||||
dependencies = Seq(testkit % "compile;test->test")
|
||||
)
|
||||
|
||||
lazy val actorTests = Project(
|
||||
id = "akka-actor-tests",
|
||||
base = file("akka-actor-tests"),
|
||||
|
|
@ -166,8 +172,9 @@ object AkkaBuild extends Build {
|
|||
base = file("akka-docs"),
|
||||
dependencies = Seq(actor, testkit % "test->test",
|
||||
remote % "compile;test->test", cluster, clusterMetrics, slf4j, agent, camel, osgi,
|
||||
persistence % "compile;test->test", persistenceTck)
|
||||
)
|
||||
persistence % "compile;test->test", persistenceTck,
|
||||
typed % "compile;test->test")
|
||||
)
|
||||
|
||||
lazy val contrib = Project(
|
||||
id = "akka-contrib",
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue