Consistently bind ActorContext to 'context' in the docs (#25852)

* Consistently bind ActorContext to 'ctx' in the docs

... over the lesser uses 'context':

    $ rg 'Behavior[s]?.setup' akka-docs/src/test/scala/docs/actor/TypedActorDocSpec.scala akka-actor-typed-tests/ akka-actor-testkit-typed/ | rg ctx | wc -l
          59
    $ rg 'Behavior[s]?.setup' akka-docs/src/test/scala/docs/actor/TypedActorDocSpec.scala akka-actor-typed-tests/ akka-actor-testkit-typed/ | rg context | wc -l
           8
    $ rg 'Behavior[s]?.receive' akka-docs/src/test/scala/docs/actor/TypedActorDocSpec.scala akka-actor-typed-tests/ akka-actor-testkit-typed/ | rg ctx | wc -l
          66
    $ rg 'Behavior[s]?.receive' akka-docs/src/test/scala/docs/actor/TypedActorDocSpec.scala akka-actor-typed-tests/ akka-actor-testkit-typed/ | rg context | wc -l
           5

* Fix "variable ctx is already defined" warnings

* Replace ctx with context, in docs & doc snippets

    fastmod '\bctx\b' 'context' \
        akka-docs/src/main/paradox/typed
        akka-docs/src/test/scala/docs/actor/TypedActorDocSpec.scala \
        akka-actor-typed-tests/ \
        akka-actor-testkit-typed/ \

* Replace msg with message, in docs & doc snippets

    fastmod '\bmsg\b' 'message' \
        akka-docs/src/test/scala/docs/actor/TypedActorDocSpec.scala \
        akka-actor-typed-tests/ \
        akka-actor-testkit-typed/
        akka-docs/src/main/paradox/typed/

* Fix code alignment
This commit is contained in:
Dale Wijnand 2018-11-05 13:53:18 +00:00 committed by Christopher Batey
parent 7bbac405c3
commit 9941f7d800
80 changed files with 1008 additions and 1008 deletions

View file

@ -108,7 +108,7 @@ object Effect {
/**
* INTERNAL API
* The behavior spawned an anonymous adapter, through `ctx.spawnMessageAdapter`
* The behavior spawned an anonymous adapter, through `context.spawnMessageAdapter`
*/
@InternalApi
private[akka] final class SpawnedAnonymousAdapter[T](val ref: ActorRef[T])
@ -153,19 +153,19 @@ object Effect {
final case class Stopped(childName: String) extends Effect
/**
* The behavior started watching `other`, through `ctx.watch(other)`
* The behavior started watching `other`, through `context.watch(other)`
*/
final case class Watched[T](other: ActorRef[T]) extends Effect
/**
* The behavior started watching `other`, through `ctx.unwatch(other)`
* The behavior started watching `other`, through `context.unwatch(other)`
*/
final case class Unwatched[T](other: ActorRef[T]) extends Effect
/**
* The behavior set a new receive timeout, with `msg` as timeout notification
* The behavior set a new receive timeout, with `message` as timeout notification
*/
final case class ReceiveTimeoutSet[T](d: FiniteDuration, msg: T) extends Effect {
final case class ReceiveTimeoutSet[T](d: FiniteDuration, message: T) extends Effect {
/**
* Java API
*/
@ -177,10 +177,10 @@ object Effect {
sealed abstract class ReceiveTimeoutCancelled extends Effect
/**
* The behavior used `ctx.schedule` to schedule `msg` to be sent to `target` after `delay`
* The behavior used `context.schedule` to schedule `message` to be sent to `target` after `delay`
* FIXME what about events scheduled through the scheduler?
*/
final case class Scheduled[U](delay: FiniteDuration, target: ActorRef[U], msg: U) extends Effect {
final case class Scheduled[U](delay: FiniteDuration, target: ActorRef[U], message: U) extends Effect {
def duration(): java.time.Duration = delay.asJava
}

View file

@ -9,5 +9,5 @@ import scala.util.control.NoStackTrace
/**
* A predefined exception that can be used in tests. It doesn't include a stack trace.
*/
class TestException(msg: String) extends RuntimeException(msg) with NoStackTrace
class TestException(message: String) extends RuntimeException(message) with NoStackTrace

View file

@ -28,7 +28,7 @@ import akka.actor.typed.internal.InternalRecipientRef
override val settings: Settings = new Settings(getClass.getClassLoader, ConfigFactory.empty, name)
override def tell(msg: Nothing): Unit = throw new UnsupportedOperationException("must not send message to ActorSystemStub")
override def tell(message: Nothing): Unit = throw new UnsupportedOperationException("must not send message to ActorSystemStub")
// impl ActorRefImpl
override def isLocal: Boolean = true

View file

@ -29,36 +29,36 @@ private[akka] final class BehaviorTestKitImpl[T](_path: ActorPath, _initialBehav
with akka.actor.testkit.typed.scaladsl.BehaviorTestKit[T] {
// really this should be private, make so when we port out tests that need it
private[akka] val ctx = new EffectfulActorContext[T](_path)
private[akka] val context = new EffectfulActorContext[T](_path)
private[akka] def as[U]: BehaviorTestKitImpl[U] = this.asInstanceOf[BehaviorTestKitImpl[U]]
private var currentUncanonical = _initialBehavior
private var current = Behavior.validateAsInitial(Behavior.start(_initialBehavior, ctx))
private var current = Behavior.validateAsInitial(Behavior.start(_initialBehavior, context))
// execute any future tasks scheduled in Actor's constructor
runAllTasks()
override def retrieveEffect(): Effect = ctx.effectQueue.poll() match {
override def retrieveEffect(): Effect = context.effectQueue.poll() match {
case null NoEffects
case x x
}
override def childInbox[U](name: String): TestInboxImpl[U] = {
val inbox = ctx.childInbox[U](name)
assert(inbox.isDefined, s"Child not created: $name. Children created: [${ctx.childrenNames.mkString(",")}]")
val inbox = context.childInbox[U](name)
assert(inbox.isDefined, s"Child not created: $name. Children created: [${context.childrenNames.mkString(",")}]")
inbox.get
}
override def childInbox[U](ref: ActorRef[U]): TestInboxImpl[U] =
childInbox(ref.path.name)
override def childTestKit[U](child: ActorRef[U]): BehaviorTestKitImpl[U] = ctx.childTestKit(child)
override def childTestKit[U](child: ActorRef[U]): BehaviorTestKitImpl[U] = context.childTestKit(child)
override def selfInbox(): TestInboxImpl[T] = ctx.selfInbox
override def selfInbox(): TestInboxImpl[T] = context.selfInbox
override def retrieveAllEffects(): immutable.Seq[Effect] = {
@tailrec def rec(acc: List[Effect]): List[Effect] = ctx.effectQueue.poll() match {
@tailrec def rec(acc: List[Effect]): List[Effect] = context.effectQueue.poll() match {
case null acc.reverse
case x rec(x :: acc)
}
@ -71,14 +71,14 @@ private[akka] final class BehaviorTestKitImpl[T](_path: ActorPath, _initialBehav
def getAllEffects(): util.List[Effect] = retrieveAllEffects().asJava
override def expectEffect(expectedEffect: Effect): Unit = {
ctx.effectQueue.poll() match {
context.effectQueue.poll() match {
case null assert(expectedEffect == NoEffects, s"expected: $expectedEffect but no effects were recorded")
case effect assert(expectedEffect == effect, s"expected: $expectedEffect but found $effect")
}
}
def expectEffectClass[E <: Effect](effectClass: Class[E]): E = {
ctx.effectQueue.poll() match {
context.effectQueue.poll() match {
case null if effectClass.isAssignableFrom(NoEffects.getClass) effectClass.cast(NoEffects)
case null throw new AssertionError(s"expected: effect type ${effectClass.getName} but no effects were recorded")
case effect if effectClass.isAssignableFrom(effect.getClass) effect.asInstanceOf[E]
@ -87,7 +87,7 @@ private[akka] final class BehaviorTestKitImpl[T](_path: ActorPath, _initialBehav
}
def expectEffectPF[R](f: PartialFunction[Effect, R]): R = {
ctx.effectQueue.poll() match {
context.effectQueue.poll() match {
case null if f.isDefinedAt(NoEffects)
f.apply(NoEffects)
case eff if f.isDefinedAt(eff)
@ -106,7 +106,7 @@ private[akka] final class BehaviorTestKitImpl[T](_path: ActorPath, _initialBehav
private def handleException: Catcher[Unit] = {
case NonFatal(e)
try Behavior.canonicalize(Behavior.interpretSignal(current, ctx, PostStop), current, ctx) // TODO why canonicalize here?
try Behavior.canonicalize(Behavior.interpretSignal(current, context, PostStop), current, context) // TODO why canonicalize here?
catch {
case NonFatal(_) /* ignore, real is logging */
}
@ -114,16 +114,16 @@ private[akka] final class BehaviorTestKitImpl[T](_path: ActorPath, _initialBehav
}
private def runAllTasks(): Unit = {
ctx.executionContext match {
context.executionContext match {
case controlled: ControlledExecutor controlled.runAll()
case _
}
}
override def run(msg: T): Unit = {
override def run(message: T): Unit = {
try {
currentUncanonical = Behavior.interpretMessage(current, ctx, msg)
current = Behavior.canonicalize(currentUncanonical, current, ctx)
currentUncanonical = Behavior.interpretMessage(current, context, message)
current = Behavior.canonicalize(currentUncanonical, current, context)
runAllTasks()
} catch handleException
}
@ -132,10 +132,10 @@ private[akka] final class BehaviorTestKitImpl[T](_path: ActorPath, _initialBehav
override def signal(signal: Signal): Unit = {
try {
currentUncanonical = Behavior.interpretSignal(current, ctx, signal)
current = Behavior.canonicalize(currentUncanonical, current, ctx)
currentUncanonical = Behavior.interpretSignal(current, context, signal)
current = Behavior.canonicalize(currentUncanonical, current, context)
} catch handleException
}
override def hasEffects(): Boolean = !ctx.effectQueue.isEmpty
override def hasEffects(): Boolean = !context.effectQueue.isEmpty
}

View file

@ -23,7 +23,7 @@ import akka.actor.typed.internal.InternalRecipientRef
private val q = new ConcurrentLinkedQueue[Either[SystemMessage, T]]
override def tell(msg: T): Unit = q.add(Right(msg))
override def tell(message: T): Unit = q.add(Right(message))
override def sendSystem(signal: SystemMessage): Unit = q.add(Left(signal))
def hasMessage: Boolean = q.peek match {
@ -41,15 +41,15 @@ import akka.actor.typed.internal.InternalRecipientRef
def hasSomething: Boolean = q.peek != null
def receiveMessage(): T = q.poll match {
case null throw new NoSuchElementException("empty DebugRef")
case Left(signal) throw new IllegalStateException(s"expected message but found signal $signal")
case Right(msg) msg
case null throw new NoSuchElementException("empty DebugRef")
case Left(signal) throw new IllegalStateException(s"expected message but found signal $signal")
case Right(message) message
}
def receiveSignal(): SystemMessage = q.poll match {
case null throw new NoSuchElementException("empty DebugRef")
case Left(signal) signal
case Right(msg) throw new IllegalStateException(s"expected signal but found message $msg")
case null throw new NoSuchElementException("empty DebugRef")
case Left(signal) signal
case Right(message) throw new IllegalStateException(s"expected signal but found message $message")
}
def receiveAll(): List[Either[SystemMessage, T]] = {

View file

@ -62,25 +62,25 @@ import scala.compat.java8.FunctionConverters._
effectQueue.offer(Watched(other))
super.watch(other)
}
override def watchWith[U](other: ActorRef[U], msg: T): Unit = {
override def watchWith[U](other: ActorRef[U], message: T): Unit = {
effectQueue.offer(Watched(other))
super.watchWith(other, msg)
super.watchWith(other, message)
}
override def unwatch[U](other: ActorRef[U]): Unit = {
effectQueue.offer(Unwatched(other))
super.unwatch(other)
}
override def setReceiveTimeout(d: FiniteDuration, msg: T): Unit = {
effectQueue.offer(ReceiveTimeoutSet(d, msg))
super.setReceiveTimeout(d, msg)
override def setReceiveTimeout(d: FiniteDuration, message: T): Unit = {
effectQueue.offer(ReceiveTimeoutSet(d, message))
super.setReceiveTimeout(d, message)
}
override def cancelReceiveTimeout(): Unit = {
effectQueue.offer(ReceiveTimeoutCancelled)
super.cancelReceiveTimeout()
}
override def schedule[U](delay: FiniteDuration, target: ActorRef[U], msg: U): Cancellable = {
effectQueue.offer(Scheduled(delay, target, msg))
super.schedule(delay, target, msg)
override def schedule[U](delay: FiniteDuration, target: ActorRef[U], message: U): Cancellable = {
effectQueue.offer(Scheduled(delay, target, message))
super.schedule(delay, target, message)
}
}

View file

@ -35,9 +35,9 @@ private[akka] final class FunctionRef[-T](
send: (T, FunctionRef[T]) Unit)
extends ActorRef[T] with ActorRefImpl[T] with InternalRecipientRef[T] {
override def tell(msg: T): Unit = {
if (msg == null) throw InvalidMessageException("[null] is not an allowed message")
send(msg, this)
override def tell(message: T): Unit = {
if (message == null) throw InvalidMessageException("[null] is not an allowed message")
send(message, this)
}
// impl ActorRefImpl
@ -156,15 +156,15 @@ final case class CapturedLogEvent(logLevel: LogLevel, message: String,
private val childName = Iterator from 0 map (Helpers.base64(_))
private val loggingAdapter = new StubbedLogger
override def children: Iterable[ActorRef[Nothing]] = _children.values map (_.ctx.self)
override def children: Iterable[ActorRef[Nothing]] = _children.values map (_.context.self)
def childrenNames: Iterable[String] = _children.keys
override def child(name: String): Option[ActorRef[Nothing]] = _children get name map (_.ctx.self)
override def child(name: String): Option[ActorRef[Nothing]] = _children get name map (_.context.self)
override def spawnAnonymous[U](behavior: Behavior[U], props: Props = Props.empty): ActorRef[U] = {
val btk = new BehaviorTestKitImpl[U](path / childName.next() withUid rnd().nextInt(), behavior)
_children += btk.ctx.self.path.name btk
btk.ctx.self
_children += btk.context.self.path.name btk
btk.context.self
}
override def spawn[U](behavior: Behavior[U], name: String, props: Props = Props.empty): ActorRef[U] =
_children get name match {
@ -172,7 +172,7 @@ final case class CapturedLogEvent(logLevel: LogLevel, message: String,
case None
val btk = new BehaviorTestKitImpl[U](path / name withUid rnd().nextInt(), behavior)
_children += name btk
btk.ctx.self
btk.context.self
}
/**
@ -189,12 +189,12 @@ final case class CapturedLogEvent(logLevel: LogLevel, message: String,
}
}
override def watch[U](other: ActorRef[U]): Unit = ()
override def watchWith[U](other: ActorRef[U], msg: T): Unit = ()
override def watchWith[U](other: ActorRef[U], message: T): Unit = ()
override def unwatch[U](other: ActorRef[U]): Unit = ()
override def setReceiveTimeout(d: FiniteDuration, msg: T): Unit = ()
override def setReceiveTimeout(d: FiniteDuration, message: T): Unit = ()
override def cancelReceiveTimeout(): Unit = ()
override def schedule[U](delay: FiniteDuration, target: ActorRef[U], msg: U): untyped.Cancellable = new untyped.Cancellable {
override def schedule[U](delay: FiniteDuration, target: ActorRef[U], message: U): untyped.Cancellable = new untyped.Cancellable {
override def cancel() = false
override def isCancelled = true
}
@ -214,7 +214,7 @@ final case class CapturedLogEvent(logLevel: LogLevel, message: String,
new FunctionRef[U](
p,
(msg, _) { val m = f(msg); if (m != null) { selfInbox.ref ! m; i.selfInbox.ref ! msg } })
(message, _) { val m = f(message); if (m != null) { selfInbox.ref ! m; i.selfInbox.ref ! message } })
}
/**
@ -223,8 +223,8 @@ final case class CapturedLogEvent(logLevel: LogLevel, message: String,
*/
def childInbox[U](child: ActorRef[U]): TestInboxImpl[U] = {
val btk = _children(child.path.name)
if (btk.ctx.self != child) throw new IllegalArgumentException(s"$child is not a child of $this")
btk.ctx.selfInbox.as[U]
if (btk.context.self != child) throw new IllegalArgumentException(s"$child is not a child of $this")
btk.context.selfInbox.as[U]
}
/**
@ -233,14 +233,14 @@ final case class CapturedLogEvent(logLevel: LogLevel, message: String,
*/
def childTestKit[U](child: ActorRef[U]): BehaviorTestKitImpl[U] = {
val btk = _children(child.path.name)
if (btk.ctx.self != child) throw new IllegalArgumentException(s"$child is not a child of $this")
if (btk.context.self != child) throw new IllegalArgumentException(s"$child is not a child of $this")
btk.as
}
/**
* Retrieve the inbox representing the child actor with the given name.
*/
def childInbox[U](name: String): Option[TestInboxImpl[U]] = _children.get(name).map(_.ctx.selfInbox.as[U])
def childInbox[U](name: String): Option[TestInboxImpl[U]] = _children.get(name).map(_.context.selfInbox.as[U])
/**
* Remove the given inbox from the list of children, for example after
@ -253,7 +253,7 @@ final case class CapturedLogEvent(logLevel: LogLevel, message: String,
override def log: Logger = loggingAdapter
/**
* The log entries logged through ctx.log.{debug, info, warn, error} are captured and can be inspected through
* The log entries logged through context.log.{debug, info, warn, error} are captured and can be inspected through
* this method.
*/
def logEntries: List[CapturedLogEvent] = loggingAdapter.logEntries

View file

@ -23,7 +23,7 @@ private[akka] final class TestInboxImpl[T](path: ActorPath)
private val q = new ConcurrentLinkedQueue[T]
override val ref: ActorRef[T] = new FunctionRef[T](path, (msg, self) q.add(msg))
override val ref: ActorRef[T] = new FunctionRef[T](path, (message, self) q.add(message))
override def getRef() = ref
override def receiveMessage(): T = q.poll() match {
@ -33,7 +33,7 @@ private[akka] final class TestInboxImpl[T](path: ActorPath)
override def expectMessage(expectedMessage: T): TestInboxImpl[T] = {
q.poll() match {
case null assert(assertion = false, s"expected msg: $expectedMessage but no messages were received")
case null assert(assertion = false, s"expected message: $expectedMessage but no messages were received")
case message assert(message == expectedMessage, s"expected: $expectedMessage but received $message")
}
this

View file

@ -25,14 +25,14 @@ private[akka] object ActorTestKitGuardian {
final case object Ack
val testKitGuardian: Behavior[TestKitCommand] = Behaviors.receive[TestKitCommand] {
case (ctx, SpawnActor(name, behavior, reply, props))
reply ! ctx.spawn(behavior, name, props)
case (context, SpawnActor(name, behavior, reply, props))
reply ! context.spawn(behavior, name, props)
Behaviors.same
case (ctx, SpawnActorAnonymous(behavior, reply, props))
reply ! ctx.spawnAnonymous(behavior, props)
case (context, SpawnActorAnonymous(behavior, reply, props))
reply ! context.spawnAnonymous(behavior, props)
Behaviors.same
case (ctx, StopActor(ref, reply))
ctx.stop(ref)
case (context, StopActor(ref, reply))
context.stop(ref)
reply ! Ack
Behaviors.same
}
@ -97,9 +97,9 @@ private[akka] object TestKitUtils {
system.terminate()
try Await.ready(system.whenTerminated, timeout) catch {
case _: TimeoutException
val msg = "Failed to stop [%s] within [%s] \n%s".format(system.name, timeout, system.printTree)
if (throwIfShutdownTimesOut) throw new RuntimeException(msg)
else println(msg)
val message = "Failed to stop [%s] within [%s] \n%s".format(system.name, timeout, system.printTree)
if (throwIfShutdownTimesOut) throw new RuntimeException(message)
else println(message)
}
}
}

View file

@ -28,9 +28,9 @@ private[akka] object TestProbeImpl {
private val testActorId = new AtomicInteger(0)
private case class WatchActor[U](actor: ActorRef[U])
private def testActor[M](queue: BlockingDeque[M], terminations: BlockingDeque[Terminated]): Behavior[M] = Behaviors.receive[M] { (ctx, msg)
msg match {
case WatchActor(ref) ctx.watch(ref)
private def testActor[M](queue: BlockingDeque[M], terminations: BlockingDeque[Terminated]): Behavior[M] = Behaviors.receive[M] { (context, message)
message match {
case WatchActor(ref) context.watch(ref)
case other queue.offerLast(other)
}
Behaviors.same
@ -186,10 +186,10 @@ private[akka] final class TestProbeImpl[M](name: String, system: ActorSystem[_])
val start = System.nanoTime()
val maybeMsg = Option(receiveOne(timeout))
maybeMsg match {
case Some(msg)
case Some(message)
try {
fisher(msg) match {
case FishingOutcome.Complete (msg :: seen).reverse
fisher(message) match {
case FishingOutcome.Complete (message :: seen).reverse
case FishingOutcome.Fail(error) throw new AssertionError(s"$error, hint: $hint")
case continue
val newTimeout =
@ -200,7 +200,7 @@ private[akka] final class TestProbeImpl[M](name: String, system: ActorSystem[_])
} else {
continue match {
case FishingOutcome.Continue loop(newTimeout, msg :: seen)
case FishingOutcome.Continue loop(newTimeout, message :: seen)
case FishingOutcome.ContinueAndIgnore loop(newTimeout, seen)
case _ ??? // cannot happen
}
@ -209,7 +209,7 @@ private[akka] final class TestProbeImpl[M](name: String, system: ActorSystem[_])
}
} catch {
case ex: MatchError throw new AssertionError(
s"Unexpected message $msg while fishing for messages, " +
s"Unexpected message $message while fishing for messages, " +
s"seen messages ${seen.reverse}, hint: $hint", ex)
}

View file

@ -64,7 +64,7 @@ abstract class BehaviorTestKit[T] {
def childTestKit[U](child: ActorRef[U]): BehaviorTestKit[U]
/**
* The self inbox contains messages the behavior sent to `ctx.self`
* The self inbox contains messages the behavior sent to `context.self`
*/
def selfInbox(): TestInbox[T]
@ -114,9 +114,9 @@ abstract class BehaviorTestKit[T] {
def isAlive: Boolean
/**
* Send the msg to the behavior and record any [[Effect]]s
* Send the message to the behavior and record any [[Effect]]s
*/
def run(msg: T): Unit
def run(message: T): Unit
/**
* Send the first message in the selfInbox to the behavior and run it, recording [[Effect]]s.

View file

@ -54,26 +54,26 @@ object Effects {
*/
def stopped(childName: String): Stopped = Stopped(childName)
/**
* The behavior started watching `other`, through `ctx.watch(other)`
* The behavior started watching `other`, through `context.watch(other)`
*/
def watched[T](other: ActorRef[T]): Watched[T] = Watched(other)
/**
* The behavior started watching `other`, through `ctx.unwatch(other)`
* The behavior started watching `other`, through `context.unwatch(other)`
*/
def unwatched[T](other: ActorRef[T]): Unwatched[T] = Unwatched(other)
/**
* The behavior set a new receive timeout, with `msg` as timeout notification
* The behavior set a new receive timeout, with `message` as timeout notification
*/
def receiveTimeoutSet[T](d: Duration, msg: T): ReceiveTimeoutSet[T] = ReceiveTimeoutSet(d.asScala, msg)
def receiveTimeoutSet[T](d: Duration, message: T): ReceiveTimeoutSet[T] = ReceiveTimeoutSet(d.asScala, message)
/**
* The behavior used `ctx.schedule` to schedule `msg` to be sent to `target` after `delay`
* The behavior used `context.schedule` to schedule `message` to be sent to `target` after `delay`
* FIXME what about events scheduled through the scheduler?
*/
def scheduled[U](delay: Duration, target: ActorRef[U], msg: U): Scheduled[U] =
Scheduled(delay.asScala, target, msg)
def scheduled[U](delay: Duration, target: ActorRef[U], message: U): Scheduled[U] =
Scheduled(delay.asScala, target, message)
/**
* Used to represent an empty list of effects - in other words, the behavior didn't do anything observable

View file

@ -38,7 +38,7 @@ object BehaviorTestKit {
trait BehaviorTestKit[T] {
// FIXME it is weird that this is public but it is used in BehaviorSpec, could we avoid that?
private[akka] def ctx: akka.actor.typed.ActorContext[T]
private[akka] def context: akka.actor.typed.ActorContext[T]
/**
* Requests the oldest [[Effect]] or [[akka.actor.testkit.typed.scaladsl.Effects.NoEffects]] if no effects
@ -64,7 +64,7 @@ trait BehaviorTestKit[T] {
def childTestKit[U](child: ActorRef[U]): BehaviorTestKit[U]
/**
* The self inbox contains messages the behavior sent to `ctx.self`
* The self inbox contains messages the behavior sent to `context.self`
*/
def selfInbox(): TestInbox[T]
@ -119,9 +119,9 @@ trait BehaviorTestKit[T] {
def isAlive: Boolean
/**
* Send the msg to the behavior and record any [[Effect]]s
* Send the message to the behavior and record any [[Effect]]s
*/
def run(msg: T): Unit
def run(message: T): Unit
/**
* Send the first message in the selfInbox to the behavior and run it, recording [[Effect]]s.

View file

@ -52,26 +52,26 @@ object Effects {
*/
def stopped(childName: String): Stopped = Stopped(childName)
/**
* The behavior started watching `other`, through `ctx.watch(other)`
* The behavior started watching `other`, through `context.watch(other)`
*/
def watched[T](other: ActorRef[T]): Watched[T] = Watched(other)
/**
* The behavior started watching `other`, through `ctx.unwatch(other)`
* The behavior started watching `other`, through `context.unwatch(other)`
*/
def unwatched[T](other: ActorRef[T]): Unwatched[T] = Unwatched(other)
/**
* The behavior set a new receive timeout, with `msg` as timeout notification
* The behavior set a new receive timeout, with `message` as timeout notification
*/
def receiveTimeoutSet[T](d: FiniteDuration, msg: T): ReceiveTimeoutSet[T] = ReceiveTimeoutSet(d, msg)
def receiveTimeoutSet[T](d: FiniteDuration, message: T): ReceiveTimeoutSet[T] = ReceiveTimeoutSet(d, message)
/**
* The behavior used `ctx.schedule` to schedule `msg` to be sent to `target` after `delay`
* The behavior used `context.schedule` to schedule `message` to be sent to `target` after `delay`
* FIXME what about events scheduled through the scheduler?
*/
def scheduled[U](delay: FiniteDuration, target: ActorRef[U], msg: U): Scheduled[U] =
Scheduled(delay, target, msg)
def scheduled[U](delay: FiniteDuration, target: ActorRef[U], message: U): Scheduled[U] =
Scheduled(delay, target, message)
/**
* Used to represent an empty list of effects - in other words, the behavior didn't do anything observable

View file

@ -29,7 +29,7 @@ object FishingOutcomes {
/**
* Fail fishing with a custom error message
*/
def fail(msg: String): FishingOutcome = FishingOutcome.Fail(msg)
def fail(message: String): FishingOutcome = FishingOutcome.Fail(message)
}
object TestProbe {

View file

@ -51,7 +51,7 @@ public class ActorTestKitTest extends JUnitSuite {
@Test
public void testKitShouldSpawnActor() throws Exception {
final CompletableFuture<Done> started = new CompletableFuture<>();
testKit.spawn(Behaviors.setup((ctx) -> {
testKit.spawn(Behaviors.setup((context) -> {
started.complete(done());
return Behaviors.same();
}));

View file

@ -119,56 +119,56 @@ public class BehaviorTestKitTest extends JUnitSuite {
private static Props props = Props.empty().withDispatcherFromConfig("cat");
private static Behavior<Command> behavior = Behaviors.receive(Command.class)
.onMessage(SpawnChildren.class, (ctx, msg) -> {
IntStream.range(0, msg.numberOfChildren).forEach(i -> {
ctx.spawn(childInitial, "child" + i);
.onMessage(SpawnChildren.class, (context, message) -> {
IntStream.range(0, message.numberOfChildren).forEach(i -> {
context.spawn(childInitial, "child" + i);
});
return Behaviors.same();
})
.onMessage(SpawnChildrenAnonymous.class, (ctx, msg) -> {
IntStream.range(0, msg.numberOfChildren).forEach(i -> {
ctx.spawnAnonymous(childInitial);
.onMessage(SpawnChildrenAnonymous.class, (context, message) -> {
IntStream.range(0, message.numberOfChildren).forEach(i -> {
context.spawnAnonymous(childInitial);
});
return Behaviors.same();
})
.onMessage(SpawnChildrenWithProps.class, (ctx, msg) -> {
IntStream.range(0, msg.numberOfChildren).forEach(i -> {
ctx.spawn(childInitial, "child" + i, msg.props);
.onMessage(SpawnChildrenWithProps.class, (context, message) -> {
IntStream.range(0, message.numberOfChildren).forEach(i -> {
context.spawn(childInitial, "child" + i, message.props);
});
return Behaviors.same();
})
.onMessage(SpawnChildrenAnonymousWithProps.class, (ctx, msg) -> {
IntStream.range(0, msg.numberOfChildren).forEach(i -> {
ctx.spawnAnonymous(childInitial, msg.props);
.onMessage(SpawnChildrenAnonymousWithProps.class, (context, message) -> {
IntStream.range(0, message.numberOfChildren).forEach(i -> {
context.spawnAnonymous(childInitial, message.props);
});
return Behaviors.same();
})
.onMessage(CreateMessageAdapter.class, (ctx, msg) -> {
ctx.messageAdapter(msg.clazz, msg.f);
.onMessage(CreateMessageAdapter.class, (context, message) -> {
context.messageAdapter(message.clazz, message.f);
return Behaviors.same();
})
.onMessage(SpawnWatchAndUnWatch.class, (ctx, msg) -> {
ActorRef<Action> c = ctx.spawn(childInitial, msg.name);
ctx.watch(c);
ctx.unwatch(c);
.onMessage(SpawnWatchAndUnWatch.class, (context, message) -> {
ActorRef<Action> c = context.spawn(childInitial, message.name);
context.watch(c);
context.unwatch(c);
return Behaviors.same();
})
.onMessage(SpawnAndWatchWith.class, (ctx, msg) -> {
ActorRef<Action> c = ctx.spawn(childInitial, msg.name);
ctx.watchWith(c, msg);
.onMessage(SpawnAndWatchWith.class, (context, message) -> {
ActorRef<Action> c = context.spawn(childInitial, message.name);
context.watchWith(c, message);
return Behaviors.same();
})
.onMessage(SpawnSession.class, (ctx, msg) -> {
ActorRef<String> session = ctx.spawnAnonymous(Behaviors.receiveMessage( m -> {
msg.sessionHandler.tell(m);
.onMessage(SpawnSession.class, (context, message) -> {
ActorRef<String> session = context.spawnAnonymous(Behaviors.receiveMessage( m -> {
message.sessionHandler.tell(m);
return Behaviors.same();
}));
msg.replyTo.tell(session);
message.replyTo.tell(session);
return Behaviors.same();
})
.onMessage(KillSession.class, (ctx, msg) -> {
ctx.stop(msg.session);
msg.replyTo.tell(Done.getInstance());
.onMessage(KillSession.class, (context, message) -> {
context.stop(message.session);
message.replyTo.tell(Done.getInstance());
return Behaviors.same();
})
.build();

View file

@ -36,9 +36,9 @@ public class TestProbeTest {
probe.expectTerminated(ref, Duration.ofSeconds(1));
Duration remaining = probe.getRemaining();
probe.fishForMessage(Duration.ofSeconds(3), "hint", (msg) -> {
if (msg.equals("one")) return FishingOutcomes.continueAndIgnore();
else if (msg.equals("two")) return FishingOutcomes.complete();
probe.fishForMessage(Duration.ofSeconds(3), "hint", (message) -> {
if (message.equals("one")) return FishingOutcomes.continueAndIgnore();
else if (message.equals("two")) return FishingOutcomes.complete();
else return FishingOutcomes.fail("error");
});

View file

@ -21,24 +21,24 @@ public class AsyncTestingExampleTest {
//#under-test
public static class Ping {
private String msg;
private String message;
private ActorRef<Pong> replyTo;
public Ping(String msg, ActorRef<Pong> replyTo) {
this.msg = msg;
public Ping(String message, ActorRef<Pong> replyTo) {
this.message = message;
this.replyTo = replyTo;
}
}
public static class Pong {
private String msg;
private String message;
public Pong(String msg) {
this.msg = msg;
public Pong(String message) {
this.message = message;
}
}
Behavior<Ping> echoActor = Behaviors.receive((ctx, ping) -> {
ping.replyTo.tell(new Pong(ping.msg));
Behavior<Ping> echoActor = Behaviors.receive((context, ping) -> {
ping.replyTo.tell(new Pong(ping.message));
return Behaviors.same();
});
//#under-test

View file

@ -34,7 +34,7 @@ public class ManualTimerExampleTest extends JUnitSuite {
TestProbe<Tock> probe = testKit.createTestProbe();
Behavior<Tick> behavior = Behaviors.withTimers(timer -> {
timer.startSingleTimer("T", new Tick(), Duration.ofMillis(10));
return Behaviors.receive( (ctx, tick) -> {
return Behaviors.receive( (context, tick) -> {
probe.ref().tell(new Tock());
return Behaviors.same();
});

View file

@ -17,7 +17,7 @@ import org.scalatest.junit.JUnitSuite;
public class SyncTestingExampleTest extends JUnitSuite {
//#child
public static Behavior<String> childActor = Behaviors.receive((ctx, msg) -> Behaviors.same());
public static Behavior<String> childActor = Behaviors.receive((context, message) -> Behaviors.same());
//#child
//#under-test
@ -45,25 +45,25 @@ public class SyncTestingExampleTest extends JUnitSuite {
}
public static Behavior<Command> myBehavior = Behaviors.receive(Command.class)
.onMessage(CreateAChild.class, (ctx, msg) -> {
ctx.spawn(childActor, msg.childName);
.onMessage(CreateAChild.class, (context, message) -> {
context.spawn(childActor, message.childName);
return Behaviors.same();
})
.onMessage(CreateAnAnonymousChild.class, (ctx, msg) -> {
ctx.spawnAnonymous(childActor);
.onMessage(CreateAnAnonymousChild.class, (context, message) -> {
context.spawnAnonymous(childActor);
return Behaviors.same();
})
.onMessage(SayHelloToChild.class, (ctx, msg) -> {
ActorRef<String> child = ctx.spawn(childActor, msg.childName);
.onMessage(SayHelloToChild.class, (context, message) -> {
ActorRef<String> child = context.spawn(childActor, message.childName);
child.tell("hello");
return Behaviors.same();
})
.onMessage(SayHelloToAnonymousChild.class, (ctx, msg) -> {
ActorRef<String> child = ctx.spawnAnonymous(childActor);
.onMessage(SayHelloToAnonymousChild.class, (context, message) -> {
ActorRef<String> child = context.spawnAnonymous(childActor);
child.tell("hello stranger");
return Behaviors.same();
}).onMessage(SayHello.class, (ctx, msg) -> {
msg.who.tell("hello");
}).onMessage(SayHello.class, (context, message) -> {
message.who.tell("hello");
return Behaviors.same();
}).build();
//#under-test

View file

@ -40,7 +40,7 @@ class ActorTestKitSpec extends ScalaTestWithActorTestKit with WordSpecLike {
"spawn an actor" in {
val sawMessage = Promise[Boolean]()
val ref = spawn(Behaviors.setup[AnyRef] { ctx
val ref = spawn(Behaviors.setup[AnyRef] { context
sawMessage.trySuccess(true)
Behaviors.empty
})
@ -50,8 +50,8 @@ class ActorTestKitSpec extends ScalaTestWithActorTestKit with WordSpecLike {
"spawn a named actor" in {
val spawnedWithName = Promise[String]()
val ref = spawn(Behaviors.setup[AnyRef] { ctx
spawnedWithName.trySuccess(ctx.self.path.name)
val ref = spawn(Behaviors.setup[AnyRef] { context
spawnedWithName.trySuccess(context.self.path.name)
Behaviors.empty
}, "name")

View file

@ -36,66 +36,66 @@ object BehaviorTestKitSpec {
case class SpawnSession(replyTo: ActorRef[ActorRef[String]], sessionHandler: ActorRef[String]) extends Command
case class KillSession(session: ActorRef[String], replyTo: ActorRef[Done]) extends Command
val init: Behavior[Command] = Behaviors.receive[Command] { (ctx, msg)
msg match {
val init: Behavior[Command] = Behaviors.receive[Command] { (context, message)
message match {
case SpawnChild
ctx.spawn(Child.initial, "child")
context.spawn(Child.initial, "child")
Behaviors.same
case SpawnChildren(numberOfChildren) if numberOfChildren > 0
0.until(numberOfChildren).foreach { i
ctx.spawn(Child.initial, s"child$i")
context.spawn(Child.initial, s"child$i")
}
Behaviors.same
case SpawnChildrenWithProps(numberOfChildren, props) if numberOfChildren > 0
0.until(numberOfChildren).foreach { i
ctx.spawn(Child.initial, s"child$i", props)
context.spawn(Child.initial, s"child$i", props)
}
Behaviors.same
case SpawnAnonymous(numberOfChildren) if numberOfChildren > 0
0.until(numberOfChildren).foreach { _
ctx.spawnAnonymous(Child.initial)
context.spawnAnonymous(Child.initial)
}
Behaviors.same
case SpawnAnonymousWithProps(numberOfChildren, props) if numberOfChildren > 0
0.until(numberOfChildren).foreach { _
ctx.spawnAnonymous(Child.initial, props)
context.spawnAnonymous(Child.initial, props)
}
Behaviors.same
case StopChild(child)
ctx.stop(child)
context.stop(child)
Behaviors.same
case SpawnAdapter
ctx.spawnMessageAdapter {
context.spawnMessageAdapter {
r: Reproduce SpawnAnonymous(r.times)
}
Behaviors.same
case SpawnAdapterWithName(name)
ctx.spawnMessageAdapter({
context.spawnMessageAdapter({
r: Reproduce SpawnAnonymous(r.times)
}, name)
Behaviors.same
case SpawnAndWatchUnwatch(name)
val c = ctx.spawn(Child.initial, name)
ctx.watch(c)
ctx.unwatch(c)
val c = context.spawn(Child.initial, name)
context.watch(c)
context.unwatch(c)
Behaviors.same
case m @ SpawnAndWatchWith(name)
val c = ctx.spawn(Child.initial, name)
ctx.watchWith(c, m)
val c = context.spawn(Child.initial, name)
context.watchWith(c, m)
Behaviors.same
case SpawnSession(replyTo, sessionHandler)
val session = ctx.spawnAnonymous[String](Behaviors.receiveMessage { msg
sessionHandler ! msg
val session = context.spawnAnonymous[String](Behaviors.receiveMessage { message
sessionHandler ! message
Behavior.same
})
replyTo ! session
Behaviors.same
case KillSession(session, replyTo)
ctx.stop(session)
context.stop(session)
replyTo ! Done
Behaviors.same
case CreateMessageAdapter(messageClass, f)
ctx.messageAdapter(f)(ClassTag(messageClass))
context.messageAdapter(f)(ClassTag(messageClass))
Behaviors.same
}
@ -106,8 +106,8 @@ object BehaviorTestKitSpec {
sealed trait Action
val initial: Behavior[Action] = Behaviors.receive[Action] { (_, msg)
msg match {
val initial: Behavior[Action] = Behaviors.receive[Action] { (_, message)
message match {
case _
Behaviors.empty
}

View file

@ -40,11 +40,11 @@ class TestProbeSpec extends ScalaTestWithActorTestKit with WordSpecLike {
"allow probing for actor stop when actor has not stopped yet" in {
case object Stop
val probe = TestProbe()
val ref = spawn(Behaviors.receive[Stop.type]((ctx, message)
val ref = spawn(Behaviors.receive[Stop.type]((context, message)
Behaviors.withTimers { (timer)
timer.startSingleTimer("key", Stop, 300.millis)
Behaviors.receive((ctx, stop)
Behaviors.receive((context, stop)
Behaviors.stopped
)
}

View file

@ -12,11 +12,11 @@ import org.scalatest.WordSpec
object AsyncTestingExampleSpec {
//#under-test
case class Ping(msg: String, response: ActorRef[Pong])
case class Pong(msg: String)
case class Ping(message: String, response: ActorRef[Pong])
case class Pong(message: String)
val echoActor: Behavior[Ping] = Behaviors.receive { (_, msg)
msg match {
val echoActor: Behavior[Ping] = Behaviors.receive { (_, message)
message match {
case Ping(m, replyTo)
replyTo ! Pong(m)
Behaviors.same

View file

@ -48,7 +48,7 @@ class ManualTimerExampleSpec extends ScalaTestWithActorTestKit(ManualTime.config
val probe = TestProbe[Tock.type]()
val behavior = Behaviors.withTimers[Tick.type] { timer
timer.startPeriodicTimer("T", Tick, 10.millis)
Behaviors.receive { (ctx, Tick)
Behaviors.receive { (context, Tick)
probe.ref ! Tock
Behaviors.same
}
@ -77,7 +77,7 @@ class ManualTimerExampleSpec extends ScalaTestWithActorTestKit(ManualTime.config
val behavior = Behaviors.withTimers[Command] { timer
timer.startPeriodicTimer("T", Tick(1), interval)
Behaviors.receive { (ctx, cmd)
Behaviors.receive { (context, cmd)
cmd match {
case Tick(n)
probe.ref ! Tock(n)

View file

@ -30,18 +30,18 @@ object SyncTestingExampleSpec {
case class SayHello(who: ActorRef[String]) extends Cmd
val myBehavior = Behaviors.receivePartial[Cmd] {
case (ctx, CreateChild(name))
ctx.spawn(childActor, name)
case (context, CreateChild(name))
context.spawn(childActor, name)
Behaviors.same
case (ctx, CreateAnonymousChild)
ctx.spawnAnonymous(childActor)
case (context, CreateAnonymousChild)
context.spawnAnonymous(childActor)
Behaviors.same
case (ctx, SayHelloToChild(childName))
val child: ActorRef[String] = ctx.spawn(childActor, childName)
case (context, SayHelloToChild(childName))
val child: ActorRef[String] = context.spawn(childActor, childName)
child ! "hello"
Behaviors.same
case (ctx, SayHelloToAnonymousChild)
val child: ActorRef[String] = ctx.spawnAnonymous(childActor)
case (context, SayHelloToAnonymousChild)
val child: ActorRef[String] = context.spawnAnonymous(childActor)
child ! "hello stranger"
Behaviors.same
case (_, SayHello(who))

View file

@ -33,37 +33,37 @@ public class ActorCompile {
}
}
Behavior<MyMsg> actor1 = Behaviors.receive((ctx, msg) -> stopped(), (ctx, signal) -> same());
Behavior<MyMsg> actor2 = Behaviors.receive((ctx, msg) -> unhandled());
Behavior<MyMsg> actor1 = Behaviors.receive((context, message) -> stopped(), (context, signal) -> same());
Behavior<MyMsg> actor2 = Behaviors.receive((context, message) -> unhandled());
Behavior<MyMsg> actor4 = empty();
Behavior<MyMsg> actor5 = ignore();
Behavior<MyMsg> actor6 = intercept(new BehaviorInterceptor<MyMsg, MyMsg>() {
@Override
public Behavior<MyMsg> aroundReceive(ActorContext<MyMsg> ctx, MyMsg msg, ReceiveTarget<MyMsg> target) {
return target.apply(ctx, msg);
public Behavior<MyMsg> aroundReceive(ActorContext<MyMsg> context, MyMsg message, ReceiveTarget<MyMsg> target) {
return target.apply(context, message);
}
@Override
public Behavior<MyMsg> aroundSignal(ActorContext<MyMsg> ctx, Signal signal, SignalTarget<MyMsg> target) {
return target.apply(ctx, signal);
public Behavior<MyMsg> aroundSignal(ActorContext<MyMsg> context, Signal signal, SignalTarget<MyMsg> target) {
return target.apply(context, signal);
}
}, actor5);
Behavior<MyMsgA> actor7 = actor6.narrow();
Behavior<MyMsg> actor8 = setup(ctx -> {
final ActorRef<MyMsg> self = ctx.getSelf();
Behavior<MyMsg> actor8 = setup(context -> {
final ActorRef<MyMsg> self = context.getSelf();
return monitor(self, ignore());
});
Behavior<MyMsg> actor9 = widened(actor7, pf -> pf.match(MyMsgA.class, x -> x));
Behavior<MyMsg> actor10 = Behaviors.receive((ctx, msg) -> stopped(actor4), (ctx, signal) -> same());
Behavior<MyMsg> actor10 = Behaviors.receive((context, message) -> stopped(actor4), (context, signal) -> same());
ActorSystem<MyMsg> system = ActorSystem.create(actor1, "Sys");
{
Behaviors.<MyMsg>receive((ctx, msg) -> {
if (msg instanceof MyMsgA) {
Behaviors.<MyMsg>receive((context, message) -> {
if (message instanceof MyMsgA) {
return Behaviors.receive((ctx2, msg2) -> {
if (msg2 instanceof MyMsgB) {
((MyMsgA) msg).replyTo.tell(((MyMsgB) msg2).greeting);
((MyMsgA) message).replyTo.tell(((MyMsgB) msg2).greeting);
ActorRef<String> adapter = ctx2.messageAdapter(String.class, s -> new MyMsgB(s.toUpperCase()));
}
@ -84,13 +84,13 @@ public class ActorCompile {
static class MyBehavior extends ExtensibleBehavior<MyMsg> {
@Override
public Behavior<MyMsg> receiveSignal(ActorContext<MyMsg> ctx, Signal msg) throws Exception {
public Behavior<MyMsg> receiveSignal(ActorContext<MyMsg> context, Signal message) throws Exception {
return this;
}
@Override
public Behavior<MyMsg> receive(ActorContext<MyMsg> ctx, MyMsg msg) throws Exception {
ActorRef<String> adapter = ctx.asJava().messageAdapter(String.class, s -> new MyMsgB(s.toUpperCase()));
public Behavior<MyMsg> receive(ActorContext<MyMsg> context, MyMsg message) throws Exception {
ActorRef<String> adapter = context.asJava().messageAdapter(String.class, s -> new MyMsgB(s.toUpperCase()));
return this;
}

View file

@ -39,8 +39,8 @@ public class ActorContextAskTest extends JUnitSuite {
final ActorRef<Ping> pingPong = testKit.spawn(pingPongBehavior);
final TestProbe<Object> probe = testKit.createTestProbe();
final Behavior<Object> snitch = Behaviors.setup((ActorContext<Object> ctx) -> {
ctx.ask(Pong.class,
final Behavior<Object> snitch = Behaviors.setup((ActorContext<Object> context) -> {
context.ask(Pong.class,
pingPong,
new Timeout(3, TimeUnit.SECONDS),
(ActorRef<Pong> ref) -> new Ping(ref),
@ -49,7 +49,7 @@ public class ActorContextAskTest extends JUnitSuite {
else return exception;
});
return Behaviors.receive((ActorContext<Object> context, Object message) -> {
return Behaviors.receiveMessage((Object message) -> {
probe.ref().tell(message);
return Behaviors.same();
});

View file

@ -46,14 +46,14 @@ public class ActorLoggingTest extends JUnitSuite {
public void loggingProvidesMDC() {
Behavior<Protocol> behavior = Behaviors.withMdc(
null,
(msg) -> {
(message) -> {
Map<String, Object> mdc = new HashMap<>();
mdc.put("txId", msg.getTransactionId());
mdc.put("txId", message.getTransactionId());
return mdc;
},
Behaviors.receive(Protocol.class)
.onMessage(Message.class, (ctx, msg) -> {
ctx.getLog().info(msg.toString());
.onMessage(Message.class, (context, message) -> {
context.getLog().info(message.toString());
return Behaviors.same();
}).build()
);

View file

@ -53,42 +53,42 @@ public class AdapterTest extends JUnitSuite {
static Behavior<String> create(akka.actor.ActorRef ref, akka.actor.ActorRef probe) {
Typed1 logic = new Typed1(ref, probe);
return receive(
(ctx, msg) -> logic.onMessage(ctx, msg),
(ctx, sig) -> logic.onSignal(ctx, sig));
(context, message) -> logic.onMessage(context, message),
(context, sig) -> logic.onSignal(context, sig));
}
Behavior<String> onMessage(ActorContext<String> ctx, String msg) {
if (msg.equals("send")) {
akka.actor.ActorRef replyTo = Adapter.toUntyped(ctx.getSelf());
Behavior<String> onMessage(ActorContext<String> context, String message) {
if (message.equals("send")) {
akka.actor.ActorRef replyTo = Adapter.toUntyped(context.getSelf());
ref.tell("ping", replyTo);
return same();
} else if (msg.equals("pong")) {
} else if (message.equals("pong")) {
probe.tell("ok", akka.actor.ActorRef.noSender());
return same();
} else if (msg.equals("actorOf")) {
akka.actor.ActorRef child = Adapter.actorOf(ctx, untyped1());
child.tell("ping", Adapter.toUntyped(ctx.getSelf()));
} else if (message.equals("actorOf")) {
akka.actor.ActorRef child = Adapter.actorOf(context, untyped1());
child.tell("ping", Adapter.toUntyped(context.getSelf()));
return same();
} else if (msg.equals("watch")) {
Adapter.watch(ctx, ref);
} else if (message.equals("watch")) {
Adapter.watch(context, ref);
return same();
} else if (msg.equals("supervise-stop")) {
akka.actor.ActorRef child = Adapter.actorOf(ctx, untyped1());
Adapter.watch(ctx, child);
child.tell(new ThrowIt3(), Adapter.toUntyped(ctx.getSelf()));
child.tell("ping", Adapter.toUntyped(ctx.getSelf()));
} else if (message.equals("supervise-stop")) {
akka.actor.ActorRef child = Adapter.actorOf(context, untyped1());
Adapter.watch(context, child);
child.tell(new ThrowIt3(), Adapter.toUntyped(context.getSelf()));
child.tell("ping", Adapter.toUntyped(context.getSelf()));
return same();
} else if (msg.equals("stop-child")) {
akka.actor.ActorRef child = Adapter.actorOf(ctx, untyped1());
Adapter.watch(ctx, child);
Adapter.stop(ctx, child);
} else if (message.equals("stop-child")) {
akka.actor.ActorRef child = Adapter.actorOf(context, untyped1());
Adapter.watch(context, child);
Adapter.stop(context, child);
return same();
} else {
return unhandled();
}
}
Behavior<String> onSignal(ActorContext<String> ctx, Signal sig) {
Behavior<String> onSignal(ActorContext<String> context, Signal sig) {
if (sig instanceof Terminated) {
probe.tell("terminated", akka.actor.ActorRef.noSender());
return same();
@ -189,15 +189,15 @@ public class AdapterTest extends JUnitSuite {
}
static Behavior<Typed2Msg> typed2() {
return Behaviors.receive((ctx, msg) -> {
if (msg instanceof Ping) {
ActorRef<String> replyTo = ((Ping) msg).replyTo;
return Behaviors.receive((context, message) -> {
if (message instanceof Ping) {
ActorRef<String> replyTo = ((Ping) message).replyTo;
replyTo.tell("pong");
return same();
} else if (msg instanceof StopIt) {
} else if (message instanceof StopIt) {
return stopped();
} else if (msg instanceof ThrowIt) {
throw (ThrowIt) msg;
} else if (message instanceof ThrowIt) {
throw (ThrowIt) message;
} else {
return unhandled();
}

View file

@ -34,16 +34,16 @@ public class BehaviorBuilderTest extends JUnitSuite {
@Test
public void shouldCompile() {
Behavior<Message> b = Behaviors.receive(Message.class)
.onMessage(One.class, (ctx, o) -> {
.onMessage(One.class, (context, o) -> {
o.foo();
return same();
})
.onMessage(One.class, o -> o.foo().startsWith("a"), (ctx, o) -> same())
.onMessageUnchecked(MyList.class, (ActorContext<Message> ctx, MyList<String> l) -> {
.onMessage(One.class, o -> o.foo().startsWith("a"), (context, o) -> same())
.onMessageUnchecked(MyList.class, (ActorContext<Message> context, MyList<String> l) -> {
String first = l.get(0);
return Behaviors.<Message>same();
})
.onSignal(Terminated.class, (ctx, t) -> {
.onSignal(Terminated.class, (context, t) -> {
System.out.println("Terminating along with " + t.getRef());
return stopped();
})
@ -67,10 +67,10 @@ public class BehaviorBuilderTest extends JUnitSuite {
public Behavior<CounterMessage> immutableCounter(int currentValue) {
return Behaviors.receive(CounterMessage.class)
.onMessage(Increase.class, (ctx, o) -> {
.onMessage(Increase.class, (context, o) -> {
return immutableCounter(currentValue + 1);
})
.onMessage(Get.class, (ctx, o) -> {
.onMessage(Get.class, (context, o) -> {
o.sender.tell(new Got(currentValue));
return same();
})

View file

@ -18,10 +18,10 @@ public class ReceiveBuilderTest extends JUnitSuite {
@Test
public void testMutableCounter() {
Behavior<BehaviorBuilderTest.CounterMessage> mutable = Behaviors.setup(ctx -> new AbstractBehavior<BehaviorBuilderTest.CounterMessage>() {
Behavior<BehaviorBuilderTest.CounterMessage> mutable = Behaviors.setup(context -> new AbstractBehavior<BehaviorBuilderTest.CounterMessage>() {
int currentValue = 0;
private Behavior<BehaviorBuilderTest.CounterMessage> receiveIncrease(BehaviorBuilderTest.Increase msg) {
private Behavior<BehaviorBuilderTest.CounterMessage> receiveIncrease(BehaviorBuilderTest.Increase message) {
currentValue++;
return this;
}

View file

@ -44,15 +44,15 @@ public class WatchTest extends JUnitSuite {
final Timeout timeout = Timeout.create(Duration.ofSeconds(5));
final Behavior<Stop> exitingActor = receive((ctx, msg) -> {
final Behavior<Stop> exitingActor = receive((context, message) -> {
System.out.println("Stopping!");
return stopped();
});
private Behavior<RunTest> waitingForTermination(ActorRef<Done> replyWhenTerminated) {
return receive(
(ctx, msg) -> unhandled(),
(ctx, sig) -> {
(context, message) -> unhandled(),
(context, sig) -> {
if (sig instanceof Terminated) {
replyWhenTerminated.tell(done());
}
@ -63,8 +63,8 @@ public class WatchTest extends JUnitSuite {
private Behavior<Message> waitingForMessage(ActorRef<Done> replyWhenReceived) {
return receive(
(ctx, msg) -> {
if (msg instanceof CustomTerminationMessage) {
(context, message) -> {
if (message instanceof CustomTerminationMessage) {
replyWhenReceived.tell(done());
return same();
} else {
@ -77,11 +77,11 @@ public class WatchTest extends JUnitSuite {
@Test
public void shouldWatchTerminatingActor() throws Exception {
Behavior<RunTest> exiting = Behaviors.receive(RunTest.class)
.onMessage(RunTest.class, (ctx, msg) -> {
ActorRef<Stop> watched = ctx.spawn(exitingActor, "exitingActor");
ctx.watch(watched);
.onMessage(RunTest.class, (context, message) -> {
ActorRef<Stop> watched = context.spawn(exitingActor, "exitingActor");
context.watch(watched);
watched.tell(new Stop());
return waitingForTermination(msg.replyTo);
return waitingForTermination(message.replyTo);
}).build();
ActorRef<RunTest> exitingRef = testKit.spawn(exiting);
@ -92,11 +92,11 @@ public class WatchTest extends JUnitSuite {
@Test
public void shouldWatchWithCustomMessage() throws Exception {
Behavior<Message> exiting = Behaviors.receive(Message.class)
.onMessage(RunTest.class, (ctx, msg) -> {
ActorRef<Stop> watched = ctx.spawn(exitingActor, "exitingActor");
ctx.watchWith(watched, new CustomTerminationMessage());
.onMessage(RunTest.class, (context, message) -> {
ActorRef<Stop> watched = context.spawn(exitingActor, "exitingActor");
context.watchWith(watched, new CustomTerminationMessage());
watched.tell(new Stop());
return waitingForMessage(msg.replyTo);
return waitingForMessage(message.replyTo);
}).build();
ActorRef<Message> exitingRef = testKit.spawn(exiting);

View file

@ -56,12 +56,12 @@ public class ReceptionistApiTest {
}
});
Behaviors.setup(ctx -> {
Behaviors.setup(context -> {
// oneoff ask inside of actor
// this is somewhat verbose, however this should be a rare use case
ctx.ask(
context.ask(
Receptionist.Listing.class,
ctx.getSystem().receptionist(),
context.getSystem().receptionist(),
timeout,
resRef -> Receptionist.find(key, resRef),
(listing, throwable) -> {
@ -71,18 +71,18 @@ public class ReceptionistApiTest {
);
// this is a more "normal" use case which is clean
ctx.getSystem().receptionist().tell(Receptionist.subscribe(key, ctx.getSelf().narrow()));
context.getSystem().receptionist().tell(Receptionist.subscribe(key, context.getSelf().narrow()));
// another more "normal" is subscribe using an adapter
ActorRef<Receptionist.Listing> listingAdapter = ctx.messageAdapter(
ActorRef<Receptionist.Listing> listingAdapter = context.messageAdapter(
Receptionist.Listing.class,
(listing) -> listing.serviceInstances(key)
);
ctx.getSystem().receptionist().tell(Receptionist.subscribe(key, listingAdapter));
context.getSystem().receptionist().tell(Receptionist.subscribe(key, listingAdapter));
// ofc this doesn't make sense to do in the same actor, this is just
// to cover as much of the API as possible
ctx.getSystem().receptionist().tell(Receptionist.register(key, ctx.getSelf().narrow(), ctx.getSelf().narrow()));
context.getSystem().receptionist().tell(Receptionist.register(key, context.getSelf().narrow(), context.getSelf().narrow()));
return Behaviors.receive(Object.class)
// matching is done best using the predicate version

View file

@ -13,13 +13,13 @@ public class DispatchersDocTest {
private static Behavior<String> yourBehavior = Behaviors.empty();
private static Behavior<Object> example = Behaviors.receive((ctx, msg) -> {
private static Behavior<Object> example = Behaviors.receive((context, message) -> {
//#spawn-dispatcher
ctx.spawn(yourBehavior, "DefaultDispatcher");
ctx.spawn(yourBehavior, "ExplicitDefaultDispatcher", DispatcherSelector.defaultDispatcher());
ctx.spawn(yourBehavior, "BlockingDispatcher", DispatcherSelector.blocking());
ctx.spawn(yourBehavior, "DispatcherFromConfig", DispatcherSelector.fromConfig("your-dispatcher"));
context.spawn(yourBehavior, "DefaultDispatcher");
context.spawn(yourBehavior, "ExplicitDefaultDispatcher", DispatcherSelector.defaultDispatcher());
context.spawn(yourBehavior, "BlockingDispatcher", DispatcherSelector.blocking());
context.spawn(yourBehavior, "DispatcherFromConfig", DispatcherSelector.fromConfig("your-dispatcher"));
//#spawn-dispatcher
return Behaviors.same();

View file

@ -158,13 +158,13 @@ public class FSMDocTest {
// FSM states represented as behaviors
private static Behavior<Event> uninitialized() {
return Behaviors.receive(Event.class)
.onMessage(SetTarget.class, (ctx, msg) -> idle(new Todo(msg.getRef(), Collections.emptyList())))
.onMessage(SetTarget.class, (context, message) -> idle(new Todo(message.getRef(), Collections.emptyList())))
.build();
}
private static Behavior<Event> idle(Todo data) {
return Behaviors.receive(Event.class)
.onMessage(Queue.class, (ctx, msg) -> active(data.addElement(msg)))
.onMessage(Queue.class, (context, message) -> active(data.addElement(message)))
.build();
}
@ -173,12 +173,12 @@ public class FSMDocTest {
// State timeouts done with withTimers
timers.startSingleTimer("Timeout", TIMEOUT, Duration.ofSeconds(1));
return Behaviors.receive(Event.class)
.onMessage(Queue.class, (ctx, msg) -> active(data.addElement(msg)))
.onMessage(Flush.class, (ctx, msg) -> {
.onMessage(Queue.class, (context, message) -> active(data.addElement(message)))
.onMessage(Flush.class, (context, message) -> {
data.getTarget().tell(new Batch(data.queue));
return idle(data.copy(new ArrayList<>()));
})
.onMessage(Timeout.class, (ctx, msg) -> {
.onMessage(Timeout.class, (context, message) -> {
data.getTarget().tell(new Batch(data.queue));
return idle(data.copy(new ArrayList<>()));
}).build();

View file

@ -30,41 +30,41 @@ public class FaultToleranceDocTest extends JUnitSuite {
public void bubblingSample() {
// #bubbling-example
final Behavior<Message> failingChildBehavior = Behaviors.receive(Message.class)
.onMessage(Fail.class, (ctx, message) -> {
.onMessage(Fail.class, (context, message) -> {
throw new RuntimeException(message.text);
})
.build();
Behavior<Message> middleManagementBehavior = Behaviors.setup((ctx) -> {
ctx.getLog().info("Middle management starting up");
final ActorRef<Message> child = ctx.spawn(failingChildBehavior, "child");
Behavior<Message> middleManagementBehavior = Behaviors.setup((context) -> {
context.getLog().info("Middle management starting up");
final ActorRef<Message> child = context.spawn(failingChildBehavior, "child");
// we want to know when the child terminates, but since we do not handle
// the Terminated signal, we will in turn fail on child termination
ctx.watch(child);
context.watch(child);
// here we don't handle Terminated at all which means that
// when the child fails or stops gracefully this actor will
// fail with a DeathWatchException
return Behaviors.receive(Message.class)
.onMessage(Message.class, (innerCtx, msg) -> {
.onMessage(Message.class, (innerCtx, message) -> {
// just pass messages on to the child
child.tell(msg);
child.tell(message);
return Behaviors.same();
}).build();
});
Behavior<Message> bossBehavior = Behaviors.setup((ctx) -> {
ctx.getLog().info("Boss starting up");
final ActorRef<Message> middleManagement = ctx.spawn(middleManagementBehavior, "middle-management");
ctx.watch(middleManagement);
Behavior<Message> bossBehavior = Behaviors.setup((context) -> {
context.getLog().info("Boss starting up");
final ActorRef<Message> middleManagement = context.spawn(middleManagementBehavior, "middle-management");
context.watch(middleManagement);
// here we don't handle Terminated at all which means that
// when middle management fails with a DeathWatchException
// this actor will also fail
return Behaviors.receive(Message.class)
.onMessage(Message.class, (innerCtx, msg) -> {
.onMessage(Message.class, (innerCtx, message) -> {
// just pass messages on to the child
middleManagement.tell(msg);
middleManagement.tell(message);
return Behaviors.same();
}).build();
});

View file

@ -43,27 +43,27 @@ public class GracefulStopDocTest {
}
public static final Behavior<JobControlLanguage> mcpa = Behaviors.receive(JobControlLanguage.class)
.onMessage(SpawnJob.class, (ctx, msg) -> {
ctx.getSystem().log().info("Spawning job {}!", msg.name);
ctx.spawn(Job.job(msg.name), msg.name);
.onMessage(SpawnJob.class, (context, message) -> {
context.getSystem().log().info("Spawning job {}!", message.name);
context.spawn(Job.job(message.name), message.name);
return Behaviors.same();
})
.onSignal(PostStop.class, (ctx, signal) -> {
ctx.getSystem().log().info("Master Control Programme stopped");
.onSignal(PostStop.class, (context, signal) -> {
context.getSystem().log().info("Master Control Programme stopped");
return Behaviors.same();
})
.onMessage(GracefulShutdown.class, (ctx, msg) -> {
ctx.getSystem().log().info("Initiating graceful shutdown...");
.onMessage(GracefulShutdown.class, (context, message) -> {
context.getSystem().log().info("Initiating graceful shutdown...");
// perform graceful stop, executing cleanup before final system termination
// behavior executing cleanup is passed as a parameter to Actor.stopped
return Behaviors.stopped(Behaviors.receiveSignal((context, PostStop) -> {
return Behaviors.stopped(Behaviors.receiveSignal((_ctx, PostStop) -> {
context.getSystem().log().info("Cleanup!");
return Behaviors.same();
}));
})
.onSignal(PostStop.class, (ctx, signal) -> {
ctx.getSystem().log().info("Master Control Programme stopped");
.onSignal(PostStop.class, (context, signal) -> {
context.getSystem().log().info("Master Control Programme stopped");
return Behaviors.same();
})
.build();
@ -92,8 +92,8 @@ public class GracefulStopDocTest {
public static class Job {
public static Behavior<JobControl.JobControlLanguage> job(String name) {
return Behaviors.receiveSignal((ctx, PostStop) -> {
ctx.getSystem().log().info("Worker {} stopped", name);
return Behaviors.receiveSignal((context, PostStop) -> {
context.getSystem().log().info("Worker {} stopped", name);
return Behaviors.same();
});

View file

@ -33,8 +33,8 @@ public class InteractionPatternsTest extends JUnitSuite {
}
static final Behavior<PrintMe> printerBehavior = Behaviors.receive(PrintMe.class)
.onMessage(PrintMe.class, (ctx, printMe) -> {
ctx.getLog().info(printMe.message);
.onMessage(PrintMe.class, (context, printMe) -> {
context.getLog().info(printMe.message);
return Behaviors.same();
}).build();
// #fire-and-forget-definition
@ -62,7 +62,7 @@ public class InteractionPatternsTest extends JUnitSuite {
// #request-response-respond
// actor behavior
Behaviors.receive(Request.class)
.onMessage(Request.class, (ctx, request) -> {
.onMessage(Request.class, (context, request) -> {
// ... process request ...
request.respondTo.tell(new Response("Here's your response!"));
return Behaviors.same();
@ -70,10 +70,10 @@ public class InteractionPatternsTest extends JUnitSuite {
// #request-response-respond
ActorRef<Request> otherActor = null;
ActorContext<Response> ctx = null;
ActorContext<Response> context = null;
// #request-response-send
otherActor.tell(new Request("give me cookies", ctx.getSelf()));
otherActor.tell(new Request("give me cookies", context.getSelf()));
// #request-response-send
}
@ -171,18 +171,18 @@ public class InteractionPatternsTest extends JUnitSuite {
}
public static class Translator extends AbstractBehavior<Command> {
private final ActorContext<Command> ctx;
private final ActorContext<Command> context;
private final ActorRef<Backend.Request> backend;
private final ActorRef<Backend.Response> backendResponseAdapter;
private int taskIdCounter = 0;
private Map<Integer, ActorRef<URI>> inProgress = new HashMap<>();
public Translator(ActorContext<Command> ctx, ActorRef<Backend.Request> backend) {
this.ctx = ctx;
public Translator(ActorContext<Command> context, ActorRef<Backend.Request> backend) {
this.context = context;
this.backend = backend;
this.backendResponseAdapter =
ctx.messageAdapter(Backend.Response.class, rsp -> {
context.messageAdapter(Backend.Response.class, rsp -> {
if (rsp instanceof Backend.JobStarted)
return new WrappedJobStarted((Backend.JobStarted) rsp);
else if (rsp instanceof Backend.JobProgress)
@ -204,16 +204,16 @@ public class InteractionPatternsTest extends JUnitSuite {
return this;
})
.onMessage(WrappedJobStarted.class, wrapped -> {
ctx.getLog().info("Started {}", wrapped.response.taskId);
context.getLog().info("Started {}", wrapped.response.taskId);
return this;
})
.onMessage(WrappedJobProgress.class, wrapped -> {
ctx.getLog().info("Progress {}: {}", wrapped.response.taskId,
context.getLog().info("Progress {}: {}", wrapped.response.taskId,
wrapped.response.progress);
return this;
})
.onMessage(WrappedJobCompleted.class, wrapped -> {
ctx.getLog().info("Completed {}: {}", wrapped.response.taskId,
context.getLog().info("Completed {}: {}", wrapped.response.taskId,
wrapped.response.result);
return this;
})
@ -273,10 +273,10 @@ public class InteractionPatternsTest extends JUnitSuite {
}
public static final class ExcitingMessage implements Msg {
private final String msg;
private final String message;
public ExcitingMessage(String msg) {
this.msg = msg;
public ExcitingMessage(String message) {
this.message = message;
}
}
@ -292,10 +292,10 @@ public class InteractionPatternsTest extends JUnitSuite {
private static Behavior<Msg> idle(TimerScheduler<Msg> timers, ActorRef<Batch> target,
Duration after, int maxSize) {
return Behaviors.receive(Msg.class)
.onMessage(Msg.class, (ctx, msg) -> {
.onMessage(Msg.class, (context, message) -> {
timers.startSingleTimer(TIMER_KEY, new TimeoutMsg(), after);
List<Msg> buffer = new ArrayList<>();
buffer.add(msg);
buffer.add(message);
return active(buffer, timers, target, after, maxSize);
})
.build();
@ -304,12 +304,12 @@ public class InteractionPatternsTest extends JUnitSuite {
private static Behavior<Msg> active(List<Msg> buffer, TimerScheduler<Msg> timers,
ActorRef<Batch> target, Duration after, int maxSize) {
return Behaviors.receive(Msg.class)
.onMessage(TimeoutMsg.class, (ctx, msg) -> {
.onMessage(TimeoutMsg.class, (context, message) -> {
target.tell(new Batch(buffer));
return idle(timers, target, after, maxSize);
})
.onMessage(Msg.class, (ctx, msg) -> {
buffer.add(msg);
.onMessage(Msg.class, (context, message) -> {
buffer.add(message);
if (buffer.size() == maxSize) {
timers.cancel(TIMER_KEY);
target.tell(new Batch(buffer));
@ -362,8 +362,8 @@ public class InteractionPatternsTest extends JUnitSuite {
static final Behavior<HalCommand> halBehavior =
Behaviors.receive(HalCommand.class)
.onMessage(OpenThePodBayDoorsPlease.class, (ctx, msg) -> {
msg.respondTo.tell(new HalResponse("I'm sorry, Dave. I'm afraid I can't do that."));
.onMessage(OpenThePodBayDoorsPlease.class, (context, message) -> {
message.respondTo.tell(new HalResponse("I'm sorry, Dave. I'm afraid I can't do that."));
return Behaviors.same();
}).build();
@ -378,13 +378,13 @@ public class InteractionPatternsTest extends JUnitSuite {
}
public static Behavior<DaveProtocol> daveBehavior(final ActorRef<HalCommand> hal) {
return Behaviors.setup((ActorContext<DaveProtocol> ctx) -> {
return Behaviors.setup((ActorContext<DaveProtocol> context) -> {
// asking someone requires a timeout, if the timeout hits without response
// the ask is failed with a TimeoutException
final Timeout timeout = Timeout.apply(3, TimeUnit.SECONDS);
ctx.ask(
context.ask(
HalResponse.class,
hal,
timeout,
@ -404,7 +404,7 @@ public class InteractionPatternsTest extends JUnitSuite {
// changed at the time the response arrives and the transformation is done, best is to
// use immutable state we have closed over like here.
final int requestId = 1;
ctx.ask(
context.ask(
HalResponse.class,
hal,
timeout,
@ -509,13 +509,13 @@ public class InteractionPatternsTest extends JUnitSuite {
// actor behavior
public Behavior<HomeCommand> homeBehavior() {
return Behaviors.setup((ctx) -> {
final ActorRef<GetKeys> keyCabinet = ctx.spawn(keyCabinetBehavior, "key-cabinet");
final ActorRef<GetWallet> drawer = ctx.spawn(drawerBehavior, "drawer");
return Behaviors.setup((context) -> {
final ActorRef<GetKeys> keyCabinet = context.spawn(keyCabinetBehavior, "key-cabinet");
final ActorRef<GetWallet> drawer = context.spawn(drawerBehavior, "drawer");
return Behaviors.receive(HomeCommand.class)
.onMessage(LeaveHome.class, (innerCtx, msg) -> {
ctx.spawn(new PrepareToLeaveHome(msg.who, msg.respondTo, keyCabinet, drawer), "leaving" + msg.who);
.onMessage(LeaveHome.class, (innerCtx, message) -> {
context.spawn(new PrepareToLeaveHome(message.who, message.respondTo, keyCabinet, drawer), "leaving" + message.who);
return Behavior.same();
}).build();
});

View file

@ -50,9 +50,9 @@ public class IntroTest {
}
}
public static final Behavior<Greet> greeter = Behaviors.receive((ctx, msg) -> {
ctx.getLog().info("Hello {}!", msg.whom);
msg.replyTo.tell(new Greeted(msg.whom, ctx.getSelf()));
public static final Behavior<Greet> greeter = Behaviors.receive((context, message) -> {
context.getLog().info("Hello {}!", message.whom);
message.replyTo.tell(new Greeted(message.whom, context.getSelf()));
return Behaviors.same();
});
}
@ -64,13 +64,13 @@ public class IntroTest {
}
public static final Behavior<HelloWorld.Greeted> bot(int greetingCounter, int max) {
return Behaviors.receive((ctx, msg) -> {
return Behaviors.receive((context, message) -> {
int n = greetingCounter + 1;
ctx.getLog().info("Greeting {} for {}", n, msg.whom);
context.getLog().info("Greeting {} for {}", n, message.whom);
if (n == max) {
return Behaviors.stopped();
} else {
msg.from.tell(new HelloWorld.Greet(msg.whom, ctx.getSelf()));
message.from.tell(new HelloWorld.Greet(message.whom, context.getSelf()));
return bot(n, max);
}
});
@ -96,10 +96,10 @@ public class IntroTest {
final ActorRef<HelloWorld.Greet> greeter =
context.spawn(HelloWorld.greeter, "greeter");
return Behaviors.receiveMessage(msg -> {
return Behaviors.receiveMessage(message -> {
ActorRef<HelloWorld.Greeted> replyTo =
context.spawn(HelloWorldBot.bot(0, 3), msg.name);
greeter.tell(new HelloWorld.Greet(msg.name, replyTo));
context.spawn(HelloWorldBot.bot(0, 3), message.name);
greeter.tell(new HelloWorld.Greet(message.name, replyTo));
return Behaviors.same();
});
});
@ -127,10 +127,10 @@ public class IntroTest {
final ActorRef<HelloWorld.Greet> greeter =
context.spawn(HelloWorld.greeter, "greeter", props);
return Behaviors.receiveMessage(msg -> {
return Behaviors.receiveMessage(message -> {
ActorRef<HelloWorld.Greeted> replyTo =
context.spawn(HelloWorldBot.bot(0, 3), msg.name);
greeter.tell(new HelloWorld.Greet(msg.name, replyTo));
context.spawn(HelloWorldBot.bot(0, 3), message.name);
greeter.tell(new HelloWorld.Greet(message.name, replyTo));
return Behaviors.same();
});
});
@ -219,10 +219,10 @@ public class IntroTest {
private static Behavior<RoomCommand> chatRoom(List<ActorRef<SessionCommand>> sessions) {
return Behaviors.receive(RoomCommand.class)
.onMessage(GetSession.class, (ctx, getSession) -> {
.onMessage(GetSession.class, (context, getSession) -> {
ActorRef<SessionEvent> client = getSession.replyTo;
ActorRef<SessionCommand> ses = ctx.spawn(
session(ctx.getSelf(), getSession.screenName, client),
ActorRef<SessionCommand> ses = context.spawn(
session(context.getSelf(), getSession.screenName, client),
URLEncoder.encode(getSession.screenName, StandardCharsets.UTF_8.name()));
// narrow to only expose PostMessage
client.tell(new SessionGranted(ses.narrow()));
@ -230,7 +230,7 @@ public class IntroTest {
newSessions.add(ses);
return chatRoom(newSessions);
})
.onMessage(PublishSessionMessage.class, (ctx, pub) -> {
.onMessage(PublishSessionMessage.class, (context, pub) -> {
NotifyClient notification =
new NotifyClient((new MessagePosted(pub.screenName, pub.message)));
sessions.forEach(s -> s.tell(notification));
@ -244,12 +244,12 @@ public class IntroTest {
String screenName,
ActorRef<SessionEvent> client) {
return Behaviors.receive(ChatRoom.SessionCommand.class)
.onMessage(PostMessage.class, (ctx, post) -> {
.onMessage(PostMessage.class, (context, post) -> {
// from client, publish to others via the room
room.tell(new PublishSessionMessage(screenName, post.message));
return Behaviors.same();
})
.onMessage(NotifyClient.class, (ctx, notification) -> {
.onMessage(NotifyClient.class, (context, notification) -> {
// published from the room
client.tell(notification.message);
return Behaviors.same();
@ -268,17 +268,17 @@ public class IntroTest {
public static Behavior<ChatRoom.SessionEvent> behavior() {
return Behaviors.receive(ChatRoom.SessionEvent.class)
.onMessage(ChatRoom.SessionDenied.class, (ctx, msg) -> {
System.out.println("cannot start chat room session: " + msg.reason);
.onMessage(ChatRoom.SessionDenied.class, (context, message) -> {
System.out.println("cannot start chat room session: " + message.reason);
return Behaviors.stopped();
})
.onMessage(ChatRoom.SessionGranted.class, (ctx, msg) -> {
msg.handle.tell(new ChatRoom.PostMessage("Hello World!"));
.onMessage(ChatRoom.SessionGranted.class, (context, message) -> {
message.handle.tell(new ChatRoom.PostMessage("Hello World!"));
return Behaviors.same();
})
.onMessage(ChatRoom.MessagePosted.class, (ctx, msg) -> {
.onMessage(ChatRoom.MessagePosted.class, (context, message) -> {
System.out.println("message has been posted by '" +
msg.screenName +"': " + msg.message);
message.screenName +"': " + message.message);
return Behaviors.stopped();
})
.build();
@ -290,12 +290,12 @@ public class IntroTest {
public static void runChatRoom() throws Exception {
//#chatroom-main
Behavior<Void> main = Behaviors.setup(ctx -> {
Behavior<Void> main = Behaviors.setup(context -> {
ActorRef<ChatRoom.RoomCommand> chatRoom =
ctx.spawn(ChatRoom.behavior(), "chatRoom");
context.spawn(ChatRoom.behavior(), "chatRoom");
ActorRef<ChatRoom.SessionEvent> gabbler =
ctx.spawn(Gabbler.behavior(), "gabbler");
ctx.watch(gabbler);
context.spawn(Gabbler.behavior(), "gabbler");
context.watch(gabbler);
chatRoom.tell(new ChatRoom.GetSession("ol Gabbler", gabbler));
return Behaviors.<Void>receiveSignal(

View file

@ -90,11 +90,11 @@ public class OOIntroTest {
}
public static class ChatRoomBehavior extends AbstractBehavior<RoomCommand> {
final ActorContext<RoomCommand> ctx;
final ActorContext<RoomCommand> context;
final List<ActorRef<SessionCommand>> sessions = new ArrayList<>();
public ChatRoomBehavior(ActorContext<RoomCommand> ctx) {
this.ctx = ctx;
public ChatRoomBehavior(ActorContext<RoomCommand> context) {
this.context = context;
}
@Override
@ -102,8 +102,8 @@ public class OOIntroTest {
return receiveBuilder()
.onMessage(GetSession.class, getSession -> {
ActorRef<SessionEvent> client = getSession.replyTo;
ActorRef<SessionCommand> ses = ctx.spawn(
session(ctx.getSelf(), getSession.screenName, client),
ActorRef<SessionCommand> ses = context.spawn(
session(context.getSelf(), getSession.screenName, client),
URLEncoder.encode(getSession.screenName, StandardCharsets.UTF_8.name()));
// narrow to only expose PostMessage
client.tell(new SessionGranted(ses.narrow()));
@ -125,12 +125,12 @@ public class OOIntroTest {
String screenName,
ActorRef<SessionEvent> client) {
return Behaviors.receive(ChatRoom.SessionCommand.class)
.onMessage(PostMessage.class, (ctx, post) -> {
.onMessage(PostMessage.class, (context, post) -> {
// from client, publish to others via the room
room.tell(new PublishSessionMessage(screenName, post.message));
return Behaviors.same();
})
.onMessage(NotifyClient.class, (ctx, notification) -> {
.onMessage(NotifyClient.class, (context, notification) -> {
// published from the room
client.tell(notification.message);
return Behaviors.same();
@ -149,17 +149,17 @@ public class OOIntroTest {
public static Behavior<ChatRoom.SessionEvent> behavior() {
return Behaviors.receive(ChatRoom.SessionEvent.class)
.onMessage(ChatRoom.SessionDenied.class, (ctx, msg) -> {
System.out.println("cannot start chat room session: " + msg.reason);
.onMessage(ChatRoom.SessionDenied.class, (context, message) -> {
System.out.println("cannot start chat room session: " + message.reason);
return Behaviors.stopped();
})
.onMessage(ChatRoom.SessionGranted.class, (ctx, msg) -> {
msg.handle.tell(new ChatRoom.PostMessage("Hello World!"));
.onMessage(ChatRoom.SessionGranted.class, (context, message) -> {
message.handle.tell(new ChatRoom.PostMessage("Hello World!"));
return Behaviors.same();
})
.onMessage(ChatRoom.MessagePosted.class, (ctx, msg) -> {
.onMessage(ChatRoom.MessagePosted.class, (context, message) -> {
System.out.println("message has been posted by '" +
msg.screenName +"': " + msg.message);
message.screenName +"': " + message.message);
return Behaviors.stopped();
})
.build();
@ -171,12 +171,12 @@ public class OOIntroTest {
public static void runChatRoom() throws Exception {
//#chatroom-main
Behavior<Void> main = Behaviors.setup(ctx -> {
Behavior<Void> main = Behaviors.setup(context -> {
ActorRef<ChatRoom.RoomCommand> chatRoom =
ctx.spawn(ChatRoom.behavior(), "chatRoom");
context.spawn(ChatRoom.behavior(), "chatRoom");
ActorRef<ChatRoom.SessionEvent> gabbler =
ctx.spawn(Gabbler.behavior(), "gabbler");
ctx.watch(gabbler);
context.spawn(Gabbler.behavior(), "gabbler");
context.watch(gabbler);
chatRoom.tell(new ChatRoom.GetSession("ol Gabbler", gabbler));
return Behaviors.<Void>receiveSignal(

View file

@ -33,9 +33,9 @@ public class SpawnProtocolDocTest {
}
public static final Behavior<SpawnProtocol> main =
Behaviors.setup( ctx -> {
Behaviors.setup( context -> {
// Start initial tasks
// ctx.spawn(...)
// context.spawn(...)
return SpawnProtocol.behavior();
});
@ -56,8 +56,8 @@ public class SpawnProtocolDocTest {
system.scheduler());
Behavior<HelloWorld.Greeted> greetedBehavior =
Behaviors.receive((ctx, msg) -> {
ctx.getLog().info("Greeting for {} from {}", msg.whom, msg.from);
Behaviors.receive((context, message) -> {
context.getLog().info("Greeting for {} from {}", message.whom, message.from);
return Behaviors.stopped();
});

View file

@ -89,13 +89,13 @@ public class StashDocTest extends JUnitSuite {
}
Behavior<Command> behavior() {
return Behaviors.setup(ctx -> {
return Behaviors.setup(context -> {
db.load(id)
.whenComplete((value, cause) -> {
if (cause == null)
ctx.getSelf().tell(new InitialState(value));
context.getSelf().tell(new InitialState(value));
else
ctx.getSelf().tell(new DBError(asRuntimeException(cause)));
context.getSelf().tell(new DBError(asRuntimeException(cause)));
});
return init();
@ -104,16 +104,16 @@ public class StashDocTest extends JUnitSuite {
private Behavior<Command> init() {
return Behaviors.receive(Command.class)
.onMessage(InitialState.class, (ctx, msg) -> {
.onMessage(InitialState.class, (context, message) -> {
// now we are ready to handle stashed messages if any
return buffer.unstashAll(ctx, active(msg.value));
return buffer.unstashAll(context, active(message.value));
})
.onMessage(DBError.class, (ctx, msg) -> {
throw msg.cause;
.onMessage(DBError.class, (context, message) -> {
throw message.cause;
})
.onMessage(Command.class, (ctx, msg) -> {
.onMessage(Command.class, (context, message) -> {
// stash all other messages for later processing
buffer.stash(msg);
buffer.stash(message);
return Behaviors.same();
})
.build();
@ -121,34 +121,34 @@ public class StashDocTest extends JUnitSuite {
private Behavior<Command> active(String state) {
return Behaviors.receive(Command.class)
.onMessage(Get.class, (ctx, msg) -> {
msg.replyTo.tell(state);
.onMessage(Get.class, (context, message) -> {
message.replyTo.tell(state);
return Behaviors.same();
})
.onMessage(Save.class, (ctx, msg) -> {
db.save(id, msg.payload)
.onMessage(Save.class, (context, message) -> {
db.save(id, message.payload)
.whenComplete((value, cause) -> {
if (cause == null)
ctx.getSelf().tell(SaveSuccess.instance);
context.getSelf().tell(SaveSuccess.instance);
else
ctx.getSelf().tell(new DBError(asRuntimeException(cause)));
context.getSelf().tell(new DBError(asRuntimeException(cause)));
});
return saving(msg.payload, msg.replyTo);
return saving(message.payload, message.replyTo);
})
.build();
}
private Behavior<Command> saving(String state, ActorRef<Done> replyTo) {
return Behaviors.receive(Command.class)
.onMessageEquals(SaveSuccess.instance, ctx -> {
.onMessageEquals(SaveSuccess.instance, context -> {
replyTo.tell(Done.getInstance());
return buffer.unstashAll(ctx, active(state));
return buffer.unstashAll(context, active(state));
})
.onMessage(DBError.class, (ctx, msg) -> {
throw msg.cause;
.onMessage(DBError.class, (context, message) -> {
throw message.cause;
})
.onMessage(Command.class, (ctx, msg) -> {
buffer.stash(msg);
.onMessage(Command.class, (context, message) -> {
buffer.stash(message);
return Behaviors.same();
})
.build();

View file

@ -46,11 +46,11 @@ public class TypedWatchingUntypedTest extends JUnitSuite {
Adapter.toUntyped(context.getSelf()));
return akka.actor.typed.javadsl.Behaviors.receive(Typed.Command.class)
.onMessage(Typed.Pong.class, (ctx, msg) -> {
Adapter.stop(ctx, second);
.onMessage(Typed.Pong.class, (_ctx, message) -> {
Adapter.stop(context, second);
return same();
})
.onSignal(akka.actor.typed.Terminated.class, (ctx, sig) -> stopped())
.onSignal(akka.actor.typed.Terminated.class, (_ctx, sig) -> stopped())
.build();
});
}
@ -66,8 +66,8 @@ public class TypedWatchingUntypedTest extends JUnitSuite {
@Override
public Receive createReceive() {
return receiveBuilder()
.match(Typed.Ping.class, msg -> {
msg.replyTo.tell(new Typed.Pong());
.match(Typed.Ping.class, message -> {
message.replyTo.tell(new Typed.Pong());
})
.build();
}

View file

@ -41,7 +41,7 @@ public class UntypedWatchingTypedTest extends JUnitSuite {
@Override
public Receive createReceive() {
return receiveBuilder()
.match(Typed.Pong.class, msg -> {
.match(Typed.Pong.class, message -> {
Adapter.stop(getContext(), second);
})
.match(akka.actor.Terminated.class, t -> {
@ -68,8 +68,8 @@ public class UntypedWatchingTypedTest extends JUnitSuite {
public static Behavior<Command> behavior() {
return Behaviors.receive(Typed.Command.class)
.onMessage(Typed.Ping.class, (ctx, msg) -> {
msg.replyTo.tell(new Pong());
.onMessage(Typed.Ping.class, (context, message) -> {
message.replyTo.tell(new Pong());
return same();
})
.build();

View file

@ -36,10 +36,10 @@ public class SupervisionCompileOnlyTest {
public static Behavior<CounterMessage> counter(int currentValue) {
return Behaviors.receive(CounterMessage.class)
.onMessage(Increase.class, (ctx, o) -> {
.onMessage(Increase.class, (context, o) -> {
return counter(currentValue + 1);
})
.onMessage(Get.class, (ctx, o) -> {
.onMessage(Get.class, (context, o) -> {
o.sender.tell(new Got(currentValue));
return Behaviors.same();
})

View file

@ -84,8 +84,8 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit(
"be usable from Behavior.interpretMessage" in {
// compilation only
lazy val b: Behavior[String] = Behaviors.receive { (ctx, msg)
Behavior.interpretMessage(b, ctx, msg)
lazy val b: Behavior[String] = Behaviors.receive { (context, message)
Behavior.interpretMessage(b, context, message)
}
}
@ -173,16 +173,16 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit(
Behavior.stopped
}).decorate
val parent: Behavior[Command] = Behaviors.setup[Command](ctx {
val childRef = ctx.spawnAnonymous(
val parent: Behavior[Command] = Behaviors.setup[Command](context {
val childRef = context.spawnAnonymous(
Behaviors.supervise(child).onFailure(SupervisorStrategy.restart)
)
ctx.watch(childRef)
context.watch(childRef)
probe.ref ! ChildMade(childRef)
(Behaviors.receivePartial[Command] {
case (ctx, StopRef(ref))
ctx.stop(ref)
case (context, StopRef(ref))
context.stop(ref)
Behavior.same
} receiveSignal {
case (_, signal)
@ -207,13 +207,13 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit(
val probe = TestProbe[Event]()
val child: Behavior[Command] = Behaviors.empty[Command].decorate
val parent: Behavior[Command] = Behaviors.setup[Command](ctx {
val childRef = ctx.spawnAnonymous(child)
ctx.watch(childRef)
val parent: Behavior[Command] = Behaviors.setup[Command](context {
val childRef = context.spawnAnonymous(child)
context.watch(childRef)
probe.ref ! ChildMade(childRef)
Behaviors.receivePartial[Command] {
case (ctx, StopRef(ref))
ctx.stop(ref)
case (context, StopRef(ref))
context.stop(ref)
Behaviors.same
} receiveSignal {
case (_, signal)
@ -291,8 +291,8 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit(
val actorToWatch = spawn(behavior)
val watcher: ActorRef[Command] = spawn((
Behaviors.receivePartial[Any] {
case (ctx, Ping)
ctx.watch(actorToWatch)
case (context, Ping)
context.watch(actorToWatch)
probe.ref ! Pong
Behavior.same
} receiveSignal {
@ -319,9 +319,9 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit(
case (_, Ping)
probe.ref ! Pong
Behaviors.same
case (ctx, StopRef(ref))
case (context, StopRef(ref))
assertThrows[IllegalArgumentException] {
ctx.stop(ref)
context.stop(ref)
probe.ref ! Pong
}
probe.ref ! Missed
@ -342,9 +342,9 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit(
Behaviors.stopped
}.decorate
spawn(
Behaviors.setup[Command](ctx {
val childRef = ctx.spawn(child, "A")
ctx.watch(childRef)
Behaviors.setup[Command](context {
val childRef = context.spawn(child, "A")
context.watch(childRef)
probe.ref ! ChildMade(childRef)
Behaviors.receivePartial[Command] {
case (_, Ping)
@ -369,12 +369,12 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit(
Behaviors.stopped
}.decorate
val actor = spawn(
Behaviors.setup[Command](ctx {
val childRef = ctx.spawn(child, "A")
Behaviors.setup[Command](context {
val childRef = context.spawn(child, "A")
probe.ref ! ChildMade(childRef)
Behaviors.receivePartial[Command] {
case (ctx, Watch(ref))
ctx.watch(ref)
case (context, Watch(ref))
context.watch(ref)
probe.ref ! Pong
Behaviors.same
} receiveSignal {
@ -400,16 +400,16 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit(
Behaviors.stopped
}.decorate
val actor = spawn(
Behaviors.setup[Command](ctx {
val childRef = ctx.spawn(child, "A")
Behaviors.setup[Command](context {
val childRef = context.spawn(child, "A")
probe.ref ! ChildMade(childRef)
Behaviors.receivePartial[Command] {
case (ctx, Watch(ref))
ctx.watch(ref)
case (context, Watch(ref))
context.watch(ref)
probe.ref ! Pong
Behaviors.same
case (ctx, UnWatch(ref))
ctx.unwatch(ref)
case (context, UnWatch(ref))
context.unwatch(ref)
probe.ref ! Pong
Behaviors.same
} receiveSignal {
@ -439,9 +439,9 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit(
Behavior.same
}).decorate
val actor = spawn(
Behaviors.setup[Command](ctx {
val childRef = ctx.spawn(child, "A")
ctx.watch(childRef)
Behaviors.setup[Command](context {
val childRef = context.spawn(child, "A")
context.watch(childRef)
probe.ref ! ChildMade(childRef)
Behaviors.receivePartial[Command] {
case (_, Inert)
@ -476,8 +476,8 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit(
type Info = (ActorSystem[Nothing], ActorRef[String])
val probe = TestProbe[Info]
val actor = spawn(Behaviors.receivePartial[String] {
case (ctx, "info")
probe.ref ! (ctx.system ctx.self)
case (context, "info")
probe.ref ! (context.system context.self)
Behaviors.same
}.decorate)
actor ! "info"
@ -488,15 +488,15 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit(
type Children = Seq[ActorRef[Nothing]]
val probe = TestProbe[Children]()
val actor = spawn(Behaviors.receivePartial[String] {
case (ctx, "create")
ctx.spawn(Behaviors.empty, "B")
probe.ref ! ctx.child("B").toSeq
case (context, "create")
context.spawn(Behaviors.empty, "B")
probe.ref ! context.child("B").toSeq
Behaviors.same
case (ctx, "all")
probe.ref ! ctx.children.toSeq
case (context, "all")
probe.ref ! context.children.toSeq
Behaviors.same
case (ctx, get)
probe.ref ! ctx.child(get).toSeq
case (context, get)
probe.ref ! context.child(get).toSeq
Behaviors.same
}.decorate)
actor ! "create"
@ -515,8 +515,8 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit(
case (_, ReceiveTimeout)
probe.ref ! GotReceiveTimeout
Behaviors.same
case (ctx, SetTimeout(duration))
ctx.setReceiveTimeout(duration, ReceiveTimeout)
case (context, SetTimeout(duration))
context.setReceiveTimeout(duration, ReceiveTimeout)
probe.ref ! TimeoutSet
Behaviors.same
}.decorate)
@ -528,8 +528,8 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit(
"set large receive timeout" in {
val probe = TestProbe[String]()
val actor = spawn(Behaviors.receivePartial[String] {
case (ctx, "schedule")
ctx.schedule(1.second, probe.ref, "scheduled")
case (context, "schedule")
context.schedule(1.second, probe.ref, "scheduled")
Behaviors.same
case (_, "ping")
probe.ref ! "pong"
@ -537,8 +537,8 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit(
case (_, "receive timeout")
probe.ref ! "received timeout"
Behaviors.same
case (ctx, duration)
ctx.setReceiveTimeout(Duration(duration).asInstanceOf[FiniteDuration], "receive timeout")
case (context, duration)
context.setReceiveTimeout(Duration(duration).asInstanceOf[FiniteDuration], "receive timeout")
probe.ref ! "timeout set"
Behaviors.same
}.decorate)
@ -553,8 +553,8 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit(
"schedule a message" in {
val probe = TestProbe[Event]()
val actor = spawn(Behaviors.receivePartial[Command] {
case (ctx, Ping)
ctx.schedule(1.nano, probe.ref, Pong)
case (context, Ping)
context.schedule(1.nano, probe.ref, Pong)
Behaviors.same
}.decorate)
actor ! Ping
@ -566,11 +566,11 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit(
val messages = TestProbe[Envelope]()
val probe = TestProbe[ActorRef[String]]()
val actor = spawn(Behaviors.receivePartial[String] {
case (ctx, "message")
messages.ref.tell((ctx.self, "received message"))
case (context, "message")
messages.ref.tell((context.self, "received message"))
Behaviors.same
case (ctx, name)
probe.ref ! ctx.spawnMessageAdapter(identity, name)
case (context, name)
probe.ref ! context.spawnMessageAdapter(identity, name)
Behaviors.same
}.decorate)
val adapterName = "hello"
@ -591,12 +591,12 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit(
"not have problems stopping already stopped child" in {
val probe = TestProbe[Event]()
val actor = spawn(
Behaviors.setup[Command](ctx {
val child = ctx.spawnAnonymous(Behaviors.empty[Command])
Behaviors.setup[Command](context {
val child = context.spawnAnonymous(Behaviors.empty[Command])
probe.ref ! ChildMade(child)
Behaviors.receivePartial[Command] {
case (ctx, StopRef(ref))
ctx.stop(ref)
case (context, StopRef(ref))
context.stop(ref)
probe.ref ! Pong
Behaviors.same
}
@ -636,10 +636,10 @@ class InterceptActorContextSpec extends ActorContextSpec {
import BehaviorInterceptor._
def tap[T] = new BehaviorInterceptor[T, T] {
override def aroundReceive(ctx: ActorContext[T], msg: T, target: ReceiveTarget[T]): Behavior[T] =
target(ctx, msg)
override def aroundSignal(ctx: ActorContext[T], signal: Signal, target: SignalTarget[T]): Behavior[T] =
target(ctx, signal)
override def aroundReceive(context: ActorContext[T], message: T, target: ReceiveTarget[T]): Behavior[T] =
target(context, message)
override def aroundSignal(context: ActorContext[T], signal: Signal, target: SignalTarget[T]): Behavior[T] =
target(context, signal)
}
override def decoration[T]: Behavior[T] Behavior[T] = b Behaviors.intercept[T, T](tap)(b)

View file

@ -132,8 +132,8 @@ class AskSpec extends ScalaTestWithActorTestKit("""
val probe = TestProbe[AnyRef]("probe")
val behv =
Behaviors.receive[String] {
case (ctx, "start-ask")
ctx.ask[Question, Long](probe.ref)(Question(_)) {
case (context, "start-ask")
context.ask[Question, Long](probe.ref)(Question(_)) {
case Success(42L)
throw new RuntimeException("Unsupported number")
case _ "test"

View file

@ -20,30 +20,30 @@ import org.scalatest.WordSpecLike
object BehaviorSpec {
sealed trait Command {
def expectedResponse(ctx: ActorContext[Command]): Seq[Event] = Nil
def expectedResponse(context: ActorContext[Command]): Seq[Event] = Nil
}
case object GetSelf extends Command {
override def expectedResponse(ctx: ActorContext[Command]): Seq[Event] = Self(ctx.asScala.self) :: Nil
override def expectedResponse(context: ActorContext[Command]): Seq[Event] = Self(context.asScala.self) :: Nil
}
// Behavior under test must return Unhandled
case object Miss extends Command {
override def expectedResponse(ctx: ActorContext[Command]): Seq[Event] = Missed :: Nil
override def expectedResponse(context: 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
override def expectedResponse(context: ActorContext[Command]): Seq[Event] = Ignored :: Nil
}
case object Ping extends Command {
override def expectedResponse(ctx: ActorContext[Command]): Seq[Event] = Pong :: Nil
override def expectedResponse(context: ActorContext[Command]): Seq[Event] = Pong :: Nil
}
case object Swap extends Command {
override def expectedResponse(ctx: ActorContext[Command]): Seq[Event] = Swapped :: Nil
override def expectedResponse(context: ActorContext[Command]): Seq[Event] = Swapped :: Nil
}
case class GetState()(s: State) extends Command {
override def expectedResponse(ctx: ActorContext[Command]): Seq[Event] = s :: Nil
override def expectedResponse(context: ActorContext[Command]): Seq[Event] = s :: Nil
}
case class AuxPing(id: Int) extends Command {
override def expectedResponse(ctx: ActorContext[Command]): Seq[Event] = Pong :: Nil
override def expectedResponse(context: ActorContext[Command]): Seq[Event] = Pong :: Nil
}
case object Stop extends Command
@ -106,13 +106,13 @@ object BehaviorSpec {
}
def check(command: Command): Setup = {
setup.testKit.run(command)
setup.inbox.receiveAll() should ===(command.expectedResponse(setup.testKit.ctx))
setup.inbox.receiveAll() should ===(command.expectedResponse(setup.testKit.context))
checkAux(command, setup.aux)
setup
}
def check2(command: Command): Setup = {
setup.testKit.run(command)
val expected = command.expectedResponse(setup.testKit.ctx)
val expected = command.expectedResponse(setup.testKit.context)
setup.inbox.receiveAll() should ===(expected ++ expected)
checkAux(command, setup.aux)
setup
@ -144,8 +144,8 @@ object BehaviorSpec {
def mkFull(monitor: ActorRef[Event], state: State = StateA): Behavior[Command] = {
SBehaviors.receive[Command] {
case (ctx, GetSelf)
monitor ! Self(ctx.self)
case (context, GetSelf)
monitor ! Self(context.self)
SBehaviors.same
case (_, Miss)
monitor ! Missed
@ -175,19 +175,19 @@ object BehaviorSpec {
*/
def fs(f: (JActorContext[Command], Signal) Behavior[Command]) =
new F2[JActorContext[Command], Signal, Behavior[Command]] {
override def apply(ctx: JActorContext[Command], sig: Signal) = f(ctx, sig)
override def apply(context: JActorContext[Command], sig: Signal) = f(context, sig)
}
def fc(f: (JActorContext[Command], Command) Behavior[Command]) =
new F2[JActorContext[Command], Command, Behavior[Command]] {
override def apply(ctx: JActorContext[Command], command: Command) = f(ctx, command)
override def apply(context: JActorContext[Command], command: Command) = f(context, command)
}
def ps(f: (JActorContext[Command], Signal) Unit) =
new P2[JActorContext[Command], Signal] {
override def apply(ctx: JActorContext[Command], sig: Signal) = f(ctx, sig)
override def apply(context: JActorContext[Command], sig: Signal) = f(context, sig)
}
def pc(f: (JActorContext[Command], Command) Unit) =
new P2[JActorContext[Command], Command] {
override def apply(ctx: JActorContext[Command], command: Command) = f(ctx, command)
override def apply(context: JActorContext[Command], command: Command) = f(context, command)
}
def pf(f: PFBuilder[Command, Command] PFBuilder[Command, Command]) =
new F1[PFBuilder[Command, Command], PFBuilder[Command, Command]] {
@ -258,7 +258,7 @@ object BehaviorSpec {
"Unhandled" must {
"must return Unhandled" in {
val Setup(testKit, inbox, aux) = mkCtx()
Behavior.interpretMessage(testKit.currentBehavior, testKit.ctx, Miss) should be(Behavior.UnhandledBehavior)
Behavior.interpretMessage(testKit.currentBehavior, testKit.context, Miss) should be(Behavior.UnhandledBehavior)
inbox.receiveAll() should ===(Missed :: Nil)
checkAux(Miss, aux)
}
@ -350,8 +350,8 @@ class ReceiveBehaviorSpec extends Messages with BecomeWithLifecycle with Stoppab
override def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux) = behv(monitor, StateA) null
private def behv(monitor: ActorRef[Event], state: State): Behavior[Command] = {
SBehaviors.receive[Command] {
case (ctx, GetSelf)
monitor ! Self(ctx.self)
case (context, GetSelf)
monitor ! Self(context.self)
SBehaviors.same
case (_, Miss)
monitor ! Missed
@ -384,10 +384,10 @@ class ImmutableWithSignalScalaBehaviorSpec extends Messages with BecomeWithLifec
def behv(monitor: ActorRef[Event], state: State = StateA): Behavior[Command] =
SBehaviors.receive[Command] {
(ctx, msg)
msg match {
(context, message)
message match {
case GetSelf
monitor ! Self(ctx.self)
monitor ! Self(context.self)
SBehaviors.same
case Miss
monitor ! Missed
@ -419,10 +419,10 @@ class ImmutableScalaBehaviorSpec extends Messages with Become with Stoppable {
override def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux) = behv(monitor, StateA) null
def behv(monitor: ActorRef[Event], state: State): Behavior[Command] =
SBehaviors.receive[Command] { (ctx, msg)
msg match {
SBehaviors.receive[Command] { (context, message)
message match {
case GetSelf
monitor ! Self(ctx.self)
monitor ! Self(context.self)
SBehaviors.same
case Miss
monitor ! Missed
@ -450,14 +450,14 @@ class MutableScalaBehaviorSpec extends Messages with Become with Stoppable {
override def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux) = behv(monitor) null
def behv(monitor: ActorRef[Event]): Behavior[Command] =
SBehaviors.setup[Command] { ctx
SBehaviors.setup[Command] { context
new SAbstractBehavior[Command] {
private var state: State = StateA
override def onMessage(msg: Command): Behavior[Command] = {
msg match {
override def onMessage(message: Command): Behavior[Command] = {
message match {
case GetSelf
monitor ! Self(ctx.self)
monitor ! Self(context.self)
this
case Miss
monitor ! Missed
@ -512,14 +512,14 @@ class InterceptScalaBehaviorSpec extends ImmutableWithSignalScalaBehaviorSpec wi
override def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux) = {
val inbox = TestInbox[Either[Signal, Command]]("tapListener")
val tap = new BehaviorInterceptor[Command, Command] {
override def aroundReceive(ctx: ActorContext[Command], msg: Command, target: ReceiveTarget[Command]): Behavior[Command] = {
inbox.ref ! Right(msg)
target(ctx, msg)
override def aroundReceive(context: ActorContext[Command], message: Command, target: ReceiveTarget[Command]): Behavior[Command] = {
inbox.ref ! Right(message)
target(context, message)
}
override def aroundSignal(ctx: ActorContext[Command], signal: Signal, target: SignalTarget[Command]): Behavior[Command] = {
override def aroundSignal(context: ActorContext[Command], signal: Signal, target: SignalTarget[Command]): Behavior[Command] = {
inbox.ref ! Left(signal)
target(ctx, signal)
target(context, signal)
}
}
(SBehaviors.intercept(tap)(super.behavior(monitor)._1), inbox)
@ -536,9 +536,9 @@ class ImmutableWithSignalJavaBehaviorSpec extends Messages with BecomeWithLifecy
override def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux) = behv(monitor) null
def behv(monitor: ActorRef[Event], state: State = StateA): Behavior[Command] =
JBehaviors.receive(
fc((ctx, msg) msg match {
fc((context, message) message match {
case GetSelf
monitor ! Self(ctx.getSelf)
monitor ! Self(context.getSelf)
SBehaviors.same
case Miss
monitor ! Missed
@ -568,10 +568,10 @@ class ImmutableJavaBehaviorSpec extends Messages with Become with Stoppable {
override def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux) = behv(monitor, StateA) null
def behv(monitor: ActorRef[Event], state: State): Behavior[Command] =
JBehaviors.receive {
fc((ctx, msg)
msg match {
fc((context, message)
message match {
case GetSelf
monitor ! Self(ctx.getSelf)
monitor ! Self(context.getSelf)
SBehaviors.same
case Miss
monitor ! Missed
@ -625,14 +625,14 @@ class TapJavaBehaviorSpec extends ImmutableWithSignalJavaBehaviorSpec with Reuse
override def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux) = {
val inbox = TestInbox[Either[Signal, Command]]("tapListener")
val tap = new BehaviorInterceptor[Command, Command] {
override def aroundReceive(ctx: ActorContext[Command], msg: Command, target: ReceiveTarget[Command]): Behavior[Command] = {
inbox.ref ! Right(msg)
target(ctx, msg)
override def aroundReceive(context: ActorContext[Command], message: Command, target: ReceiveTarget[Command]): Behavior[Command] = {
inbox.ref ! Right(message)
target(context, message)
}
override def aroundSignal(ctx: ActorContext[Command], signal: Signal, target: SignalTarget[Command]): Behavior[Command] = {
override def aroundSignal(context: ActorContext[Command], signal: Signal, target: SignalTarget[Command]): Behavior[Command] = {
inbox.ref ! Left(signal)
target(ctx, signal)
target(context, signal)
}
}
(JBehaviors.intercept(tap, super.behavior(monitor)._1), inbox)

View file

@ -56,12 +56,12 @@ class DeferredSpec extends ScalaTestWithActorTestKit(
"must stop when exception from factory" in {
val probe = TestProbe[Event]("evt")
val behv = Behaviors.setup[Command] { ctx
val child = ctx.spawnAnonymous(Behaviors.setup[Command] { _
val behv = Behaviors.setup[Command] { context
val child = context.spawnAnonymous(Behaviors.setup[Command] { _
probe.ref ! Started
throw new RuntimeException("simulated exc from factory") with NoStackTrace
})
ctx.watch(child)
context.watch(child)
Behaviors.receive[Command]((_, _) Behaviors.same).receiveSignal {
case (_, Terminated(`child`))
probe.ref ! Pong
@ -77,9 +77,9 @@ class DeferredSpec extends ScalaTestWithActorTestKit(
"must stop when deferred result it Stopped" in {
val probe = TestProbe[Event]("evt")
val behv = Behaviors.setup[Command] { ctx
val child = ctx.spawnAnonymous(Behaviors.setup[Command](_ Behaviors.stopped))
ctx.watch(child)
val behv = Behaviors.setup[Command] { context
val child = context.spawnAnonymous(Behaviors.setup[Command](_ Behaviors.stopped))
context.watch(child)
Behaviors.receive[Command]((_, _) Behaviors.same).receiveSignal {
case (_, Terminated(`child`))
probe.ref ! Pong

View file

@ -32,15 +32,15 @@ class InterceptSpec extends ScalaTestWithActorTestKit(
implicit val untypedSystem = system.toUntyped
private def snitchingInterceptor(probe: ActorRef[String]) = new BehaviorInterceptor[String, String] {
override def aroundReceive(ctx: ActorContext[String], msg: String, target: ReceiveTarget[String]): Behavior[String] = {
probe ! ("before " + msg)
val b = target(ctx, msg)
probe ! ("after " + msg)
override def aroundReceive(context: ActorContext[String], message: String, target: ReceiveTarget[String]): Behavior[String] = {
probe ! ("before " + message)
val b = target(context, message)
probe ! ("after " + message)
b
}
override def aroundSignal(ctx: ActorContext[String], signal: Signal, target: SignalTarget[String]): Behavior[String] = {
target(ctx, signal)
override def aroundSignal(context: ActorContext[String], signal: Signal, target: SignalTarget[String]): Behavior[String] = {
target(context, signal)
}
// keeping the instance equality as "isSame" for these
@ -176,19 +176,19 @@ class InterceptSpec extends ScalaTestWithActorTestKit(
"allow an interceptor to replace started behavior" in {
val interceptor = new BehaviorInterceptor[String, String] {
override def aroundStart(ctx: ActorContext[String], target: PreStartTarget[String]): Behavior[String] = {
override def aroundStart(context: ActorContext[String], target: PreStartTarget[String]): Behavior[String] = {
Behaviors.stopped
}
def aroundReceive(ctx: ActorContext[String], msg: String, target: ReceiveTarget[String]): Behavior[String] =
target(ctx, msg)
def aroundReceive(context: ActorContext[String], message: String, target: ReceiveTarget[String]): Behavior[String] =
target(context, message)
def aroundSignal(ctx: ActorContext[String], signal: Signal, target: SignalTarget[String]): Behavior[String] =
target(ctx, signal)
def aroundSignal(context: ActorContext[String], signal: Signal, target: SignalTarget[String]): Behavior[String] =
target(context, signal)
}
val innerBehaviorStarted = new AtomicBoolean(false)
val ref = spawn(Behaviors.intercept(interceptor)(Behaviors.setup { ctx
val ref = spawn(Behaviors.intercept(interceptor)(Behaviors.setup { context
innerBehaviorStarted.set(true)
Behaviors.unhandled[String]
}))
@ -282,15 +282,15 @@ class InterceptSpec extends ScalaTestWithActorTestKit(
}
val poisonInterceptor = new BehaviorInterceptor[Any, Msg] {
override def aroundReceive(ctx: ActorContext[Any], msg: Any, target: ReceiveTarget[Msg]): Behavior[Msg] =
msg match {
override def aroundReceive(context: ActorContext[Any], message: Any, target: ReceiveTarget[Msg]): Behavior[Msg] =
message match {
case MyPoisonPill Behaviors.stopped
case m: Msg target(ctx, m)
case m: Msg target(context, m)
case _ Behaviors.unhandled
}
override def aroundSignal(ctx: ActorContext[Any], signal: Signal, target: SignalTarget[Msg]): Behavior[Msg] =
target.apply(ctx, signal)
override def aroundSignal(context: ActorContext[Any], signal: Signal, target: SignalTarget[Msg]): Behavior[Msg] =
target.apply(context, signal)
}

View file

@ -16,7 +16,7 @@ class MonitorSpec extends ScalaTestWithActorTestKit with WordSpecLike {
"monitor messages" in {
val probe = TestProbe[String]()
val beh: Behavior[String] = Behaviors.monitor(probe.ref, Behaviors.receiveMessage(msg
val beh: Behavior[String] = Behaviors.monitor(probe.ref, Behaviors.receiveMessage(message
Behaviors.same
))
val ref: ActorRef[String] = spawn(beh)
@ -35,7 +35,7 @@ class MonitorSpec extends ScalaTestWithActorTestKit with WordSpecLike {
val beh: Behavior[String] =
monitor(
monitor(
Behaviors.receiveMessage(msg
Behaviors.receiveMessage(message
Behaviors.same
)
)
@ -55,7 +55,7 @@ class MonitorSpec extends ScalaTestWithActorTestKit with WordSpecLike {
Behaviors.monitor(probe.ref, beh)
def next: Behavior[String] =
monitor(Behaviors.receiveMessage(msg next))
monitor(Behaviors.receiveMessage(message next))
val ref: ActorRef[String] = spawn(next)
ref ! "message 1"

View file

@ -36,12 +36,12 @@ object SupervisionSpec {
case object Started extends Event
case object StartFailed extends Event
class Exc1(msg: String = "exc-1") extends RuntimeException(msg) with NoStackTrace
class Exc1(message: String = "exc-1") extends RuntimeException(message) with NoStackTrace
class Exc2 extends Exc1("exc-2")
class Exc3(msg: String = "exc-3") extends RuntimeException(msg) with NoStackTrace
class Exc3(message: String = "exc-3") extends RuntimeException(message) with NoStackTrace
def targetBehavior(monitor: ActorRef[Event], state: State = State(0, Map.empty)): Behavior[Command] =
receive[Command] { (ctx, cmd)
receive[Command] { (context, cmd)
cmd match {
case Ping
monitor ! Pong
@ -49,11 +49,11 @@ object SupervisionSpec {
case IncrementState
targetBehavior(monitor, state.copy(n = state.n + 1))
case GetState
val reply = state.copy(children = ctx.children.map(c c.path.name c.upcast[Command]).toMap)
val reply = state.copy(children = context.children.map(c c.path.name c.upcast[Command]).toMap)
monitor ! reply
Behaviors.same
case CreateChild(childBehv, childName)
ctx.spawn(childBehv, childName)
context.spawn(childBehv, childName)
Behaviors.same
case Throw(e)
throw e
@ -68,7 +68,7 @@ object SupervisionSpec {
monitor ! Started
throw new RuntimeException("simulated exc from constructor") with NoStackTrace
override def onMessage(msg: Command): Behavior[Command] = {
override def onMessage(message: Command): Behavior[Command] = {
monitor ! Pong
Behaviors.same
}
@ -262,7 +262,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit(
if (failCounter.getAndIncrement() < failCount) {
throw TE("simulated exc from constructor")
}
override def onMessage(msg: Command): Behavior[Command] = {
override def onMessage(message: Command): Behavior[Command] = {
monitor ! Pong
Behaviors.same
}
@ -746,8 +746,8 @@ class SupervisionSpec extends ScalaTestWithActorTestKit(
"work with nested supervisions and defers" in {
val strategy = SupervisorStrategy.restartWithLimit(3, 1.second)
val probe = TestProbe[AnyRef]("p")
val beh = supervise[String](setup(ctx
supervise[String](setup { ctx
val beh = supervise[String](setup(context
supervise[String](setup { context
probe.ref ! Started
scaladsl.Behaviors.empty[String]
}).onFailure[RuntimeException](strategy)
@ -805,18 +805,18 @@ class SupervisionSpec extends ScalaTestWithActorTestKit(
// irrelevant for test case but needed to use intercept in the pyramid of doom below
val whateverInterceptor = new BehaviorInterceptor[String, String] {
// identity intercept
override def aroundReceive(ctx: ActorContext[String], msg: String, target: ReceiveTarget[String]): Behavior[String] =
target(ctx, msg)
override def aroundReceive(context: ActorContext[String], message: String, target: ReceiveTarget[String]): Behavior[String] =
target(context, message)
override def aroundSignal(ctx: ActorContext[String], signal: Signal, target: SignalTarget[String]): Behavior[String] =
target(ctx, signal)
override def aroundSignal(context: ActorContext[String], signal: Signal, target: SignalTarget[String]): Behavior[String] =
target(context, signal)
}
val behv = supervise[String](Behaviors.receiveMessage {
case "boom" throw TE("boom indeed")
case "switch"
supervise[String](
setup(ctx
setup(context
supervise[String](
Behaviors.intercept(whateverInterceptor)(
supervise[String](
@ -854,7 +854,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit(
val probe = TestProbe[AnyRef]("probeMcProbeFace")
val restartCount = new AtomicInteger(0)
val behv = supervise[String](
Behaviors.setup { ctx
Behaviors.setup { context
// a bit superficial, but just to be complete
if (restartCount.incrementAndGet() == 1) {
@ -906,15 +906,15 @@ class SupervisionSpec extends ScalaTestWithActorTestKit(
"be able to recover from a DeathPactException" in {
val probe = TestProbe[AnyRef]()
val actor = spawn(Behaviors.supervise(Behaviors.setup[String] { ctx
val child = ctx.spawnAnonymous(Behaviors.receive[String] { (ctx, msg)
msg match {
val actor = spawn(Behaviors.supervise(Behaviors.setup[String] { context
val child = context.spawnAnonymous(Behaviors.receive[String] { (context, message)
message match {
case "boom"
probe.ref ! ctx.self
probe.ref ! context.self
Behaviors.stopped
}
})
ctx.watch(child)
context.watch(child)
Behaviors.receiveMessage {
case "boom"

View file

@ -51,7 +51,7 @@ class TimerSpec extends ScalaTestWithActorTestKit(
target(monitor, timer, nextCount)
}
Behaviors.receive[Command] { (ctx, cmd)
Behaviors.receive[Command] { (context, cmd)
cmd match {
case Tick(n)
monitor ! Tock(n)
@ -74,10 +74,10 @@ class TimerSpec extends ScalaTestWithActorTestKit(
throw e
}
} receiveSignal {
case (ctx, PreRestart)
case (context, PreRestart)
monitor ! GotPreRestart(timer.isTimerActive("T"))
Behaviors.same
case (ctx, PostStop)
case (context, PostStop)
monitor ! GotPostStop(timer.isTimerActive("T"))
Behaviors.same
}
@ -152,7 +152,7 @@ class TimerSpec extends ScalaTestWithActorTestKit(
case _: Tock FishingOutcomes.continue
// but we know that after we saw Cancelled we won't see any more
case Cancelled FishingOutcomes.complete
case msg FishingOutcomes.fail(s"unexpected msg: $msg")
case message FishingOutcomes.fail(s"unexpected message: $message")
}
probe.expectNoMessage(interval + 100.millis.dilated)
@ -241,13 +241,13 @@ class TimerSpec extends ScalaTestWithActorTestKit(
"allow for nested timers" in {
val probe = TestProbe[String]()
val ref = spawn(Behaviors.withTimers[String] { outerTimer
outerTimer.startPeriodicTimer("outer-key", "outer-msg", 50.millis)
outerTimer.startPeriodicTimer("outer-key", "outer-message", 50.millis)
Behaviors.withTimers { innerTimer
innerTimer.startPeriodicTimer("inner-key", "inner-msg", 50.millis)
Behaviors.receiveMessage { msg
if (msg == "stop") Behaviors.stopped
innerTimer.startPeriodicTimer("inner-key", "inner-message", 50.millis)
Behaviors.receiveMessage { message
if (message == "stop") Behaviors.stopped
else {
probe.ref ! msg
probe.ref ! message
Behaviors.same
}
}
@ -256,8 +256,8 @@ class TimerSpec extends ScalaTestWithActorTestKit(
var seen = Set.empty[String]
probe.fishForMessage(500.millis) {
case msg
seen += msg
case message
seen += message
if (seen.size == 2) FishingOutcomes.complete
else FishingOutcomes.continue
}
@ -268,11 +268,11 @@ class TimerSpec extends ScalaTestWithActorTestKit(
"keep timers when behavior changes" in {
val probe = TestProbe[String]()
def newBehavior(n: Int): Behavior[String] = Behaviors.withTimers[String] { timers
timers.startPeriodicTimer(s"key${n}", s"msg${n}", 50.milli)
Behaviors.receiveMessage { msg
if (msg == "stop") Behaviors.stopped
timers.startPeriodicTimer(s"key${n}", s"message${n}", 50.milli)
Behaviors.receiveMessage { message
if (message == "stop") Behaviors.stopped
else {
probe.ref ! msg
probe.ref ! message
newBehavior(n + 1)
}
}
@ -281,8 +281,8 @@ class TimerSpec extends ScalaTestWithActorTestKit(
val ref = spawn(newBehavior(1))
var seen = Set.empty[String]
probe.fishForMessage(500.millis) {
case msg
seen += msg
case message
seen += message
if (seen.size == 2) FishingOutcomes.complete
else FishingOutcomes.continue
}
@ -293,7 +293,7 @@ class TimerSpec extends ScalaTestWithActorTestKit(
"not grow stack when nesting withTimers" in {
def next(n: Int, probe: ActorRef[Array[StackTraceElement]]): Behavior[String] = Behaviors.withTimers { timers
timers.startSingleTimer("key", "tick", 1.millis)
Behaviors.receiveMessage { msg
Behaviors.receiveMessage { message
if (n == 20) {
val e = new RuntimeException().fillInStackTrace()
val trace = e.getStackTrace

View file

@ -29,7 +29,7 @@ object WatchSpec {
}
val mutableTerminatorBehavior = new AbstractBehavior[Stop.type] {
override def onMessage(msg: Stop.type) = msg match {
override def onMessage(message: Stop.type) = message match {
case Stop Behaviors.stopped
}
}
@ -39,7 +39,7 @@ object WatchSpec {
case object CustomTerminationMessage extends CustomTerminationMessage
case object CustomTerminationMessage2 extends CustomTerminationMessage
case class StartWatching(watchee: ActorRef[Stop.type]) extends Message
case class StartWatchingWith(watchee: ActorRef[Stop.type], msg: CustomTerminationMessage) extends Message
case class StartWatchingWith(watchee: ActorRef[Stop.type], message: CustomTerminationMessage) extends Message
}
class WatchSpec extends ScalaTestWithActorTestKit(WatchSpec.config) with WordSpecLike {
@ -58,8 +58,8 @@ class WatchSpec extends ScalaTestWithActorTestKit(WatchSpec.config) with WordSpe
val watcher = spawn(
Behaviors.supervise(
Behaviors.receive[StartWatching] {
case (ctx, StartWatching(watchee))
ctx.watch(watchee)
case (context, StartWatching(watchee))
context.watch(watchee)
watchProbe.ref ! Done
Behaviors.same
}.receiveSignal {
@ -85,14 +85,14 @@ class WatchSpec extends ScalaTestWithActorTestKit(WatchSpec.config) with WordSpe
case class Failed(t: Terminated) // we need to wrap it as it is handled specially
val probe = TestProbe[Any]()
val ex = new TestException("boom")
val parent = spawn(Behaviors.setup[Any] { ctx
val child = ctx.spawn(Behaviors.receive[Any]((ctx, msg)
val parent = spawn(Behaviors.setup[Any] { context
val child = context.spawn(Behaviors.receive[Any]((context, message)
throw ex
), "child")
ctx.watch(child)
context.watch(child)
Behaviors.receive[Any] { (ctx, msg)
child ! msg
Behaviors.receive[Any] { (context, message)
child ! message
Behaviors.same
}.receiveSignal {
case (_, t: Terminated)
@ -111,23 +111,23 @@ class WatchSpec extends ScalaTestWithActorTestKit(WatchSpec.config) with WordSpe
case class Failed(t: Terminated) // we need to wrap it as it is handled specially
val probe = TestProbe[Any]()
val ex = new TestException("boom")
val grossoBosso = spawn(Behaviors.setup[Any] { ctx
val middleManagement = ctx.spawn(Behaviors.setup[Any] { ctx
val sixPackJoe = ctx.spawn(Behaviors.receive[Any]((ctx, msg)
val grossoBosso = spawn(Behaviors.setup[Any] { context
val middleManagement = context.spawn(Behaviors.setup[Any] { context
val sixPackJoe = context.spawn(Behaviors.receive[Any]((context, message)
throw ex
), "joe")
ctx.watch(sixPackJoe)
context.watch(sixPackJoe)
Behaviors.receive[Any] { (ctx, msg)
sixPackJoe ! msg
Behaviors.receive[Any] { (context, message)
sixPackJoe ! message
Behaviors.same
} // no handling of terminated, even though we watched!!!
}, "middle-management")
ctx.watch(middleManagement)
context.watch(middleManagement)
Behaviors.receive[Any] { (ctx, msg)
middleManagement ! msg
Behaviors.receive[Any] { (context, message)
middleManagement ! message
Behaviors.same
}.receiveSignal {
case (_, t: Terminated)
@ -165,12 +165,12 @@ class WatchSpec extends ScalaTestWithActorTestKit(WatchSpec.config) with WordSpe
val watcher = spawn(
Behaviors.supervise(
Behaviors.receive[Message] {
case (ctx, StartWatchingWith(watchee, msg))
ctx.watchWith(watchee, msg)
case (context, StartWatchingWith(watchee, message))
context.watchWith(watchee, message)
watchProbe.ref ! Done
Behaviors.same
case (_, msg)
receivedTerminationSignal.success(msg)
case (_, message)
receivedTerminationSignal.success(message)
Behaviors.stopped
}).onFailure[Throwable](SupervisorStrategy.stop)
)
@ -201,16 +201,16 @@ class WatchSpec extends ScalaTestWithActorTestKit(WatchSpec.config) with WordSpe
val watcher = spawn(
Behaviors.supervise(
Behaviors.receive[Message] {
case (ctx, StartWatching(watchee))
ctx.watch(watchee)
case (context, StartWatching(watchee))
context.watch(watchee)
Behaviors.same
case (ctx, StartWatchingWith(watchee, msg))
ctx.unwatch(watchee)
ctx.watchWith(watchee, msg)
case (context, StartWatchingWith(watchee, message))
context.unwatch(watchee)
context.watchWith(watchee, message)
watchProbe.ref ! Done
Behaviors.same
case (_, msg)
receivedTerminationSignal.success(msg)
case (_, message)
receivedTerminationSignal.success(message)
Behaviors.stopped
}).onFailure[Throwable](SupervisorStrategy.stop)
)
@ -231,13 +231,13 @@ class WatchSpec extends ScalaTestWithActorTestKit(WatchSpec.config) with WordSpe
val watcher = spawn(
Behaviors.supervise(
Behaviors.receive[Message] {
case (ctx, StartWatchingWith(watchee, msg))
ctx.unwatch(watchee)
ctx.watchWith(watchee, msg)
case (context, StartWatchingWith(watchee, message))
context.unwatch(watchee)
context.watchWith(watchee, message)
watchProbe.ref ! Done
Behaviors.same
case (_, msg)
receivedTerminationSignal.success(msg)
case (_, message)
receivedTerminationSignal.success(message)
Behaviors.stopped
}).onFailure[Throwable](SupervisorStrategy.stop)
)
@ -257,13 +257,13 @@ class WatchSpec extends ScalaTestWithActorTestKit(WatchSpec.config) with WordSpe
val watcher = spawn(
Behaviors.supervise(
Behaviors.receive[Message] {
case (ctx, StartWatchingWith(watchee, msg))
ctx.watchWith(watchee, msg)
case (context, StartWatchingWith(watchee, message))
context.watchWith(watchee, message)
Behaviors.same
case (ctx, StartWatching(watchee))
ctx.watch(watchee)
case (context, StartWatching(watchee))
context.watch(watchee)
Behaviors.same
case (_, msg)
case (_, message)
Behaviors.stopped
}.receiveSignal {
case (_, PostStop)

View file

@ -24,8 +24,8 @@ class WidenSpec extends ScalaTestWithActorTestKit(
implicit val untypedSystem = system.toUntyped
def intToString(probe: ActorRef[String]): Behavior[Int] = {
Behaviors.receiveMessage[String] { msg
probe ! msg
Behaviors.receiveMessage[String] { message
probe ! message
Behaviors.same
}.widen[Int] {
case n if n != 13 n.toString
@ -72,8 +72,8 @@ class WidenSpec extends ScalaTestWithActorTestKit(
val beh =
widen(
widen(
Behaviors.receiveMessage[String] { msg
probe.ref ! msg
Behaviors.receiveMessage[String] { message
probe.ref ! message
Behaviors.same
}
)
@ -101,8 +101,8 @@ class WidenSpec extends ScalaTestWithActorTestKit(
def next: Behavior[String] =
widen(
Behaviors.receiveMessage[String] { msg
probe.ref ! msg
Behaviors.receiveMessage[String] { message
probe.ref ! message
next
}
)
@ -131,7 +131,7 @@ class WidenSpec extends ScalaTestWithActorTestKit(
val ref = spawn(
widen(
widen(
Behaviors.receiveMessage[String] { msg
Behaviors.receiveMessage[String] { message
Behaviors.same
}
)

View file

@ -23,7 +23,7 @@ class ActorSystemSpec extends WordSpec with Matchers with BeforeAndAfterAll
def system[T](behavior: Behavior[T], name: String) = ActorSystem(behavior, name)
def suite = "adapter"
case class Probe(msg: String, replyTo: ActorRef[String])
case class Probe(message: String, replyTo: ActorRef[String])
def withSystem[T](name: String, behavior: Behavior[T], doTerminate: Boolean = true)(block: ActorSystem[T] Unit): Terminated = {
val sys = system(behavior, s"$suite-$name")
@ -41,7 +41,7 @@ class ActorSystemSpec extends WordSpec with Matchers with BeforeAndAfterAll
"start the guardian actor and terminate when it terminates" in {
val t = withSystem(
"a",
Behaviors.receive[Probe] { case (_, p) p.replyTo ! p.msg; Behaviors.stopped }, doTerminate = false) { sys
Behaviors.receive[Probe] { case (_, p) p.replyTo ! p.message; Behaviors.stopped }, doTerminate = false) { sys
val inbox = TestInbox[String]("a")
sys ! Probe("hello", inbox.ref)
eventually {
@ -58,7 +58,7 @@ class ActorSystemSpec extends WordSpec with Matchers with BeforeAndAfterAll
"shutdown if guardian shuts down immediately" in {
val stoppable =
Behaviors.receive[Done] {
case (ctx, Done) Behaviors.stopped
case (context, Done) Behaviors.stopped
}
withSystem("shutdown", stoppable, doTerminate = false) { sys: ActorSystem[Done]
sys ! Done

View file

@ -27,8 +27,8 @@ object LocalReceptionistSpec {
val behaviorB = Behaviors.empty[ServiceB]
case object Stop extends ServiceA with ServiceB
val stoppableBehavior = Behaviors.receive[Any] { (_, msg)
msg match {
val stoppableBehavior = Behaviors.receive[Any] { (_, message)
message match {
case Stop Behavior.stopped
case _ Behavior.same
}

View file

@ -46,29 +46,29 @@ object ReceptionistApiSpec {
instances.foreach(_ ! "woho")
}
Behaviors.setup[Any] { ctx
Behaviors.setup[Any] { context
// oneoff ask inside of actor, this should be a rare use case
ctx.ask(system.receptionist)(Receptionist.Find(key)) {
context.ask(system.receptionist)(Receptionist.Find(key)) {
case Success(key.Listing(services)) services // Set[ActorRef[String]] !!
case _ "unexpected"
}
// this is a more "normal" use case which is clean
ctx.system.receptionist ! Receptionist.Subscribe(key, ctx.self.narrow)
context.system.receptionist ! Receptionist.Subscribe(key, context.self.narrow)
// another more "normal" is subscribe using an adapter
// FIXME inference doesn't work with partial function
val adapter = ctx.spawnMessageAdapter { listing: Receptionist.Listing
val adapter = context.spawnMessageAdapter { listing: Receptionist.Listing
listing.serviceInstances(key) // Set[ActorRef[String]] !!
}
ctx.system.receptionist ! Receptionist.Subscribe(key, adapter)
context.system.receptionist ! Receptionist.Subscribe(key, adapter)
// ofc this doesn't make sense to do in the same actor, this is just
// to cover as much of the API as possible
ctx.system.receptionist ! Receptionist.Register(key, ctx.self.narrow, ctx.self.narrow)
context.system.receptionist ! Receptionist.Register(key, context.self.narrow, context.self.narrow)
Behaviors.receive { (ctx, msg)
msg match {
Behaviors.receive { (context, message)
message match {
case key.Listing(services)
services.foreach(_ ! "woho")
Behaviors.same

View file

@ -42,19 +42,19 @@ class ActorContextAskSpec extends ScalaTestWithActorTestKit(ActorContextAskSpec.
case class Ping(sender: ActorRef[Pong])
case class Pong(selfName: String, threadName: String)
val pingPong = spawn(Behaviors.receive[Ping] { (ctx, msg)
msg.sender ! Pong(ctx.self.path.name, Thread.currentThread().getName)
val pingPong = spawn(Behaviors.receive[Ping] { (context, message)
message.sender ! Pong(context.self.path.name, Thread.currentThread().getName)
Behaviors.same
}, "ping-pong", Props.empty.withDispatcherFromConfig("ping-pong-dispatcher"))
val probe = TestProbe[Pong]()
val snitch = Behaviors.setup[Pong] { ctx
val snitch = Behaviors.setup[Pong] { context
// Timeout comes from TypedAkkaSpec
ctx.ask(pingPong)(Ping) {
case Success(_) Pong(ctx.self.path.name + "1", Thread.currentThread().getName)
context.ask(pingPong)(Ping) {
case Success(_) Pong(context.self.path.name + "1", Thread.currentThread().getName)
case Failure(ex) throw ex
}
@ -79,23 +79,23 @@ class ActorContextAskSpec extends ScalaTestWithActorTestKit(ActorContextAskSpec.
case class Ping(respondTo: ActorRef[Pong.type]) extends Protocol
case object Pong extends Protocol
val pingPong = spawn(Behaviors.receive[Protocol]((_, msg)
msg match {
val pingPong = spawn(Behaviors.receive[Protocol]((_, message)
message match {
case Ping(respondTo)
respondTo ! Pong
Behaviors.same
}
))
val snitch = Behaviors.setup[AnyRef] { ctx
ctx.ask(pingPong)(Ping) {
case Success(msg) throw new NotImplementedError(msg.toString)
case Failure(x) x
val snitch = Behaviors.setup[AnyRef] { context
context.ask(pingPong)(Ping) {
case Success(message) throw new NotImplementedError(message.toString)
case Failure(x) x
}
Behaviors.receive[AnyRef] {
case (_, msg)
probe.ref ! msg
case (_, message)
probe.ref ! message
Behaviors.same
}.receiveSignal {
@ -115,15 +115,15 @@ class ActorContextAskSpec extends ScalaTestWithActorTestKit(ActorContextAskSpec.
"deal with timeouts in ask" in {
val probe = TestProbe[AnyRef]()
val snitch = Behaviors.setup[AnyRef] { ctx
val snitch = Behaviors.setup[AnyRef] { context
ctx.ask[String, String](system.deadLetters)(ref "boo") {
context.ask[String, String](system.deadLetters)(ref "boo") {
case Success(m) m
case Failure(x) x
}(10.millis, implicitly[ClassTag[String]])
Behaviors.receiveMessage { msg
probe.ref ! msg
Behaviors.receiveMessage { message
probe.ref ! message
Behaviors.same
}
}
@ -141,15 +141,15 @@ class ActorContextAskSpec extends ScalaTestWithActorTestKit(ActorContextAskSpec.
"must timeout if recipient doesn't reply in time" in {
val target = spawn(Behaviors.ignore[String])
val probe = TestProbe[AnyRef]()
val snitch = Behaviors.setup[AnyRef] { ctx
val snitch = Behaviors.setup[AnyRef] { context
ctx.ask[String, String](target)(_ "bar") {
context.ask[String, String](target)(_ "bar") {
case Success(m) m
case Failure(x) x
}(10.millis, implicitly[ClassTag[String]])
Behaviors.receiveMessage { msg
probe.ref ! msg
Behaviors.receiveMessage { message
probe.ref ! message
Behaviors.same
}
}

View file

@ -27,13 +27,13 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
"Logging in a typed actor" must {
"be conveniently available from the ctx" in {
"be conveniently available from the context" in {
val actor = EventFilter.info("Started", source = "akka://ActorLoggingSpec/user/the-actor", occurrences = 1).intercept {
spawn(Behaviors.setup[String] { ctx
ctx.log.info("Started")
spawn(Behaviors.setup[String] { context
context.log.info("Started")
Behaviors.receive { (ctx, msg)
ctx.log.info("got message {}", msg)
Behaviors.receive { (context, message)
context.log.info("got message {}", message)
Behaviors.same
}
}, "the-actor")
@ -48,14 +48,14 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
EventFilter.custom({
case event: LogEventWithMarker if event.marker == marker true
}, occurrences = 9).intercept(
spawn(Behaviors.setup[Any] { ctx
ctx.log.debug(marker, "whatever")
ctx.log.info(marker, "whatever")
ctx.log.warning(marker, "whatever")
ctx.log.error(marker, "whatever")
ctx.log.error(marker, cause, "whatever")
spawn(Behaviors.setup[Any] { context
context.log.debug(marker, "whatever")
context.log.info(marker, "whatever")
context.log.warning(marker, "whatever")
context.log.error(marker, "whatever")
context.log.error(marker, cause, "whatever")
Logging.AllLogLevels.foreach(level {
ctx.log.log(level, marker, "whatever")
context.log.log(level, marker, "whatever")
})
Behaviors.stopped
})
@ -66,9 +66,9 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
EventFilter.custom({
case event: LogEventWithCause if event.cause == cause true
}, occurrences = 2).intercept(
spawn(Behaviors.setup[Any] { ctx
ctx.log.warning(cause, "whatever")
ctx.log.warning(marker, cause, "whatever")
spawn(Behaviors.setup[Any] { context
context.log.warning(cause, "whatever")
context.log.warning(marker, cause, "whatever")
Behaviors.stopped
})
)
@ -81,99 +81,99 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
EventFilter.custom({
case _ true // any is fine, we're just after the right count of statements reaching the listener
}, occurrences = 120).intercept {
spawn(Behaviors.setup[String] { ctx
ctx.log.debug("message")
ctx.log.debug("{}", "arg1")
ctx.log.debug("{} {}", "arg1", "arg2")
ctx.log.debug("{} {} {}", "arg1", "arg2", "arg3")
ctx.log.debug("{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
ctx.log.debug("{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
ctx.log.debug(marker, "message")
ctx.log.debug(marker, "{}", "arg1")
ctx.log.debug(marker, "{} {}", "arg1", "arg2")
ctx.log.debug(marker, "{} {} {}", "arg1", "arg2", "arg3")
ctx.log.debug(marker, "{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
ctx.log.debug(marker, "{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
spawn(Behaviors.setup[String] { context
context.log.debug("message")
context.log.debug("{}", "arg1")
context.log.debug("{} {}", "arg1", "arg2")
context.log.debug("{} {} {}", "arg1", "arg2", "arg3")
context.log.debug("{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
context.log.debug("{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
context.log.debug(marker, "message")
context.log.debug(marker, "{}", "arg1")
context.log.debug(marker, "{} {}", "arg1", "arg2")
context.log.debug(marker, "{} {} {}", "arg1", "arg2", "arg3")
context.log.debug(marker, "{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
context.log.debug(marker, "{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
ctx.log.info("message")
ctx.log.info("{}", "arg1")
ctx.log.info("{} {}", "arg1", "arg2")
ctx.log.info("{} {} {}", "arg1", "arg2", "arg3")
ctx.log.info("{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
ctx.log.info("{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
ctx.log.info(marker, "message")
ctx.log.info(marker, "{}", "arg1")
ctx.log.info(marker, "{} {}", "arg1", "arg2")
ctx.log.info(marker, "{} {} {}", "arg1", "arg2", "arg3")
ctx.log.info(marker, "{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
ctx.log.info(marker, "{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
context.log.info("message")
context.log.info("{}", "arg1")
context.log.info("{} {}", "arg1", "arg2")
context.log.info("{} {} {}", "arg1", "arg2", "arg3")
context.log.info("{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
context.log.info("{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
context.log.info(marker, "message")
context.log.info(marker, "{}", "arg1")
context.log.info(marker, "{} {}", "arg1", "arg2")
context.log.info(marker, "{} {} {}", "arg1", "arg2", "arg3")
context.log.info(marker, "{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
context.log.info(marker, "{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
ctx.log.warning("message")
ctx.log.warning("{}", "arg1")
ctx.log.warning("{} {}", "arg1", "arg2")
ctx.log.warning("{} {} {}", "arg1", "arg2", "arg3")
ctx.log.warning("{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
ctx.log.warning("{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
ctx.log.warning(marker, "message")
ctx.log.warning(marker, "{}", "arg1")
ctx.log.warning(marker, "{} {}", "arg1", "arg2")
ctx.log.warning(marker, "{} {} {}", "arg1", "arg2", "arg3")
ctx.log.warning(marker, "{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
ctx.log.warning(marker, "{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
context.log.warning("message")
context.log.warning("{}", "arg1")
context.log.warning("{} {}", "arg1", "arg2")
context.log.warning("{} {} {}", "arg1", "arg2", "arg3")
context.log.warning("{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
context.log.warning("{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
context.log.warning(marker, "message")
context.log.warning(marker, "{}", "arg1")
context.log.warning(marker, "{} {}", "arg1", "arg2")
context.log.warning(marker, "{} {} {}", "arg1", "arg2", "arg3")
context.log.warning(marker, "{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
context.log.warning(marker, "{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
ctx.log.warning(cause, "message")
ctx.log.warning(cause, "{}", "arg1")
ctx.log.warning(cause, "{} {}", "arg1", "arg2")
ctx.log.warning(cause, "{} {} {}", "arg1", "arg2", "arg3")
ctx.log.warning(cause, "{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
ctx.log.warning(cause, "{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
ctx.log.warning(marker, cause, "message")
ctx.log.warning(marker, cause, "{}", "arg1")
ctx.log.warning(marker, cause, "{} {}", "arg1", "arg2")
ctx.log.warning(marker, cause, "{} {} {}", "arg1", "arg2", "arg3")
ctx.log.warning(marker, cause, "{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
ctx.log.warning(marker, cause, "{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
context.log.warning(cause, "message")
context.log.warning(cause, "{}", "arg1")
context.log.warning(cause, "{} {}", "arg1", "arg2")
context.log.warning(cause, "{} {} {}", "arg1", "arg2", "arg3")
context.log.warning(cause, "{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
context.log.warning(cause, "{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
context.log.warning(marker, cause, "message")
context.log.warning(marker, cause, "{}", "arg1")
context.log.warning(marker, cause, "{} {}", "arg1", "arg2")
context.log.warning(marker, cause, "{} {} {}", "arg1", "arg2", "arg3")
context.log.warning(marker, cause, "{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
context.log.warning(marker, cause, "{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
ctx.log.error("message")
ctx.log.error("{}", "arg1")
ctx.log.error("{} {}", "arg1", "arg2")
ctx.log.error("{} {} {}", "arg1", "arg2", "arg3")
ctx.log.error("{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
ctx.log.error("{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
ctx.log.error(marker, "message")
ctx.log.error(marker, "{}", "arg1")
ctx.log.error(marker, "{} {}", "arg1", "arg2")
ctx.log.error(marker, "{} {} {}", "arg1", "arg2", "arg3")
ctx.log.error(marker, "{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
ctx.log.error(marker, "{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
context.log.error("message")
context.log.error("{}", "arg1")
context.log.error("{} {}", "arg1", "arg2")
context.log.error("{} {} {}", "arg1", "arg2", "arg3")
context.log.error("{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
context.log.error("{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
context.log.error(marker, "message")
context.log.error(marker, "{}", "arg1")
context.log.error(marker, "{} {}", "arg1", "arg2")
context.log.error(marker, "{} {} {}", "arg1", "arg2", "arg3")
context.log.error(marker, "{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
context.log.error(marker, "{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
ctx.log.error(cause, "message")
ctx.log.error(cause, "{}", "arg1")
ctx.log.error(cause, "{} {}", "arg1", "arg2")
ctx.log.error(cause, "{} {} {}", "arg1", "arg2", "arg3")
ctx.log.error(cause, "{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
ctx.log.error(cause, "{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
ctx.log.error(marker, cause, "message")
ctx.log.error(marker, cause, "{}", "arg1")
ctx.log.error(marker, cause, "{} {}", "arg1", "arg2")
ctx.log.error(marker, cause, "{} {} {}", "arg1", "arg2", "arg3")
ctx.log.error(marker, cause, "{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
ctx.log.error(marker, cause, "{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
context.log.error(cause, "message")
context.log.error(cause, "{}", "arg1")
context.log.error(cause, "{} {}", "arg1", "arg2")
context.log.error(cause, "{} {} {}", "arg1", "arg2", "arg3")
context.log.error(cause, "{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
context.log.error(cause, "{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
context.log.error(marker, cause, "message")
context.log.error(marker, cause, "{}", "arg1")
context.log.error(marker, cause, "{} {}", "arg1", "arg2")
context.log.error(marker, cause, "{} {} {}", "arg1", "arg2", "arg3")
context.log.error(marker, cause, "{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
context.log.error(marker, cause, "{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
Logging.AllLogLevels.foreach(level {
ctx.log.log(level, "message")
ctx.log.log(level, "{}", "arg1")
ctx.log.log(level, "{} {}", "arg1", "arg2")
ctx.log.log(level, "{} {} {}", "arg1", "arg2", "arg3")
ctx.log.log(level, "{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
ctx.log.log(level, "{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
context.log.log(level, "message")
context.log.log(level, "{}", "arg1")
context.log.log(level, "{} {}", "arg1", "arg2")
context.log.log(level, "{} {} {}", "arg1", "arg2", "arg3")
context.log.log(level, "{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
context.log.log(level, "{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
ctx.log.log(level, marker, "message")
ctx.log.log(level, marker, "{}", "arg1")
ctx.log.log(level, marker, "{} {}", "arg1", "arg2")
ctx.log.log(level, marker, "{} {} {}", "arg1", "arg2", "arg3")
ctx.log.log(level, marker, "{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
ctx.log.log(level, marker, "{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
context.log.log(level, marker, "message")
context.log.log(level, marker, "{}", "arg1")
context.log.log(level, marker, "{} {}", "arg1", "arg2")
context.log.log(level, marker, "{} {} {}", "arg1", "arg2", "arg3")
context.log.log(level, marker, "{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
context.log.log(level, marker, "{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
})
Behaviors.stopped
@ -194,18 +194,18 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
val behaviors = Behaviors.withMdc[Protocol](
Map("static" -> 1),
// FIXME why u no infer the type here Scala??
(msg: Protocol)
if (msg.transactionId == 1)
(message: Protocol)
if (message.transactionId == 1)
Map(
"txId" -> msg.transactionId,
"txId" -> message.transactionId,
"first" -> true
)
else Map("txId" -> msg.transactionId)
else Map("txId" -> message.transactionId)
) {
Behaviors.setup { ctx
ctx.log.info("Starting")
Behaviors.receiveMessage { msg
ctx.log.info("Got message!")
Behaviors.setup { context
context.log.info("Starting")
Behaviors.receiveMessage { message
context.log.info("Got message!")
Behaviors.same
}
}
@ -250,8 +250,8 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
val behavior =
Behaviors.withMdc[String](Map("outermost" -> true)) {
Behaviors.withMdc(Map("innermost" -> true)) {
Behaviors.receive { (ctx, msg)
ctx.log.info(msg)
Behaviors.receive { (context, message)
context.log.info(message)
Behaviors.same
}
}
@ -271,12 +271,12 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
"keep being applied when behavior changes to other behavior" in {
def behavior: Behavior[String] =
Behaviors.receive { (ctx, msg)
msg match {
Behaviors.receive { (context, message)
message match {
case "new-behavior"
behavior
case other
ctx.log.info(other)
context.log.info(other)
Behaviors.same
}
}
@ -311,12 +311,12 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
val id = new AtomicInteger(0)
def behavior: Behavior[String] =
Behaviors.withMdc(Map("mdc-version" -> id.incrementAndGet())) {
Behaviors.receive { (ctx, msg)
msg match {
Behaviors.receive { (context, message)
message match {
case "new-mdc"
behavior
case other
ctx.log.info(other)
context.log.info(other)
Behaviors.same
}
}
@ -347,11 +347,11 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
"provide a withMdc decorator" in {
val behavior = Behaviors.withMdc[Protocol](Map("mdc" -> "outer"))(
Behaviors.setup { ctx
Behaviors.receiveMessage { msg
ctx.log.withMdc(Map("mdc" -> "inner")).info("Got message log.withMDC!")
Behaviors.setup { context
Behaviors.receiveMessage { message
context.log.withMdc(Map("mdc" -> "inner")).info("Got message log.withMDC!")
// after log.withMdc so we know it didn't change the outer mdc
ctx.log.info("Got message behavior.withMdc!")
context.log.info("Got message behavior.withMdc!")
Behaviors.same
}
}

View file

@ -45,22 +45,22 @@ class MessageAdapterSpec extends ScalaTestWithActorTestKit(MessageAdapterSpec.co
case class AnotherPong(selfName: String, threadName: String)
val pingPong = spawn(Behaviors.receive[Ping] { (ctx, msg)
msg.sender ! Pong(ctx.self.path.name, Thread.currentThread().getName)
val pingPong = spawn(Behaviors.receive[Ping] { (context, message)
message.sender ! Pong(context.self.path.name, Thread.currentThread().getName)
Behaviors.same
}, "ping-pong", Props.empty.withDispatcherFromConfig("ping-pong-dispatcher"))
val probe = TestProbe[AnotherPong]()
val snitch = Behaviors.setup[AnotherPong] { ctx
val snitch = Behaviors.setup[AnotherPong] { context
val replyTo = ctx.messageAdapter[Response](_
AnotherPong(ctx.self.path.name, Thread.currentThread().getName))
val replyTo = context.messageAdapter[Response](_
AnotherPong(context.self.path.name, Thread.currentThread().getName))
pingPong ! Ping(replyTo)
// also verify the internal spawnMessageAdapter
val replyTo2: ActorRef[Response] = ctx.spawnMessageAdapter(_
AnotherPong(ctx.self.path.name, Thread.currentThread().getName))
val replyTo2: ActorRef[Response] = context.spawnMessageAdapter(_
AnotherPong(context.self.path.name, Thread.currentThread().getName))
pingPong ! Ping(replyTo2)
Behaviors.receiveMessage { anotherPong
@ -102,11 +102,11 @@ class MessageAdapterSpec extends ScalaTestWithActorTestKit(MessageAdapterSpec.co
val probe = TestProbe[Wrapped]()
val snitch = Behaviors.setup[Wrapped] { ctx
val snitch = Behaviors.setup[Wrapped] { context
ctx.messageAdapter[Response](pong Wrapped(qualifier = "wrong", pong)) // this is replaced
val replyTo1: ActorRef[Response] = ctx.messageAdapter(pong Wrapped(qualifier = "1", pong))
val replyTo2 = ctx.messageAdapter[Pong2](pong Wrapped(qualifier = "2", pong))
context.messageAdapter[Response](pong Wrapped(qualifier = "wrong", pong)) // this is replaced
val replyTo1: ActorRef[Response] = context.messageAdapter(pong Wrapped(qualifier = "1", pong))
val replyTo2 = context.messageAdapter[Pong2](pong Wrapped(qualifier = "2", pong))
pingPong ! Ping1(replyTo1)
pingPong ! Ping2(replyTo2)
@ -144,9 +144,9 @@ class MessageAdapterSpec extends ScalaTestWithActorTestKit(MessageAdapterSpec.co
val probe = TestProbe[Wrapped]()
val snitch = Behaviors.setup[Wrapped] { ctx
val snitch = Behaviors.setup[Wrapped] { context
val replyTo1 = ctx.messageAdapter[Pong1](pong Wrapped(qualifier = "1", pong))
val replyTo1 = context.messageAdapter[Pong1](pong Wrapped(qualifier = "1", pong))
pingPong ! Ping1(replyTo1)
// doing something terribly wrong
// Pong2 message adapter not registered
@ -180,10 +180,10 @@ class MessageAdapterSpec extends ScalaTestWithActorTestKit(MessageAdapterSpec.co
val probe = TestProbe[Any]()
val snitch = Behaviors.setup[Wrapped] { ctx
val snitch = Behaviors.setup[Wrapped] { context
var count = 0
val replyTo = ctx.messageAdapter[Pong] { pong
val replyTo = context.messageAdapter[Pong] { pong
count += 1
if (count == 3) throw new TestException("boom")
else Wrapped(count, pong)
@ -227,9 +227,9 @@ class MessageAdapterSpec extends ScalaTestWithActorTestKit(MessageAdapterSpec.co
val probe = TestProbe[Any]()
val snitch = Behaviors.setup[Wrapped] { ctx
val snitch = Behaviors.setup[Wrapped] { context
val replyTo = ctx.messageAdapter[Pong] { pong
val replyTo = context.messageAdapter[Pong] { pong
Wrapped(pong)
}
(1 to 5).foreach { _

View file

@ -11,7 +11,7 @@ import org.scalatest.{ Matchers, WordSpec }
class StashBufferSpec extends WordSpec with Matchers {
val ctx = new StubbedActorContext[String]("StashBufferSpec")
val context = new StubbedActorContext[String]("StashBufferSpec")
"A StashBuffer" must {
@ -34,12 +34,12 @@ class StashBufferSpec extends WordSpec with Matchers {
val m1 = buffer.head
m1 should ===("m1")
buffer.size should ===(2)
buffer.unstash(ctx, Behaviors.ignore, 1, identity)
buffer.unstash(context, Behaviors.ignore, 1, identity)
buffer.size should ===(1)
m1 should ===("m1")
val m2 = buffer.head
m2 should ===("m2")
buffer.unstash(ctx, Behaviors.ignore, 1, identity)
buffer.unstash(context, Behaviors.ignore, 1, identity)
buffer.size should ===(0)
intercept[NoSuchElementException] {
buffer.head
@ -70,7 +70,7 @@ class StashBufferSpec extends WordSpec with Matchers {
val sb1 = new StringBuilder()
buffer.foreach(sb1.append(_))
sb1.toString() should ===("m1m2m3")
buffer.unstash(ctx, Behaviors.ignore, 1, identity)
buffer.unstash(context, Behaviors.ignore, 1, identity)
val sb2 = new StringBuilder()
buffer.foreach(sb2.append(_))
sb2.toString() should ===("m2m3")
@ -85,16 +85,16 @@ class StashBufferSpec extends WordSpec with Matchers {
val valueInbox = TestInbox[String]()
def behavior(state: String): Behavior[String] =
Behaviors.receive[String] { (_, msg)
if (msg == "get") {
Behaviors.receive[String] { (_, message)
if (message == "get") {
valueInbox.ref ! state
Behaviors.same
} else {
behavior(state + msg)
behavior(state + message)
}
}
buffer.unstashAll(ctx, behavior(""))
buffer.unstashAll(context, behavior(""))
valueInbox.expectMessage("m1m2m3")
buffer.isEmpty should ===(true)
}
@ -108,16 +108,16 @@ class StashBufferSpec extends WordSpec with Matchers {
val valueInbox = TestInbox[String]()
def behavior(state: String): Behavior[String] =
Behaviors.receive[String] { (_, msg)
if (msg == "get") {
Behaviors.receive[String] { (_, message)
if (message == "get") {
valueInbox.ref ! state
Behaviors.same
} else {
Behaviors.setup[String](_ behavior(state + msg))
Behaviors.setup[String](_ behavior(state + message))
}
}
buffer.unstashAll(ctx, behavior(""))
buffer.unstashAll(context, behavior(""))
valueInbox.expectMessage("m1m2m3")
buffer.isEmpty should ===(true)
}
@ -131,27 +131,27 @@ class StashBufferSpec extends WordSpec with Matchers {
val valueInbox = TestInbox[String]()
def behavior(state: String): Behavior[String] =
Behaviors.receive[String] { (_, msg)
if (msg == "get") {
Behaviors.receive[String] { (_, message)
if (message == "get") {
valueInbox.ref ! state
Behaviors.same
} else if (msg == "m2") {
} else if (message == "m2") {
buffer.stash("m2")
Behaviors.same
} else {
behavior(state + msg)
behavior(state + message)
}
}
// It's only supposed to unstash the messages that are in the buffer when
// the call is made, not unstash new messages added to the buffer while
// unstashing.
val b2 = buffer.unstashAll(ctx, behavior(""))
val b2 = buffer.unstashAll(context, behavior(""))
valueInbox.expectMessage("m1m3")
buffer.size should ===(1)
buffer.head should ===("m2")
buffer.unstashAll(ctx, b2)
buffer.unstashAll(context, b2)
buffer.size should ===(1)
buffer.head should ===("m2")
}

View file

@ -25,10 +25,10 @@ object StashSpec {
val buffer = StashBuffer[Command](capacity = 10)
def active(processed: Vector[String]): Behavior[Command] =
Behaviors.receive { (ctx, cmd)
Behaviors.receive { (context, cmd)
cmd match {
case msg: Msg
active(processed :+ msg.s)
case message: Msg
active(processed :+ message.s)
case GetProcessed(replyTo)
replyTo ! processed
Behaviors.same
@ -47,10 +47,10 @@ object StashSpec {
}
def stashing(processed: Vector[String]): Behavior[Command] =
Behaviors.receive { (ctx, cmd)
Behaviors.receive { (context, cmd)
cmd match {
case msg: Msg
buffer.stash(msg)
case message: Msg
buffer.stash(message)
Behaviors.same
case g: GetProcessed
buffer.stash(g)
@ -59,16 +59,16 @@ object StashSpec {
replyTo ! buffer.size
Behaviors.same
case UnstashAll
buffer.unstashAll(ctx, active(processed))
buffer.unstashAll(context, active(processed))
case Unstash
ctx.log.debug(s"Unstash ${buffer.size}")
context.log.debug(s"Unstash ${buffer.size}")
if (buffer.isEmpty)
active(processed)
else {
ctx.self ! Unstash // continue unstashing until buffer is empty
context.self ! Unstash // continue unstashing until buffer is empty
val numberOfMessages = 2
ctx.log.debug(s"Unstash $numberOfMessages of ${buffer.size}, starting with ${buffer.head}")
buffer.unstash(ctx, unstashing(processed), numberOfMessages, Unstashed)
context.log.debug(s"Unstash $numberOfMessages of ${buffer.size}, starting with ${buffer.head}")
buffer.unstash(context, unstashing(processed), numberOfMessages, Unstashed)
}
case Stash
Behaviors.unhandled
@ -78,34 +78,34 @@ object StashSpec {
}
def unstashing(processed: Vector[String]): Behavior[Command] =
Behaviors.receive { (ctx, cmd)
Behaviors.receive { (context, cmd)
cmd match {
case Unstashed(msg: Msg)
ctx.log.debug(s"unstashed $msg")
unstashing(processed :+ msg.s)
case Unstashed(message: Msg)
context.log.debug(s"unstashed $message")
unstashing(processed :+ message.s)
case Unstashed(GetProcessed(replyTo))
ctx.log.debug(s"unstashed GetProcessed")
context.log.debug(s"unstashed GetProcessed")
replyTo ! processed
Behaviors.same
case msg: Msg
ctx.log.debug(s"got $msg in unstashing")
buffer.stash(msg)
case message: Msg
context.log.debug(s"got $message in unstashing")
buffer.stash(message)
Behaviors.same
case g: GetProcessed
ctx.log.debug(s"got GetProcessed in unstashing")
context.log.debug(s"got GetProcessed in unstashing")
buffer.stash(g)
Behaviors.same
case Stash
stashing(processed)
case Unstash
if (buffer.isEmpty) {
ctx.log.debug(s"unstashing done")
context.log.debug(s"unstashing done")
active(processed)
} else {
ctx.self ! Unstash // continue unstashing until buffer is empty
context.self ! Unstash // continue unstashing until buffer is empty
val numberOfMessages = 2
ctx.log.debug(s"Unstash $numberOfMessages of ${buffer.size}, starting with ${buffer.head}")
buffer.unstash(ctx, unstashing(processed), numberOfMessages, Unstashed)
context.log.debug(s"Unstash $numberOfMessages of ${buffer.size}, starting with ${buffer.head}")
buffer.unstash(context, unstashing(processed), numberOfMessages, Unstashed)
}
case GetStashSize(replyTo)
replyTo ! buffer.size
@ -120,7 +120,7 @@ object StashSpec {
active(Vector.empty)
}
class MutableStash(ctx: ActorContext[Command]) extends AbstractBehavior[Command] {
class MutableStash(context: ActorContext[Command]) extends AbstractBehavior[Command] {
private val buffer = StashBuffer.apply[Command](capacity = 10)
private var stashing = false
@ -128,11 +128,11 @@ object StashSpec {
override def onMessage(cmd: Command): Behavior[Command] = {
cmd match {
case msg: Msg
case message: Msg
if (stashing)
buffer.stash(msg)
buffer.stash(message)
else
processed :+= msg.s
processed :+= message.s
this
case g @ GetProcessed(replyTo)
if (stashing)
@ -148,23 +148,23 @@ object StashSpec {
this
case UnstashAll
stashing = false
buffer.unstashAll(ctx, this)
buffer.unstashAll(context, this)
case Unstash
if (buffer.isEmpty) {
stashing = false
this
} else {
ctx.self ! Unstash // continue unstashing until buffer is empty
context.self ! Unstash // continue unstashing until buffer is empty
val numberOfMessages = 2
ctx.log.debug(s"Unstash $numberOfMessages of ${buffer.size}, starting with ${buffer.head}")
buffer.unstash(ctx, this, numberOfMessages, Unstashed)
context.log.debug(s"Unstash $numberOfMessages of ${buffer.size}, starting with ${buffer.head}")
buffer.unstash(context, this, numberOfMessages, Unstashed)
}
case Unstashed(msg: Msg)
ctx.log.debug(s"unstashed $msg")
processed :+= msg.s
case Unstashed(message: Msg)
context.log.debug(s"unstashed $message")
processed :+= message.s
this
case Unstashed(GetProcessed(replyTo))
ctx.log.debug(s"unstashed GetProcessed")
context.log.debug(s"unstashed GetProcessed")
replyTo ! processed
Behaviors.same
case _: Unstashed
@ -185,7 +185,7 @@ class ImmutableStashSpec extends StashSpec {
class MutableStashSpec extends StashSpec {
import StashSpec._
def testQualifier: String = "mutable behavior"
def behaviorUnderTest: Behavior[Command] = Behaviors.setup(ctx new MutableStash(ctx))
def behaviorUnderTest: Behavior[Command] = Behaviors.setup(context new MutableStash(context))
}
abstract class StashSpec extends ScalaTestWithActorTestKit with WordSpecLike {

View file

@ -24,7 +24,7 @@ class StopSpec extends ScalaTestWithActorTestKit with WordSpecLike {
val sawSignal = Promise[Done]()
spawn(Behaviors.setup[AnyRef] { _
Behaviors.stopped[AnyRef](Behaviors.receiveSignal[AnyRef] {
case (ctx, PostStop)
case (context, PostStop)
sawSignal.success(Done)
Behaviors.empty
})
@ -38,16 +38,16 @@ class StopSpec extends ScalaTestWithActorTestKit with WordSpecLike {
val ref = spawn(Behaviors.setup[AnyRef] { _
Behaviors.intercept(
new BehaviorInterceptor[AnyRef, AnyRef] {
override def aroundReceive(ctx: typed.ActorContext[AnyRef], msg: AnyRef, target: ReceiveTarget[AnyRef]): Behavior[AnyRef] = {
target(ctx, msg)
override def aroundReceive(context: typed.ActorContext[AnyRef], message: AnyRef, target: ReceiveTarget[AnyRef]): Behavior[AnyRef] = {
target(context, message)
}
override def aroundSignal(ctx: typed.ActorContext[AnyRef], signal: Signal, target: SignalTarget[AnyRef]): Behavior[AnyRef] = {
target(ctx, signal)
override def aroundSignal(context: typed.ActorContext[AnyRef], signal: Signal, target: SignalTarget[AnyRef]): Behavior[AnyRef] = {
target(context, signal)
}
}
)(Behaviors.stopped[AnyRef](Behaviors.receiveSignal[AnyRef] {
case (ctx, PostStop)
case (context, PostStop)
sawSignal.success(Done)
Behaviors.empty
}))
@ -60,7 +60,7 @@ class StopSpec extends ScalaTestWithActorTestKit with WordSpecLike {
"execute the post stop early" in {
val sawSignal = Promise[Done]()
spawn(Behaviors.stopped[AnyRef](Behaviors.receiveSignal[AnyRef] {
case (ctx, PostStop)
case (context, PostStop)
sawSignal.success(Done)
Behaviors.empty
}))

View file

@ -32,36 +32,36 @@ object AdapterSpec {
def typed1(ref: untyped.ActorRef, probe: ActorRef[String]): Behavior[String] =
Behaviors.receive[String] {
(ctx, msg)
msg match {
(context, message)
message match {
case "send"
val replyTo = ctx.self.toUntyped
val replyTo = context.self.toUntyped
ref.tell("ping", replyTo)
Behaviors.same
case "pong"
probe ! "ok"
Behaviors.same
case "actorOf"
val child = ctx.actorOf(untyped1)
child.tell("ping", ctx.self.toUntyped)
val child = context.actorOf(untyped1)
child.tell("ping", context.self.toUntyped)
Behaviors.same
case "watch"
ctx.watch(ref)
context.watch(ref)
Behaviors.same
case "supervise-stop"
val child = ctx.actorOf(untyped1)
ctx.watch(child)
val child = context.actorOf(untyped1)
context.watch(child)
child ! ThrowIt3
child.tell("ping", ctx.self.toUntyped)
child.tell("ping", context.self.toUntyped)
Behaviors.same
case "stop-child"
val child = ctx.actorOf(untyped1)
ctx.watch(child)
ctx.stop(child)
val child = context.actorOf(untyped1)
context.watch(child)
context.stop(child)
Behaviors.same
}
} receiveSignal {
case (ctx, Terminated(ref))
case (context, Terminated(ref))
probe ! "terminated"
Behaviors.same
}
@ -129,8 +129,8 @@ object AdapterSpec {
}
def typed2: Behavior[Typed2Msg] =
Behaviors.receive { (ctx, msg)
msg match {
Behaviors.receive { (context, message)
message match {
case Ping(replyTo)
replyTo ! "pong"
Behaviors.same
@ -172,9 +172,9 @@ class AdapterSpec extends AkkaSpec(
for { _ 0 to 10 } {
var system: akka.actor.typed.ActorSystem[Done] = null
try {
system = ActorSystem.create(Behaviors.receive[Done] { (ctx, msg)
ctx.self ! Done
msg match {
system = ActorSystem.create(Behaviors.receive[Done] { (context, message)
context.self ! Done
message match {
case Done Behaviors.stopped
}

View file

@ -33,22 +33,22 @@ object DispatchersDocSpec {
case class WhichDispatcher(replyTo: ActorRef[Dispatcher])
val giveMeYourDispatcher = Behaviors.receive[WhichDispatcher] { (ctx, msg)
msg.replyTo ! ctx.executionContext.asInstanceOf[Dispatcher]
val giveMeYourDispatcher = Behaviors.receive[WhichDispatcher] { (context, message)
message.replyTo ! context.executionContext.asInstanceOf[Dispatcher]
Behaviors.same
}
val yourBehavior: Behavior[String] = Behaviors.same
val example = Behaviors.receive[Any] { (ctx, msg)
val example = Behaviors.receive[Any] { (context, message)
//#spawn-dispatcher
import akka.actor.typed.DispatcherSelector
ctx.spawn(yourBehavior, "DefaultDispatcher")
ctx.spawn(yourBehavior, "ExplicitDefaultDispatcher", DispatcherSelector.default())
ctx.spawn(yourBehavior, "BlockingDispatcher", DispatcherSelector.blocking())
ctx.spawn(yourBehavior, "DispatcherFromConfig", DispatcherSelector.fromConfig("your-dispatcher"))
context.spawn(yourBehavior, "DefaultDispatcher")
context.spawn(yourBehavior, "ExplicitDefaultDispatcher", DispatcherSelector.default())
context.spawn(yourBehavior, "BlockingDispatcher", DispatcherSelector.blocking())
context.spawn(yourBehavior, "DispatcherFromConfig", DispatcherSelector.fromConfig("your-dispatcher"))
//#spawn-dispatcher
Behaviors.same

View file

@ -31,8 +31,8 @@ object FSMDocSpec {
//#simple-state
// states of the FSM represented as behaviors
def idle(data: Data): Behavior[Event] = Behaviors.receiveMessage[Event] { msg: Event
(msg, data) match {
def idle(data: Data): Behavior[Event] = Behaviors.receiveMessage[Event] { message: Event
(message, data) match {
case (SetTarget(ref), Uninitialized)
idle(Todo(ref, Vector.empty))
case (Queue(obj), t @ Todo(_, v))

View file

@ -25,36 +25,36 @@ class FaultToleranceDocSpec extends ScalaTestWithActorTestKit(
sealed trait Message
case class Fail(text: String) extends Message
val worker = Behaviors.receive[Message] { (ctx, msg)
msg match {
val worker = Behaviors.receive[Message] { (context, message)
message match {
case Fail(text) throw new RuntimeException(text)
}
}
val middleManagementBehavior = Behaviors.setup[Message] { ctx
ctx.log.info("Middle management starting up")
val child = ctx.spawn(worker, "child")
ctx.watch(child)
val middleManagementBehavior = Behaviors.setup[Message] { context
context.log.info("Middle management starting up")
val child = context.spawn(worker, "child")
context.watch(child)
// here we don't handle Terminated at all which means that
// when the child fails or stops gracefully this actor will
// fail with a DeathWatchException
Behaviors.receive[Message] { (ctx, msg)
child ! msg
Behaviors.receive[Message] { (context, message)
child ! message
Behaviors.same
}
}
val bossBehavior = Behaviors.supervise(Behaviors.setup[Message] { ctx
ctx.log.info("Boss starting up")
val middleManagment = ctx.spawn(middleManagementBehavior, "middle-management")
ctx.watch(middleManagment)
val bossBehavior = Behaviors.supervise(Behaviors.setup[Message] { context
context.log.info("Boss starting up")
val middleManagment = context.spawn(middleManagementBehavior, "middle-management")
context.watch(middleManagment)
// here we don't handle Terminated at all which means that
// when middle management fails with a DeathWatchException
// this actor will also fail
Behaviors.receive[Message] { (ctx, msg)
middleManagment ! msg
Behaviors.receive[Message] { (context, message)
middleManagment ! message
Behaviors.same
}
}).onFailure[DeathPactException](SupervisorStrategy.restart)

View file

@ -28,14 +28,14 @@ object GracefulStopDocSpec {
// Predefined cleanup operation
def cleanup(log: Logger): Unit = log.info("Cleaning up!")
val mcpa = Behaviors.receive[JobControlLanguage] { (ctx, msg)
msg match {
val mcpa = Behaviors.receive[JobControlLanguage] { (context, message)
message match {
case SpawnJob(jobName)
ctx.log.info("Spawning job {}!", jobName)
ctx.spawn(Job.job(jobName), name = jobName)
context.log.info("Spawning job {}!", jobName)
context.spawn(Job.job(jobName), name = jobName)
Behaviors.same
case GracefulShutdown
ctx.log.info("Initiating graceful shutdown...")
context.log.info("Initiating graceful shutdown...")
// perform graceful stop, executing cleanup before final system termination
// behavior executing cleanup is passed as a parameter to Actor.stopped
Behaviors.stopped {
@ -47,8 +47,8 @@ object GracefulStopDocSpec {
}
}
}.receiveSignal {
case (ctx, PostStop)
ctx.log.info("MCPA stopped")
case (context, PostStop)
context.log.info("MCPA stopped")
Behaviors.same
}
}
@ -60,8 +60,8 @@ object GracefulStopDocSpec {
import GracefulStopDocSpec.MasterControlProgramActor.JobControlLanguage
def job(name: String) = Behaviors.receiveSignal[JobControlLanguage] {
case (ctx, PostStop)
ctx.log.info("Worker {} stopped", name)
case (context, PostStop)
context.log.info("Worker {} stopped", name)
Behaviors.same
}
}

View file

@ -27,8 +27,8 @@ class InteractionPatternsSpec extends ScalaTestWithActorTestKit with WordSpecLik
case class PrintMe(message: String)
val printerBehavior: Behavior[PrintMe] = Behaviors.receive {
case (ctx, PrintMe(message))
ctx.log.info(message)
case (context, PrintMe(message))
context.log.info(message)
Behaviors.same
}
// #fire-and-forget-definition
@ -67,11 +67,11 @@ class InteractionPatternsSpec extends ScalaTestWithActorTestKit with WordSpecLik
val probe = TestProbe[Response]()
// shhh, don't tell anyone
import scala.language.reflectiveCalls
val ctx = new {
val context = new {
def self = probe.ref
}
// #request-response-send
otherActor ! Request("give me cookies", ctx.self)
otherActor ! Request("give me cookies", context.self)
// #request-response-send
probe.expectMessageType[Response]
@ -101,9 +101,9 @@ class InteractionPatternsSpec extends ScalaTestWithActorTestKit with WordSpecLik
private final case class WrappedBackendResponse(response: Backend.Response) extends Command
def translator(backend: ActorRef[Backend.Request]): Behavior[Command] =
Behaviors.setup[Command] { ctx
Behaviors.setup[Command] { context
val backendResponseMapper: ActorRef[Backend.Response] =
ctx.messageAdapter(rsp WrappedBackendResponse(rsp))
context.messageAdapter(rsp WrappedBackendResponse(rsp))
def active(
inProgress: Map[Int, ActorRef[URI]],
@ -116,13 +116,13 @@ class InteractionPatternsSpec extends ScalaTestWithActorTestKit with WordSpecLik
case wrapped: WrappedBackendResponse wrapped.response match {
case Backend.JobStarted(taskId)
ctx.log.info("Started {}", taskId)
context.log.info("Started {}", taskId)
Behaviors.same
case Backend.JobProgress(taskId, progress)
ctx.log.info("Progress {}: {}", taskId, progress)
context.log.info("Progress {}: {}", taskId, progress)
Behaviors.same
case Backend.JobCompleted(taskId, result)
ctx.log.info("Completed {}: {}", taskId, result)
context.log.info("Completed {}: {}", taskId, result)
inProgress(taskId) ! result
active(inProgress - taskId, count)
}
@ -158,7 +158,7 @@ class InteractionPatternsSpec extends ScalaTestWithActorTestKit with WordSpecLik
case object TimerKey
trait Msg
case class ExcitingMessage(msg: String) extends Msg
case class ExcitingMessage(message: String) extends Msg
final case class Batch(messages: Vector[Msg])
case object Timeout extends Msg
@ -168,9 +168,9 @@ class InteractionPatternsSpec extends ScalaTestWithActorTestKit with WordSpecLik
def idle(timers: TimerScheduler[Msg], target: ActorRef[Batch],
after: FiniteDuration, maxSize: Int): Behavior[Msg] = {
Behaviors.receiveMessage[Msg] { msg
Behaviors.receiveMessage[Msg] { message
timers.startSingleTimer(TimerKey, Timeout, after)
active(Vector(msg), timers, target, after, maxSize)
active(Vector(message), timers, target, after, maxSize)
}
}
@ -216,7 +216,7 @@ class InteractionPatternsSpec extends ScalaTestWithActorTestKit with WordSpecLik
// this is a part of the protocol that is internal to the actor itself
case class AdaptedResponse(message: String) extends DaveMessage
def daveBehavior(hal: ActorRef[HalCommand]) = Behaviors.setup[DaveMessage] { ctx
def daveBehavior(hal: ActorRef[HalCommand]) = Behaviors.setup[DaveMessage] { context
// asking someone requires a timeout, if the timeout hits without response
// the ask is failed with a TimeoutException
@ -226,7 +226,7 @@ class InteractionPatternsSpec extends ScalaTestWithActorTestKit with WordSpecLik
// as OpenThePodBayDoorsPlease is a case class it has a factory apply method
// that is what we are passing as the second parameter here it could also be written
// as `ref => OpenThePodBayDoorsPlease(ref)`
ctx.ask(hal)(OpenThePodBayDoorsPlease) {
context.ask(hal)(OpenThePodBayDoorsPlease) {
case Success(HalResponse(message)) AdaptedResponse(message)
case Failure(ex) AdaptedResponse("Request failed")
}
@ -236,7 +236,7 @@ class InteractionPatternsSpec extends ScalaTestWithActorTestKit with WordSpecLik
// changed at the time the response arrives and the transformation is done, best is to
// use immutable state we have closed over like here.
val requestId = 1
ctx.ask(hal)(OpenThePodBayDoorsPlease) {
context.ask(hal)(OpenThePodBayDoorsPlease) {
case Success(HalResponse(message)) AdaptedResponse(s"$requestId: $message")
case Failure(ex) AdaptedResponse(s"$requestId: Request failed")
}
@ -244,8 +244,8 @@ class InteractionPatternsSpec extends ScalaTestWithActorTestKit with WordSpecLik
Behaviors.receiveMessage {
// the adapted message ends up being processed like any other
// message sent to the actor
case AdaptedResponse(msg)
ctx.log.info("Got response from hal: {}", msg)
case AdaptedResponse(message)
context.log.info("Got response from hal: {}", message)
Behaviors.same
}
}
@ -287,13 +287,13 @@ class InteractionPatternsSpec extends ScalaTestWithActorTestKit with WordSpecLik
case class GetKeys(whoseKeys: String, respondTo: ActorRef[Keys])
case class GetWallet(whoseWallet: String, respondTo: ActorRef[Wallet])
def homeBehavior = Behaviors.receive[HomeCommand] { (ctx, msg)
val keyCabinet: ActorRef[GetKeys] = ctx.spawn(keyCabinetBehavior, "key-cabinet")
val drawer: ActorRef[GetWallet] = ctx.spawn(drawerBehavior, "drawer")
def homeBehavior = Behaviors.receive[HomeCommand] { (context, message)
val keyCabinet: ActorRef[GetKeys] = context.spawn(keyCabinetBehavior, "key-cabinet")
val drawer: ActorRef[GetWallet] = context.spawn(drawerBehavior, "drawer")
msg match {
message match {
case LeaveHome(who, respondTo)
ctx.spawn(prepareToLeaveHome(who, respondTo, keyCabinet, drawer), s"leaving-$who")
context.spawn(prepareToLeaveHome(who, respondTo, keyCabinet, drawer), s"leaving-$who")
Behavior.same
}
}
@ -307,13 +307,13 @@ class InteractionPatternsSpec extends ScalaTestWithActorTestKit with WordSpecLik
// we don't _really_ care about the actor protocol here as nobody will send us
// messages except for responses to our queries, so we just accept any kind of message
// but narrow that to more limited types then we interact
Behaviors.setup[AnyRef] { ctx
Behaviors.setup[AnyRef] { context
var wallet: Option[Wallet] = None
var keys: Option[Keys] = None
// we narrow the ActorRef type to any subtype of the actual type we accept
keyCabinet ! GetKeys(whoIsLeaving, ctx.self.narrow[Keys])
drawer ! GetWallet(whoIsLeaving, ctx.self.narrow[Wallet])
keyCabinet ! GetKeys(whoIsLeaving, context.self.narrow[Keys])
drawer ! GetWallet(whoIsLeaving, context.self.narrow[Wallet])
def nextBehavior: Behavior[AnyRef] =
(keys, wallet) match {
@ -354,8 +354,8 @@ class InteractionPatternsSpec extends ScalaTestWithActorTestKit with WordSpecLik
// #standalone-ask
// keep this out of the sample as it uses the testkit spawn
val cookieActorRef = spawn(Behaviors.receiveMessage[GiveMeCookies] { msg
msg.replyTo ! Cookies(5)
val cookieActorRef = spawn(Behaviors.receiveMessage[GiveMeCookies] { message
message.replyTo ! Cookies(5)
Behaviors.same
})

View file

@ -29,14 +29,14 @@ object IntroSpec {
final case class Greet(whom: String, replyTo: ActorRef[Greeted])
final case class Greeted(whom: String, from: ActorRef[Greet])
val greeter: Behavior[Greet] = Behaviors.receive { (ctx, msg)
val greeter: Behavior[Greet] = Behaviors.receive { (context, message)
//#fiddle_code
ctx.log.info("Hello {}!", msg.whom)
context.log.info("Hello {}!", message.whom)
//#fiddle_code
//#hello-world-actor
println(s"Hello ${msg.whom}!")
println(s"Hello ${message.whom}!")
//#hello-world-actor
msg.replyTo ! Greeted(msg.whom, ctx.self)
message.replyTo ! Greeted(message.whom, context.self)
Behaviors.same
}
}
@ -46,18 +46,18 @@ object IntroSpec {
object HelloWorldBot {
def bot(greetingCounter: Int, max: Int): Behavior[HelloWorld.Greeted] =
Behaviors.receive { (ctx, msg)
Behaviors.receive { (context, message)
val n = greetingCounter + 1
//#fiddle_code
ctx.log.info("Greeting {} for {}", n, msg.whom)
context.log.info("Greeting {} for {}", n, message.whom)
//#fiddle_code
//#hello-world-bot
println(s"Greeting ${n} for ${msg.whom}")
println(s"Greeting ${n} for ${message.whom}")
//#hello-world-bot
if (n == max) {
Behaviors.stopped
} else {
msg.from ! HelloWorld.Greet(msg.whom, ctx.self)
message.from ! HelloWorld.Greet(message.whom, context.self)
bot(n, max)
}
}
@ -73,9 +73,9 @@ object IntroSpec {
Behaviors.setup { context
val greeter = context.spawn(HelloWorld.greeter, "greeter")
Behaviors.receiveMessage { msg
val replyTo = context.spawn(HelloWorldBot.bot(greetingCounter = 0, max = 3), msg.name)
greeter ! HelloWorld.Greet(msg.name, replyTo)
Behaviors.receiveMessage { message
val replyTo = context.spawn(HelloWorldBot.bot(greetingCounter = 0, max = 3), message.name)
greeter ! HelloWorld.Greet(message.name, replyTo)
Behaviors.same
}
}
@ -95,10 +95,10 @@ object IntroSpec {
val props = DispatcherSelector.fromConfig(dispatcherPath)
val greeter = context.spawn(HelloWorld.greeter, "greeter", props)
Behaviors.receiveMessage { msg
val replyTo = context.spawn(HelloWorldBot.bot(greetingCounter = 0, max = 3), msg.name)
Behaviors.receiveMessage { message
val replyTo = context.spawn(HelloWorldBot.bot(greetingCounter = 0, max = 3), message.name)
greeter ! HelloWorld.Greet(msg.name, replyTo)
greeter ! HelloWorld.Greet(message.name, replyTo)
Behaviors.same
}
}
@ -133,12 +133,12 @@ object IntroSpec {
chatRoom(List.empty)
private def chatRoom(sessions: List[ActorRef[SessionCommand]]): Behavior[RoomCommand] =
Behaviors.receive { (ctx, msg)
msg match {
Behaviors.receive { (context, message)
message match {
case GetSession(screenName, client)
// create a child actor for further interaction with the client
val ses = ctx.spawn(
session(ctx.self, screenName, client),
val ses = context.spawn(
session(context.self, screenName, client),
name = URLEncoder.encode(screenName, StandardCharsets.UTF_8.name))
client ! SessionGranted(ses)
chatRoom(ses :: sessions)
@ -153,8 +153,8 @@ object IntroSpec {
room: ActorRef[PublishSessionMessage],
screenName: String,
client: ActorRef[SessionEvent]): Behavior[SessionCommand] =
Behaviors.receive { (ctx, msg)
msg match {
Behaviors.receive { (context, message)
message match {
case PostMessage(message)
// from client, publish to others via the room
room ! PublishSessionMessage(screenName, message)
@ -216,10 +216,10 @@ class IntroSpec extends ScalaTestWithActorTestKit with WordSpecLike {
//#chatroom-main
val main: Behavior[NotUsed] =
Behaviors.setup { ctx
val chatRoom = ctx.spawn(ChatRoom.behavior, "chatroom")
val gabblerRef = ctx.spawn(gabbler, "gabbler")
ctx.watch(gabblerRef)
Behaviors.setup { context
val chatRoom = context.spawn(ChatRoom.behavior, "chatroom")
val gabblerRef = context.spawn(gabbler, "gabbler")
context.watch(gabblerRef)
chatRoom ! GetSession("ol Gabbler", gabblerRef)
Behaviors.receiveSignal {

View file

@ -44,17 +44,17 @@ object OOIntroSpec {
//#chatroom-behavior
def behavior(): Behavior[RoomCommand] =
Behaviors.setup[RoomCommand](ctx new ChatRoomBehavior(ctx))
Behaviors.setup[RoomCommand](context new ChatRoomBehavior(context))
class ChatRoomBehavior(ctx: ActorContext[RoomCommand]) extends AbstractBehavior[RoomCommand] {
class ChatRoomBehavior(context: ActorContext[RoomCommand]) extends AbstractBehavior[RoomCommand] {
private var sessions: List[ActorRef[SessionCommand]] = List.empty
override def onMessage(msg: RoomCommand): Behavior[RoomCommand] = {
msg match {
override def onMessage(message: RoomCommand): Behavior[RoomCommand] = {
message match {
case GetSession(screenName, client)
// create a child actor for further interaction with the client
val ses = ctx.spawn(
session(ctx.self, screenName, client),
val ses = context.spawn(
session(context.self, screenName, client),
name = URLEncoder.encode(screenName, StandardCharsets.UTF_8.name))
client ! SessionGranted(ses)
sessions = ses :: sessions
@ -112,10 +112,10 @@ class OOIntroSpec extends ScalaTestWithActorTestKit with WordSpecLike {
//#chatroom-main
val main: Behavior[String] =
Behaviors.setup { ctx
val chatRoom = ctx.spawn(ChatRoom.behavior(), "chatroom")
val gabblerRef = ctx.spawn(gabbler, "gabbler")
ctx.watch(gabblerRef)
Behaviors.setup { context
val chatRoom = context.spawn(ChatRoom.behavior(), "chatroom")
val gabblerRef = context.spawn(gabbler, "gabbler")
context.watch(gabblerRef)
Behaviors.receiveMessagePartial[String] {
case "go"

View file

@ -33,9 +33,9 @@ object SpawnProtocolDocSpec {
//#main
object HelloWorldMain {
val main: Behavior[SpawnProtocol] =
Behaviors.setup { ctx
Behaviors.setup { context
// Start initial tasks
// ctx.spawn(...)
// context.spawn(...)
SpawnProtocol.behavior
}
@ -63,8 +63,8 @@ class SpawnProtocolDocSpec extends ScalaTestWithActorTestKit with WordSpecLike {
val greeter: Future[ActorRef[HelloWorld.Greet]] =
system ? SpawnProtocol.Spawn(behavior = HelloWorld.greeter, name = "greeter", props = Props.empty)
val greetedBehavior = Behaviors.receive[HelloWorld.Greeted] { (ctx, msg)
ctx.log.info("Greeting for {} from {}", msg.whom, msg.from)
val greetedBehavior = Behaviors.receive[HelloWorld.Greeted] { (context, message)
context.log.info("Greeting for {} from {}", message.whom, message.from)
Behaviors.stopped
}

View file

@ -33,16 +33,16 @@ object StashDocSpec {
private final case class DBError(cause: Throwable) extends Command
def behavior(id: String, db: DB): Behavior[Command] =
Behaviors.setup[Command] { ctx
Behaviors.setup[Command] { context
val buffer = StashBuffer[Command](capacity = 100)
def init(): Behavior[Command] =
Behaviors.receive[Command] { (ctx, msg)
msg match {
Behaviors.receive[Command] { (context, message)
message match {
case InitialState(value)
// now we are ready to handle stashed messages if any
buffer.unstashAll(ctx, active(value))
buffer.unstashAll(context, active(value))
case DBError(cause)
throw cause
case other
@ -53,27 +53,27 @@ object StashDocSpec {
}
def active(state: String): Behavior[Command] =
Behaviors.receive { (ctx, msg)
msg match {
Behaviors.receive { (context, message)
message match {
case Get(replyTo)
replyTo ! state
Behaviors.same
case Save(value, replyTo)
import ctx.executionContext
import context.executionContext
db.save(id, value).onComplete {
case Success(_) ctx.self ! SaveSuccess
case Failure(cause) ctx.self ! DBError(cause)
case Success(_) context.self ! SaveSuccess
case Failure(cause) context.self ! DBError(cause)
}
saving(value, replyTo)
}
}
def saving(state: String, replyTo: ActorRef[Done]): Behavior[Command] =
Behaviors.receive[Command] { (ctx, msg)
msg match {
Behaviors.receive[Command] { (context, message)
message match {
case SaveSuccess
replyTo ! Done
buffer.unstashAll(ctx, active(state))
buffer.unstashAll(context, active(state))
case DBError(cause)
throw cause
case other
@ -82,12 +82,12 @@ object StashDocSpec {
}
}
import ctx.executionContext
import context.executionContext
db.load(id).onComplete {
case Success(value)
ctx.self ! InitialState(value)
context.self ! InitialState(value)
case Failure(cause)
ctx.self ! DBError(cause)
context.self ! DBError(cause)
}
init()

View file

@ -28,7 +28,7 @@ object TypedWatchingUntypedSpec {
val behavior: Behavior[Command] =
Behaviors.setup { context
// context.spawn is an implicit extension method
// context.actorOf is an implicit extension method
val untyped = context.actorOf(Untyped.props(), "second")
// context.watch is an implicit extension method
@ -38,10 +38,10 @@ object TypedWatchingUntypedSpec {
untyped.tell(Typed.Ping(context.self), context.self.toUntyped)
Behaviors.receivePartial[Command] {
case (ctx, Pong)
case (context, Pong)
// it's not possible to get the sender, that must be sent in message
// context.stop is an implicit extension method
ctx.stop(untyped)
context.stop(untyped)
Behaviors.same
} receiveSignal {
case (_, akka.actor.typed.Terminated(_))

View file

@ -58,10 +58,10 @@ object UntypedWatchingTypedSpec {
case object Pong
val behavior: Behavior[Command] =
Behaviors.receive { (ctx, msg)
msg match {
Behaviors.receive { (context, message)
message match {
case Ping(replyTo)
ctx.log.info(s"${ctx.self} got Ping from $replyTo")
context.log.info(s"${context.self} got Ping from $replyTo")
// replyTo is an untyped actor that has been converted for coexistence
replyTo ! Pong
Behaviors.same

View file

@ -57,7 +57,7 @@ case we don't need to update any state, so we return `same`, which means
the next behavior is "the same as the current one".
The type of the messages handled by this behavior is declared to be of class
`Greet`, meaning that `msg` argument is
`Greet`, meaning that `message` argument is
also typed as such. This is why we can access the `whom` and `replyTo`
members without needing to use a pattern match.
@ -208,7 +208,7 @@ If we did not care about securing the correspondence between a session and a
screen name then we could change the protocol such that `PostMessage` is
removed and all clients just get an @scala[`ActorRef[PublishSessionMessage]`]@java[`ActorRef<PublishSessionMessage>`] to
send to. In this case no session actor would be needed and we could use
@scala[`ctx.self`]@java[`ctx.getSelf()`]. The type-checks work out in that case because
@scala[`context.self`]@java[`context.getSelf()`]. The type-checks work out in that case because
@scala[`ActorRef[-T]`]@java[`ActorRef<T>`] is contravariant in its type parameter, meaning that we
can use a @scala[`ActorRef[RoomCommand]`]@java[`ActorRef<RoomCommand>`] wherever an
@scala[`ActorRef[PublishSessionMessage]`]@java[`ActorRef<PublishSessionMessage>`] is needed—this makes sense because the
@ -270,7 +270,7 @@ that creates the behavior instance immediately before the actor is running. The
`setup` is passed the `ActorContext` as parameter and that can for example be used for spawning child actors.
This `main` Actor creates the chat room and the gabbler and the session between them is initiated, and when the
gabbler is finished we will receive the `Terminated` event due to having
called `ctx.watch` for it. This allows us to shut down the Actor system: when
called `context.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
@ -388,7 +388,7 @@ If we did not care about securing the correspondence between a session and a
screen name then we could change the protocol such that `PostMessage` is
removed and all clients just get an @scala[`ActorRef[PublishSessionMessage]`]@java[`ActorRef<PublishSessionMessage>`] to
send to. In this case no session actor would be needed and we could use
@scala[`ctx.self`]@java[`ctx.getSelf()`]. The type-checks work out in that case because
@scala[`context.self`]@java[`context.getSelf()`]. The type-checks work out in that case because
@scala[`ActorRef[-T]`]@java[`ActorRef<T>`] is contravariant in its type parameter, meaning that we
can use a @scala[`ActorRef[RoomCommand]`]@java[`ActorRef<RoomCommand>`] wherever an
@scala[`ActorRef[PublishSessionMessage]`]@java[`ActorRef<PublishSessionMessage>`] is needed—this makes sense because the
@ -438,7 +438,7 @@ that creates the behavior instance immediately before the actor is running. The
`setup` is passed the `ActorContext` as parameter and that can for example be used for spawning child actors.
This `main` Actor creates the chat room and the gabbler and the session between them is initiated, and when the
gabbler is finished we will receive the `Terminated` event due to having
called `ctx.watch` for it. This allows us to shut down the Actor system: when
called `context.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

View file

@ -53,6 +53,6 @@ stashed messages low. Actors that hog the message processing thread for too long
of other actors.
That can be mitigated by using the `StashBuffer.unstash` with `numberOfMessages` parameter and then send a
message to @scala[`ctx.self`]@java[`ctx.getSelf`] before continuing unstashing more. That means that other
message to @scala[`context.self`]@java[`context.getSelf`] before continuing unstashing more. That means that other
new messages may arrive in-between and those must be stashed to keep the original order of messages. It
becomes more complicated, so better keep the number of stashed messages low.