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:
Roland Kuhn 2015-01-28 20:45:21 +01:00
parent 50d1569f37
commit d9efd041f7
40 changed files with 4724 additions and 21 deletions

View file

@ -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;
}
};
}
}

View file

@ -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)
}
}
}

View file

@ -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))
}
}
}
}

View file

@ -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"
}
}

View file

@ -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()

View file

@ -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

View file

@ -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

View file

@ -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(

View 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")
}

View file

@ -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
==============

View 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
}
}

View file

@ -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

View file

@ -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.

View file

@ -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.

View 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 functions ``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 requestreply
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 Actors 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 clients 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 sessions
: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 dont
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`` Actors
: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.

View file

@ -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({

View file

@ -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
View 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

View 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 Actors 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 Actors 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 Actors
* 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 dont 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 behaviors 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]

View 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 Actors 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)
}

View 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 guardians 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])
}

View 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 requestreply 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
}
}

View 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 Actors 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
}

View 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)
}
}

View 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)
}
}

View 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
}
}

View 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)
}

View 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")
}
}
}

View 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
* Behaviors 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 actors
* 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
* Actors 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])]
}

View 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
}
}

View 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)
}
}
}
}

View 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 Receptionists 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 Actors 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) })
}
}

View 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
}
}

View 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
}
}
}

View 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
}
}
}

View 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")))
}
}
}

View 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
}
}

View 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)
}
}
}

View file

@ -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))
}
}
})
}
}

View file

@ -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",