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.TypedActorContext;
import java.math.BigDecimal;
import java.math.BigInteger;
import java.time.Duration;
import java.util.concurrent.CompletionStage;
@ -62,7 +64,28 @@ public class ActorCompile {
final ActorRef<MyMsg> self = context.getSelf();
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 =
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
}
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 {

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) = {
val inbox = TestInbox[Command]("widenedListener")
super.behavior(monitor)._1.widen[Command] { case c => inbox.ref ! c; c } -> inbox
val inbox = TestInbox[Command]("transformMessagesListener")
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) = {
val inbox = TestInbox[Command]("widenedListener")
JBehaviors.widened(classOf[Command], super.behavior(monitor)._1, pf(_.`match`(classOf[Command], fi(x => {
val inbox = TestInbox[Command]("transformMessagesListener")
JBehaviors.transformMessages(classOf[Command], super.behavior(monitor)._1, pf(_.`match`(classOf[Command], fi(x => {
inbox.ref ! x
x
})))) -> inbox

View file

@ -102,14 +102,14 @@ class DeferredSpec extends ScalaTestWithActorTestKit("""
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 behv = Behaviors
.setup[Command] { _ =>
probe.ref ! Started
target(probe.ref)
}
.widen[Command] {
.transformMessages[Command] {
case m => m
}
probe.expectNoMessage() // not yet

View file

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

View file

@ -4,6 +4,7 @@
package akka.actor.typed
import java.math.BigInteger
import java.util.concurrent.atomic.AtomicInteger
import akka.actor.ActorInitializationException
@ -13,10 +14,25 @@ import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.adapter._
import akka.testkit.EventFilter
import org.scalatest.WordSpecLike
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]
""") with WordSpecLike {
@ -28,12 +44,12 @@ class WidenSpec extends ScalaTestWithActorTestKit("""
probe ! message
Behaviors.same
}
.widen[Int] {
.transformMessages[Int] {
case n if n != 13 => n.toString
}
}
"Widen" should {
"transformMessages" should {
"transform from an outer type to an inner type" in {
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 transformCount = new AtomicInteger(0)
// 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 =>
transformCount.incrementAndGet()
s
}
def widen(behavior: Behavior[String]): Behavior[String] =
behavior.widen(transform)
def transform(behavior: Behavior[String]): Behavior[String] =
behavior.transformMessages(transformPF)
val beh =
widen(widen(Behaviors.receiveMessage[String] { message =>
transform(transform(Behaviors.receiveMessage[String] { message =>
probe.ref ! message
Behaviors.same
}))
@ -83,21 +99,21 @@ class WidenSpec extends ScalaTestWithActorTestKit("""
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 transformCount = new AtomicInteger(0)
// 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 =>
transformCount.incrementAndGet()
s
}
def widen(behavior: Behavior[String]): Behavior[String] =
behavior.widen(transform)
def transform(behavior: Behavior[String]): Behavior[String] =
behavior.transformMessages(transformPF)
def next: Behavior[String] =
widen(Behaviors.receiveMessage[String] { message =>
transform(Behaviors.receiveMessage[String] { message =>
probe.ref ! message
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]()
def widen(behavior: Behavior[String]): Behavior[String] =
behavior.widen[String] {
def transform(behavior: Behavior[String]): Behavior[String] =
behavior.transformMessages[String] {
case s => s.toLowerCase
}
EventFilter[ActorInitializationException](occurrences = 1).intercept {
val ref = spawn(widen(widen(Behaviors.receiveMessage[String] { _ =>
val ref = spawn(transform(transform(Behaviors.receiveMessage[String] { _ =>
Behaviors.same
})))
@ -142,7 +158,7 @@ class WidenSpec extends ScalaTestWithActorTestKit("""
Behaviors.same
}
}
.widen[String] {
.transformMessages[String] {
case msg => msg.toUpperCase()
}
@ -163,7 +179,7 @@ class WidenSpec extends ScalaTestWithActorTestKit("""
probe.ref ! msg
Behaviors.same
}
.widen[String] {
.transformMessages[String] {
case msg => msg.toUpperCase()
}
}

View file

@ -110,18 +110,24 @@ object Behavior {
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
* at) and may transform the incoming message to place them into the wrapped
* Behaviors type hierarchy. Signals are not transformed.
*
* Example:
* {{{
* receive[String] { (ctx, msg) => println(msg); same }.widen[Number] {
* case b: BigDecimal => s"BigDecimal(&dollar;b)"
* case i: BigInteger => s"BigInteger(&dollar;i)"
* // all other kinds of Number will be `unhandled`
* }
* 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`
* }
* }}}
*
* The `ClassTag` for `Outer` ensures that only messages of this class or a subclass thereof will be
@ -129,8 +135,8 @@ object Behavior {
* the interceptor and be continue to the inner behavior untouched.
*
*/
def widen[Outer: ClassTag](matcher: PartialFunction[Outer, Inner]): Behavior[Outer] =
BehaviorImpl.widened(behavior, matcher)
def transformMessages[Outer: ClassTag](matcher: PartialFunction[Outer, Inner]): Behavior[Outer] =
BehaviorImpl.transformMessages(behavior, matcher)
}

View file

@ -42,8 +42,8 @@ private[akka] object BehaviorTags {
def as[U]: AC[U] = ctx.asInstanceOf[AC[U]]
}
def widened[O: ClassTag, I](behavior: Behavior[I], matcher: PartialFunction[O, I]): Behavior[O] =
intercept(() => WidenedInterceptor(matcher))(behavior)
def transformMessages[O: ClassTag, I](behavior: Behavior[I], matcher: PartialFunction[O, I]): Behavior[O] =
intercept(() => TransformMessagesInterceptor(matcher))(behavior)
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
*/
@InternalApi
private[akka] object WidenedInterceptor {
private[akka] object TransformMessagesInterceptor {
private final val _any2null = (_: Any) => null
private final def any2null[T] = _any2null.asInstanceOf[Any => T]
@ -187,19 +187,19 @@ private[akka] object WidenedInterceptor {
* INTERNAL API
*/
@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] {
import BehaviorInterceptor._
import WidenedInterceptor._
import TransformMessagesInterceptor._
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
// "recursive" narrowed behaviors.
case WidenedInterceptor(`matcher`) => true
case WidenedInterceptor(otherMatcher) =>
case TransformMessagesInterceptor(`matcher`) => true
case TransformMessagesInterceptor(otherMatcher) =>
// there is no safe way to allow this
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)}")
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,25 +270,24 @@ 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
* at) and may transform the incoming message to place them into the wrapped
* Behaviors type hierarchy. Signals are not transformed.
*
* Example:
* {{{
* Behavior<String> s = Behaviors.receive((ctx, msg) -> {
* System.out.println(msg);
* return Behaviors.same();
* });
* Behavior<Number> n = Behaviors.widened(s, pf -> pf.
* match(BigInteger.class, i -> "BigInteger(" + i + ")").
* match(BigDecimal.class, d -> "BigDecimal(" + d + ")")
* 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
* );
* );
* }}}
*
*
* @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
* the interceptor and be continue to the inner behavior untouched.
@ -297,13 +296,13 @@ object Behaviors {
* @param selector
* a partial function builder for describing the selection and
* 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],
behavior: Behavior[Outer],
selector: JFunction[PFBuilder[Inner, Outer], PFBuilder[Inner, Outer]]): Behavior[Inner] =
BehaviorImpl.widened(behavior, selector.apply(new PFBuilder).build())(ClassTag(interceptMessageClass))
behavior: Behavior[Inner],
selector: JFunction[PFBuilder[Outer, Inner], PFBuilder[Outer, Inner]]): Behavior[Outer] =
BehaviorImpl.transformMessages(behavior, selector.apply(new PFBuilder).build())(ClassTag(interceptMessageClass))
/**
* 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`.
* 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`.
* `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].
`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`.

View file

@ -83,10 +83,10 @@ class EventSourcedBehaviorInterceptorSpec
probe.expectMessage("ABC")
}
"be possible to combine with widen" in {
"be possible to combine with transformMessages" in {
val probe = createTestProbe[String]()
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()
})