Rename widen to transformMessages, #27253

This commit is contained in:
Patrik Nordwall 2019-07-10 14:17:59 +02:00 committed by Johan Andrén
parent b1a7ff5749
commit 041db6f549
12 changed files with 114 additions and 69 deletions

View file

@ -7,6 +7,8 @@ package akka.actor.typed.javadsl;
import akka.actor.typed.*; import akka.actor.typed.*;
import akka.actor.typed.TypedActorContext; import akka.actor.typed.TypedActorContext;
import java.math.BigDecimal;
import java.math.BigInteger;
import java.time.Duration; import java.time.Duration;
import java.util.concurrent.CompletionStage; import java.util.concurrent.CompletionStage;
@ -62,7 +64,28 @@ public class ActorCompile {
final ActorRef<MyMsg> self = context.getSelf(); final ActorRef<MyMsg> self = context.getSelf();
return monitor(MyMsg.class, self, ignore()); return monitor(MyMsg.class, self, ignore());
}); });
Behavior<MyMsg> actor9 = widened(MyMsgA.class, actor7, pf -> pf.match(MyMsgA.class, x -> x));
Behavior<MyMsgA> actor9a =
transformMessages(MyMsgA.class, actor7, pf -> pf.match(MyMsgA.class, x -> x));
Behavior<MyMsg> actor9b =
transformMessages(MyMsg.class, actor7, pf -> pf.match(MyMsgA.class, x -> x));
// this is the example from the Javadoc
Behavior<String> s =
Behaviors.receive(
(ctx, msg) -> {
return Behaviors.same();
});
Behavior<Number> n =
Behaviors.transformMessages(
Number.class,
s,
pf ->
pf.match(BigInteger.class, i -> "BigInteger(" + i + ")")
.match(BigDecimal.class, d -> "BigDecimal(" + d + ")")
// drop all other kinds of Number
);
Behavior<MyMsg> actor10 = Behavior<MyMsg> actor10 =
Behaviors.receive((context, message) -> stopped(() -> {}), (context, signal) -> same()); Behaviors.receive((context, message) -> stopped(() -> {}), (context, signal) -> same());

View file

@ -683,9 +683,9 @@ class NormalActorContextSpec extends ActorContextSpec {
override def decoration[T: ClassTag] = x => x override def decoration[T: ClassTag] = x => x
} }
class WidenActorContextSpec extends ActorContextSpec { class TransformMessagesActorContextSpec extends ActorContextSpec {
override def decoration[T: ClassTag] = b => b.widen { case x => x } override def decoration[T: ClassTag] = b => b.transformMessages { case x => x }
} }
class DeferredActorContextSpec extends ActorContextSpec { class DeferredActorContextSpec extends ActorContextSpec {

View file

@ -469,11 +469,11 @@ class MutableScalaBehaviorSpec extends Messages with Become with Stoppable {
} }
} }
class WidenedScalaBehaviorSpec extends ImmutableWithSignalScalaBehaviorSpec with Reuse with Siphon { class TransformMessagesScalaBehaviorSpec extends ImmutableWithSignalScalaBehaviorSpec with Reuse with Siphon {
override def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux) = { override def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux) = {
val inbox = TestInbox[Command]("widenedListener") val inbox = TestInbox[Command]("transformMessagesListener")
super.behavior(monitor)._1.widen[Command] { case c => inbox.ref ! c; c } -> inbox super.behavior(monitor)._1.transformMessages[Command] { case c => inbox.ref ! c; c } -> inbox
} }
} }
@ -586,10 +586,10 @@ class ImmutableJavaBehaviorSpec extends Messages with Become with Stoppable {
} }
} }
class WidenedJavaBehaviorSpec extends ImmutableWithSignalJavaBehaviorSpec with Reuse with Siphon { class TransformMessagesJavaBehaviorSpec extends ImmutableWithSignalJavaBehaviorSpec with Reuse with Siphon {
override def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux) = { override def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux) = {
val inbox = TestInbox[Command]("widenedListener") val inbox = TestInbox[Command]("transformMessagesListener")
JBehaviors.widened(classOf[Command], super.behavior(monitor)._1, pf(_.`match`(classOf[Command], fi(x => { JBehaviors.transformMessages(classOf[Command], super.behavior(monitor)._1, pf(_.`match`(classOf[Command], fi(x => {
inbox.ref ! x inbox.ref ! x
x x
})))) -> inbox })))) -> inbox

View file

@ -102,14 +102,14 @@ class DeferredSpec extends ScalaTestWithActorTestKit("""
probe.expectMessage(Started) probe.expectMessage(Started)
} }
"must un-defer underlying when wrapped by widen" in { "must un-defer underlying when wrapped by transformMessages" in {
val probe = TestProbe[Event]("evt") val probe = TestProbe[Event]("evt")
val behv = Behaviors val behv = Behaviors
.setup[Command] { _ => .setup[Command] { _ =>
probe.ref ! Started probe.ref ! Started
target(probe.ref) target(probe.ref)
} }
.widen[Command] { .transformMessages[Command] {
case m => m case m => m
} }
probe.expectNoMessage() // not yet probe.expectNoMessage() // not yet

View file

@ -470,16 +470,16 @@ class InterceptSpec extends ScalaTestWithActorTestKit("""
probe.expectMessage("b") // bypass toUpper interceptor probe.expectMessage("b") // bypass toUpper interceptor
} }
"be possible to combine with widen" in { "be possible to combine with transformMessages" in {
val probe = createTestProbe[String]() val probe = createTestProbe[String]()
val ref = spawn(MultiProtocol(probe.ref).widen[String] { val ref = spawn(MultiProtocol(probe.ref).transformMessages[String] {
case s => Command(s.toUpperCase()) case s => Command(s.toUpperCase())
}) })
ref ! "a" ref ! "a"
probe.expectMessage("A") probe.expectMessage("A")
ref.unsafeUpcast ! ExternalResponse("b") ref.unsafeUpcast ! ExternalResponse("b")
probe.expectMessage("b") // bypass widen interceptor probe.expectMessage("b") // bypass transformMessages interceptor
} }
"be possible to combine with MDC" in { "be possible to combine with MDC" in {

View file

@ -4,6 +4,7 @@
package akka.actor.typed package akka.actor.typed
import java.math.BigInteger
import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicInteger
import akka.actor.ActorInitializationException import akka.actor.ActorInitializationException
@ -13,10 +14,25 @@ import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.adapter._ import akka.actor.typed.scaladsl.adapter._
import akka.testkit.EventFilter import akka.testkit.EventFilter
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
import scala.concurrent.duration._ import scala.concurrent.duration._
class WidenSpec extends ScalaTestWithActorTestKit(""" object TransformMessagesSpec {
// this is the sample from the Scaladoc
val b: Behavior[Number] =
Behaviors
.receive[String] { (ctx, msg) =>
println(msg)
Behaviors.same
}
.transformMessages[Number] {
case b: BigDecimal => s"BigDecimal(&dollar;b)"
case i: BigInt => s"BigInteger(&dollar;i)"
// all other kinds of Number will be `unhandled`
}
}
class TransformMessagesSpec extends ScalaTestWithActorTestKit("""
akka.loggers = [akka.testkit.TestEventListener] akka.loggers = [akka.testkit.TestEventListener]
""") with WordSpecLike { """) with WordSpecLike {
@ -28,12 +44,12 @@ class WidenSpec extends ScalaTestWithActorTestKit("""
probe ! message probe ! message
Behaviors.same Behaviors.same
} }
.widen[Int] { .transformMessages[Int] {
case n if n != 13 => n.toString case n if n != 13 => n.toString
} }
} }
"Widen" should { "transformMessages" should {
"transform from an outer type to an inner type" in { "transform from an outer type to an inner type" in {
val probe = TestProbe[String]() val probe = TestProbe[String]()
@ -57,21 +73,21 @@ class WidenSpec extends ScalaTestWithActorTestKit("""
} }
} }
"not build up when the same widen is used many times (initially)" in { "not build up when the same transformMessages is used many times (initially)" in {
val probe = TestProbe[String]() val probe = TestProbe[String]()
val transformCount = new AtomicInteger(0) val transformCount = new AtomicInteger(0)
// sadly the only "same" we can know is if it is the same PF // sadly the only "same" we can know is if it is the same PF
val transform: PartialFunction[String, String] = { val transformPF: PartialFunction[String, String] = {
case s => case s =>
transformCount.incrementAndGet() transformCount.incrementAndGet()
s s
} }
def widen(behavior: Behavior[String]): Behavior[String] = def transform(behavior: Behavior[String]): Behavior[String] =
behavior.widen(transform) behavior.transformMessages(transformPF)
val beh = val beh =
widen(widen(Behaviors.receiveMessage[String] { message => transform(transform(Behaviors.receiveMessage[String] { message =>
probe.ref ! message probe.ref ! message
Behaviors.same Behaviors.same
})) }))
@ -83,21 +99,21 @@ class WidenSpec extends ScalaTestWithActorTestKit("""
transformCount.get should ===(1) transformCount.get should ===(1)
} }
"not build up when the same widen is used many times (recursively)" in { "not build up when the same transformMessages is used many times (recursively)" in {
val probe = TestProbe[String]() val probe = TestProbe[String]()
val transformCount = new AtomicInteger(0) val transformCount = new AtomicInteger(0)
// sadly the only "same" we can know is if it is the same PF // sadly the only "same" we can know is if it is the same PF
val transform: PartialFunction[String, String] = { val transformPF: PartialFunction[String, String] = {
case s => case s =>
transformCount.incrementAndGet() transformCount.incrementAndGet()
s s
} }
def widen(behavior: Behavior[String]): Behavior[String] = def transform(behavior: Behavior[String]): Behavior[String] =
behavior.widen(transform) behavior.transformMessages(transformPF)
def next: Behavior[String] = def next: Behavior[String] =
widen(Behaviors.receiveMessage[String] { message => transform(Behaviors.receiveMessage[String] { message =>
probe.ref ! message probe.ref ! message
next next
}) })
@ -114,16 +130,16 @@ class WidenSpec extends ScalaTestWithActorTestKit("""
} }
"not allow mixing different widens in the same behavior stack" in { "not allow mixing different transformMessages in the same behavior stack" in {
val probe = TestProbe[String]() val probe = TestProbe[String]()
def widen(behavior: Behavior[String]): Behavior[String] = def transform(behavior: Behavior[String]): Behavior[String] =
behavior.widen[String] { behavior.transformMessages[String] {
case s => s.toLowerCase case s => s.toLowerCase
} }
EventFilter[ActorInitializationException](occurrences = 1).intercept { EventFilter[ActorInitializationException](occurrences = 1).intercept {
val ref = spawn(widen(widen(Behaviors.receiveMessage[String] { _ => val ref = spawn(transform(transform(Behaviors.receiveMessage[String] { _ =>
Behaviors.same Behaviors.same
}))) })))
@ -142,7 +158,7 @@ class WidenSpec extends ScalaTestWithActorTestKit("""
Behaviors.same Behaviors.same
} }
} }
.widen[String] { .transformMessages[String] {
case msg => msg.toUpperCase() case msg => msg.toUpperCase()
} }
@ -163,7 +179,7 @@ class WidenSpec extends ScalaTestWithActorTestKit("""
probe.ref ! msg probe.ref ! msg
Behaviors.same Behaviors.same
} }
.widen[String] { .transformMessages[String] {
case msg => msg.toUpperCase() case msg => msg.toUpperCase()
} }
} }

View file

@ -110,16 +110,22 @@ object Behavior {
final implicit class BehaviorDecorators[Inner](val behavior: Behavior[Inner]) extends AnyVal { final implicit class BehaviorDecorators[Inner](val behavior: Behavior[Inner]) extends AnyVal {
/** /**
* Widen the wrapped Behavior by placing a funnel in front of it: the supplied * Transform the incoming messages by placing a funnel in front of the wrapped `Behavior`: the supplied
* PartialFunction decides which message to pull in (those that it is defined * PartialFunction decides which message to pull in (those that it is defined
* at) and may transform the incoming message to place them into the wrapped * at) and may transform the incoming message to place them into the wrapped
* Behaviors type hierarchy. Signals are not transformed. * Behaviors type hierarchy. Signals are not transformed.
* *
* Example: * Example:
* {{{ * {{{
* receive[String] { (ctx, msg) => println(msg); same }.widen[Number] { * val b: Behavior[Number] =
* Behaviors
* .receive[String] { (ctx, msg) =>
* println(msg)
* Behaviors.same
* }
* .transformMessages[Number] {
* case b: BigDecimal => s"BigDecimal(&dollar;b)" * case b: BigDecimal => s"BigDecimal(&dollar;b)"
* case i: BigInteger => s"BigInteger(&dollar;i)" * case i: BigInt => s"BigInteger(&dollar;i)"
* // all other kinds of Number will be `unhandled` * // all other kinds of Number will be `unhandled`
* } * }
* }}} * }}}
@ -129,8 +135,8 @@ object Behavior {
* the interceptor and be continue to the inner behavior untouched. * the interceptor and be continue to the inner behavior untouched.
* *
*/ */
def widen[Outer: ClassTag](matcher: PartialFunction[Outer, Inner]): Behavior[Outer] = def transformMessages[Outer: ClassTag](matcher: PartialFunction[Outer, Inner]): Behavior[Outer] =
BehaviorImpl.widened(behavior, matcher) BehaviorImpl.transformMessages(behavior, matcher)
} }

View file

@ -42,8 +42,8 @@ private[akka] object BehaviorTags {
def as[U]: AC[U] = ctx.asInstanceOf[AC[U]] def as[U]: AC[U] = ctx.asInstanceOf[AC[U]]
} }
def widened[O: ClassTag, I](behavior: Behavior[I], matcher: PartialFunction[O, I]): Behavior[O] = def transformMessages[O: ClassTag, I](behavior: Behavior[I], matcher: PartialFunction[O, I]): Behavior[O] =
intercept(() => WidenedInterceptor(matcher))(behavior) intercept(() => TransformMessagesInterceptor(matcher))(behavior)
def same[T]: Behavior[T] = SameBehavior.unsafeCast[T] def same[T]: Behavior[T] = SameBehavior.unsafeCast[T]

View file

@ -177,7 +177,7 @@ private[akka] final class LogMessagesInterceptor(val opts: LogOptions) extends B
* INTERNAL API * INTERNAL API
*/ */
@InternalApi @InternalApi
private[akka] object WidenedInterceptor { private[akka] object TransformMessagesInterceptor {
private final val _any2null = (_: Any) => null private final val _any2null = (_: Any) => null
private final def any2null[T] = _any2null.asInstanceOf[Any => T] private final def any2null[T] = _any2null.asInstanceOf[Any => T]
@ -187,19 +187,19 @@ private[akka] object WidenedInterceptor {
* INTERNAL API * INTERNAL API
*/ */
@InternalApi @InternalApi
private[akka] final case class WidenedInterceptor[O: ClassTag, I](matcher: PartialFunction[O, I]) private[akka] final case class TransformMessagesInterceptor[O: ClassTag, I](matcher: PartialFunction[O, I])
extends BehaviorInterceptor[O, I] { extends BehaviorInterceptor[O, I] {
import BehaviorInterceptor._ import BehaviorInterceptor._
import WidenedInterceptor._ import TransformMessagesInterceptor._
override def isSame(other: BehaviorInterceptor[Any, Any]): Boolean = other match { override def isSame(other: BehaviorInterceptor[Any, Any]): Boolean = other match {
// If they use the same pf instance we can allow it, to have one way to workaround defining // If they use the same pf instance we can allow it, to have one way to workaround defining
// "recursive" narrowed behaviors. // "recursive" narrowed behaviors.
case WidenedInterceptor(`matcher`) => true case TransformMessagesInterceptor(`matcher`) => true
case WidenedInterceptor(otherMatcher) => case TransformMessagesInterceptor(otherMatcher) =>
// there is no safe way to allow this // there is no safe way to allow this
throw new IllegalStateException( throw new IllegalStateException(
"Widen can only be used one time in the same behavior stack. " + "transformMessages can only be used one time in the same behavior stack. " +
s"One defined in ${LineNumbers(matcher)}, and another in ${LineNumbers(otherMatcher)}") s"One defined in ${LineNumbers(matcher)}, and another in ${LineNumbers(otherMatcher)}")
case _ => false case _ => false
} }
@ -211,5 +211,5 @@ private[akka] final case class WidenedInterceptor[O: ClassTag, I](matcher: Parti
} }
} }
override def toString: String = s"Widen(${LineNumbers(matcher)})" override def toString: String = s"TransformMessages(${LineNumbers(matcher)})"
} }

View file

@ -270,7 +270,7 @@ object Behaviors {
} }
/** /**
* Widen the wrapped Behavior by placing a funnel in front of it: the supplied * Transform the incoming messages by placing a funnel in front of the wrapped `Behavior`: the supplied
* PartialFunction decides which message to pull in (those that it is defined * PartialFunction decides which message to pull in (those that it is defined
* at) and may transform the incoming message to place them into the wrapped * at) and may transform the incoming message to place them into the wrapped
* Behaviors type hierarchy. Signals are not transformed. * Behaviors type hierarchy. Signals are not transformed.
@ -278,17 +278,16 @@ object Behaviors {
* Example: * Example:
* {{{ * {{{
* Behavior<String> s = Behaviors.receive((ctx, msg) -> { * Behavior<String> s = Behaviors.receive((ctx, msg) -> {
* System.out.println(msg);
* return Behaviors.same(); * return Behaviors.same();
* }); * });
* Behavior<Number> n = Behaviors.widened(s, pf -> pf. * Behavior<Number> n = Behaviors.transformMessages(Number.class, s, pf ->
* match(BigInteger.class, i -> "BigInteger(" + i + ")"). * pf
* match(BigDecimal.class, d -> "BigDecimal(" + d + ")") * .match(BigInteger.class, i -> "BigInteger(" + i + ")")
* .match(BigDecimal.class, d -> "BigDecimal(" + d + ")")
* // drop all other kinds of Number * // drop all other kinds of Number
* ); * );
* }}} * }}}
* *
*
* @param interceptMessageClass Ensures that only messages of this class or a subclass thereof will be * @param interceptMessageClass Ensures that only messages of this class or a subclass thereof will be
* intercepted. Other message types (e.g. a private protocol) will bypass * intercepted. Other message types (e.g. a private protocol) will bypass
* the interceptor and be continue to the inner behavior untouched. * the interceptor and be continue to the inner behavior untouched.
@ -297,13 +296,13 @@ object Behaviors {
* @param selector * @param selector
* a partial function builder for describing the selection and * a partial function builder for describing the selection and
* transformation * transformation
* @return a behavior of the widened type * @return a behavior of the `Outer` type
*/ */
def widened[Outer, Inner]( def transformMessages[Outer, Inner](
interceptMessageClass: Class[Outer], interceptMessageClass: Class[Outer],
behavior: Behavior[Outer], behavior: Behavior[Inner],
selector: JFunction[PFBuilder[Inner, Outer], PFBuilder[Inner, Outer]]): Behavior[Inner] = selector: JFunction[PFBuilder[Outer, Inner], PFBuilder[Outer, Inner]]): Behavior[Outer] =
BehaviorImpl.widened(behavior, selector.apply(new PFBuilder).build())(ClassTag(interceptMessageClass)) BehaviorImpl.transformMessages(behavior, selector.apply(new PFBuilder).build())(ClassTag(interceptMessageClass))
/** /**
* Support for scheduled `self` messages in an actor. * Support for scheduled `self` messages in an actor.

View file

@ -393,7 +393,8 @@ made before finalizing the APIs. Compared to Akka 2.5.x the source incompatible
to retrieve `ActorContext`, and use an enclosing class to hold initialization parameters and `ActorContext`. to retrieve `ActorContext`, and use an enclosing class to hold initialization parameters and `ActorContext`.
* Java @javadoc[EntityRef](akka.cluster.sharding.typed.javadsl.EntityRef) ask timeout now takes a `java.time.Duration` rather than a @apidoc[Timeout] * Java @javadoc[EntityRef](akka.cluster.sharding.typed.javadsl.EntityRef) ask timeout now takes a `java.time.Duration` rather than a @apidoc[Timeout]
* Changed method signature for `EventAdapter.fromJournal` and support for `manifest` in `EventAdapter`. * Changed method signature for `EventAdapter.fromJournal` and support for `manifest` in `EventAdapter`.
* `BehaviorInterceptor`, `Behaviors.monitor`, `Behaviors.withMdc` and @scala[`widen`]@java[`Behaviors.widen`] takes * Renamed @scala[`widen`]@java[`Behaviors.widen`] to @scala[`transformMessages`]@java[`Behaviors.transformMessages`]
* `BehaviorInterceptor`, `Behaviors.monitor`, `Behaviors.withMdc` and @scala[`transformMessages`]@java[`Behaviors.transformMessages`] takes
a @scala[`ClassTag` parameter (probably source compatible)]@java[`interceptMessageClass` parameter]. a @scala[`ClassTag` parameter (probably source compatible)]@java[`interceptMessageClass` parameter].
`interceptMessageType` method in `BehaviorInterceptor` is replaced with this @scala[`ClassTag`]@java[`Class`] parameter. `interceptMessageType` method in `BehaviorInterceptor` is replaced with this @scala[`ClassTag`]@java[`Class`] parameter.
* `Behavior.orElse` has been removed because it wasn't safe together with `narrow`. * `Behavior.orElse` has been removed because it wasn't safe together with `narrow`.

View file

@ -83,10 +83,10 @@ class EventSourcedBehaviorInterceptorSpec
probe.expectMessage("ABC") probe.expectMessage("ABC")
} }
"be possible to combine with widen" in { "be possible to combine with transformMessages" in {
val probe = createTestProbe[String]() val probe = createTestProbe[String]()
val pid = nextPid() val pid = nextPid()
val ref = spawn(testBehavior(pid, probe.ref).widen[String] { val ref = spawn(testBehavior(pid, probe.ref).transformMessages[String] {
case s => s.toUpperCase() case s => s.toUpperCase()
}) })