+str #18142 ask pattern integration for akka streams
progressed with cleanup, removing the same thread exec context is weird... causes issues :-/ Need to debug more, could be that some race also exists in mapAsync then :\ WIP finish ask impl via watch stage mima consistency spec fix paradox, and fix adding ask/watch to javadsl source follow up review
This commit is contained in:
parent
5040ce82f1
commit
4714f16dcf
18 changed files with 643 additions and 47 deletions
|
|
@ -7,10 +7,10 @@ package akka.actor
|
|||
import akka.AkkaException
|
||||
import akka.event.LoggingAdapter
|
||||
|
||||
import java.util.Optional
|
||||
import scala.annotation.tailrec
|
||||
import scala.beans.BeanProperty
|
||||
import scala.util.control.NoStackTrace
|
||||
import java.util.Optional
|
||||
|
||||
import akka.annotation.InternalApi
|
||||
|
||||
|
|
|
|||
|
|
@ -92,6 +92,7 @@ package akka.actor
|
|||
* }
|
||||
* }}}
|
||||
*/
|
||||
@Deprecated
|
||||
@deprecated("Use AbstractActor instead of UntypedActor.", since = "2.5.0")
|
||||
abstract class UntypedActor extends Actor {
|
||||
|
||||
|
|
|
|||
|
|
@ -1135,7 +1135,40 @@ Adheres to the `ActorAttributes.SupervisionStrategy` attribute.
|
|||
|
||||
**completes** when upstream completes and all elements have been emitted from the internal flow
|
||||
|
||||
**cancels** when downstream cancels
|
||||
**completes** when upstream completes and all futures have been completed and all elements have been emitted
|
||||
|
||||
---------------------------------------------------------------
|
||||
|
||||
### watch
|
||||
|
||||
Watch a specific `ActorRef` and signal a failure downstream once the actor terminates.
|
||||
The signaled failure will be an @java[@javadoc:[WatchedActorTerminatedException](akka.stream.WatchedActorTerminatedException)]
|
||||
@scala[@scaladoc[WatchedActorTerminatedException](akka.stream.WatchedActorTerminatedException)].
|
||||
|
||||
**emits** when upstream emits
|
||||
|
||||
**backpressures** when downstream backpressures
|
||||
|
||||
**completes** when upstream completes
|
||||
|
||||
---------------------------------------------------------------
|
||||
|
||||
### ask
|
||||
|
||||
Specialized stage implementing the @scala[@extref[ask](github:akka-actor/src/main/scala/akka/pattern/AskSupport.scala)]@java[@extref[ask](github:akka-actor/src/main/scala/akka/pattern/Patterns.scala)] pattern for inter-op with untyped actors.
|
||||
|
||||
The stream will be failed using an an @java[@javadoc:[WatchedActorTerminatedException](akka.stream.WatchedActorTerminatedException)]
|
||||
@scala[@scaladoc[WatchedActorTerminatedException](akka.stream.WatchedActorTerminatedException)] if the target actor terminates,
|
||||
or with an @java[@javadoc:[WatchedActorTerminatedException](akka.pattern.AskTimeoutException)] @scala[@scaladoc[WatchedActorTerminatedException](akka.pattern.AskTimeoutException)] if any of the asks times out.
|
||||
|
||||
**emits** when the futures (in submission order) created by the ask pattern internally are completed
|
||||
|
||||
**backpressures** when the number of futures reaches the configured parallelism and the downstream backpressures
|
||||
|
||||
**fails** when the passed in actor terminates, or a timeout is exceeded in any of the asks performed
|
||||
|
||||
**completes** when upstream completes and all futures have been completed and all elements have been emitted
|
||||
|
||||
|
||||
---------------------------------------------------------------
|
||||
|
||||
|
|
@ -1215,6 +1248,28 @@ If a @scala[`Future`] @java[`CompletionStage`] fails, the stream also fails (unl
|
|||
|
||||
**completes** upstream completes and all @scala[`Future` s] @java[`CompletionStage` s] has been completed and all elements has been emitted
|
||||
|
||||
---------------------------------------------------------------
|
||||
|
||||
### ask
|
||||
|
||||
Use the `ask` pattern to send a request-reply message to the target `ref` actor.
|
||||
If any of the asks times out it will fail the stream with a [[akka.pattern.AskTimeoutException]].
|
||||
|
||||
The `mapTo` class parameter is used to cast the incoming responses to the expected response type.
|
||||
|
||||
Similar to the plain ask pattern, the target actor is allowed to reply with `akka.util.Status`.
|
||||
An `akka.util.Status#Failure` will cause the stage to fail with the cause carried in the `Failure` message.
|
||||
|
||||
Adheres to the [[ActorAttributes.SupervisionStrategy]] attribute.
|
||||
|
||||
**emits** when the ask @scala[`Future`] @java[`CompletionStage`] returned by the provided function finishes for the next element in sequence
|
||||
|
||||
|
||||
**backpressures** when the number of ask @scala[`Future` s] @java[`CompletionStage` s] reaches the configured parallelism and the downstream backpressures
|
||||
|
||||
**completes** when upstream completes and all ask @scala[`Future` s] @java[`CompletionStage` s] has been completed and all elements has been emitted
|
||||
|
||||
|
||||
---------------------------------------------------------------
|
||||
|
||||
<br/>
|
||||
|
|
|
|||
|
|
@ -8,18 +8,20 @@ For piping the elements of a stream as messages to an ordinary actor you can use
|
|||
Messages can be sent to a stream with `Source.queue` or via the `ActorRef` that is
|
||||
materialized by `Source.actorRef`.
|
||||
|
||||
### mapAsync + ask
|
||||
### ask
|
||||
|
||||
A nice way to delegate some processing of elements in a stream to an actor is to
|
||||
use `ask` in `mapAsync`. The back-pressure of the stream is maintained by
|
||||
the @scala[`Future`]@java[`CompletionStage`] of the `ask` and the mailbox of the actor will not be filled with
|
||||
more messages than the given `parallelism` of the `mapAsync` stage.
|
||||
### ask
|
||||
|
||||
A nice way to delegate some processing of elements in a stream to an actor is to use `ask`.
|
||||
The back-pressure of the stream is maintained by the @scala[`Future`]@java[`CompletionStage`] of
|
||||
the `ask` and the mailbox of the actor will not be filled with more messages than the given
|
||||
`parallelism` of the `ask` stage (similarly to how the `mapAsync` stage works).
|
||||
|
||||
Scala
|
||||
: @@snip [IntegrationDocSpec.scala]($code$/scala/docs/stream/IntegrationDocSpec.scala) { #mapAsync-ask }
|
||||
: @@snip [IntegrationDocSpec.scala]($code$/scala/docs/stream/IntegrationDocSpec.scala) { #ask }
|
||||
|
||||
Java
|
||||
: @@snip [IntegrationDocTest.java]($code$/java/jdocs/stream/IntegrationDocTest.java) { #mapAsync-ask }
|
||||
: @@snip [IntegrationDocTest.java]($code$/java/jdocs/stream/IntegrationDocTest.java) { #ask }
|
||||
|
||||
Note that the messages received in the actor will be in the same order as
|
||||
the stream elements, i.e. the `parallelism` does not change the ordering
|
||||
|
|
@ -29,8 +31,9 @@ is already a message in the mailbox when the actor has completed previous
|
|||
message.
|
||||
|
||||
The actor must reply to the @scala[`sender()`]@java[`getSender()`] for each message from the stream. That
|
||||
reply will complete the @scala[`Future`]@java[`CompletionStage`] of the `ask` and it will be the element that
|
||||
is emitted downstreams from `mapAsync`.
|
||||
reply will complete the @scala[`Future`]@java[`CompletionStage`] of the `ask` and it will be the element that is emitted downstreams.
|
||||
|
||||
In case the target actor is stopped, the stage will fail with an `AskStageTargetActorTerminatedException`
|
||||
|
||||
Scala
|
||||
: @@snip [IntegrationDocSpec.scala]($code$/scala/docs/stream/IntegrationDocSpec.scala) { #ask-actor }
|
||||
|
|
@ -38,20 +41,21 @@ Scala
|
|||
Java
|
||||
: @@snip [IntegrationDocTest.java]($code$/java/jdocs/stream/IntegrationDocTest.java) { #ask-actor }
|
||||
|
||||
The stream can be completed with failure by sending `akka.actor.Status.Failure`
|
||||
as reply from the actor.
|
||||
The stream can be completed with failure by sending `akka.actor.Status.Failure` as reply from the actor.
|
||||
|
||||
If the `ask` fails due to timeout the stream will be completed with
|
||||
`TimeoutException` failure. If that is not desired outcome you can use `recover`
|
||||
on the `ask` @scala[`Future`]@java[`CompletionStage`].
|
||||
on the `ask` @scala[`Future`]@java[`CompletionStage`], or use the other "restart" stages to restart it.
|
||||
|
||||
If you don't care about the reply values and only use them as back-pressure signals you
|
||||
can use `Sink.ignore` after the `mapAsync` stage and then actor is effectively a sink
|
||||
can use `Sink.ignore` after the `ask` stage and then actor is effectively a sink
|
||||
of the stream.
|
||||
|
||||
The same pattern can be used with @ref:[Actor routers](../routing.md). Then you
|
||||
can use `mapAsyncUnordered` for better efficiency if you don't care about the
|
||||
order of the emitted downstream elements (the replies).
|
||||
Note that while you may implement the same concept using `mapAsync`, that style would not be aware of the actor terminating.
|
||||
|
||||
If you are intending to ask multiple actors by using @ref:[Actor routers](../routing.md), then
|
||||
you should use `mapAsyncUnordered` and perform the ask manually in there, as the ordering of the replies is not important,
|
||||
since multiple actors are being asked concurrently to begin with, and no single actor is the one to be watched by the stage.
|
||||
|
||||
### Sink.actorRefWithAck
|
||||
|
||||
|
|
|
|||
|
|
@ -311,19 +311,18 @@ public class IntegrationDocTest extends AbstractJavaTest {
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Test
|
||||
public void mapAsyncPlusAsk() throws Exception {
|
||||
//#mapAsync-ask
|
||||
public void askStage() throws Exception {
|
||||
//#ask
|
||||
Source<String, NotUsed> words =
|
||||
Source.from(Arrays.asList("hello", "hi"));
|
||||
Timeout askTimeout = Timeout.apply(5, TimeUnit.SECONDS);
|
||||
|
||||
words
|
||||
.mapAsync(5, elem -> ask(ref, elem, askTimeout))
|
||||
.map(elem -> (String) elem)
|
||||
.ask(5, ref, String.class, askTimeout)
|
||||
// continue processing of the replies from the actor
|
||||
.map(elem -> elem.toLowerCase())
|
||||
.runWith(Sink.ignore(), mat);
|
||||
//#mapAsync-ask
|
||||
//#ask
|
||||
}
|
||||
|
||||
|
||||
|
|
|
|||
|
|
@ -140,19 +140,18 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) {
|
|||
implicit val materializer = ActorMaterializer()
|
||||
val ref: ActorRef = system.actorOf(Props[Translator])
|
||||
|
||||
"mapAsync + ask" in {
|
||||
//#mapAsync-ask
|
||||
import akka.pattern.ask
|
||||
"ask" in {
|
||||
//#ask
|
||||
implicit val askTimeout = Timeout(5.seconds)
|
||||
val words: Source[String, NotUsed] =
|
||||
Source(List("hello", "hi"))
|
||||
|
||||
words
|
||||
.mapAsync(parallelism = 5)(elem ⇒ (ref ? elem).mapTo[String])
|
||||
.ask[String](parallelism = 5)(ref)
|
||||
// continue processing of the replies from the actor
|
||||
.map(_.toLowerCase)
|
||||
.runWith(Sink.ignore)
|
||||
//#mapAsync-ask
|
||||
//#ask
|
||||
}
|
||||
|
||||
"calling external service with mapAsync" in {
|
||||
|
|
|
|||
|
|
@ -42,13 +42,13 @@ class DslConsistencySpec extends WordSpec with Matchers {
|
|||
val graphHelpers = Set("zipGraph", "zipWithGraph", "mergeGraph", "mergeSortedGraph", "interleaveGraph", "concatGraph", "prependGraph", "alsoToGraph", "wireTapGraph", "orElseGraph", "divertToGraph")
|
||||
val allowMissing: Map[Class[_], Set[String]] = Map(
|
||||
jFlowClass → graphHelpers,
|
||||
jSourceClass → graphHelpers,
|
||||
jSourceClass → (graphHelpers ++ Set("watch", "ask")),
|
||||
// Java subflows can only be nested using .via and .to (due to type system restrictions)
|
||||
jSubFlowClass → (graphHelpers ++ Set("groupBy", "splitAfter", "splitWhen", "subFlow")),
|
||||
jSubSourceClass → (graphHelpers ++ Set("groupBy", "splitAfter", "splitWhen", "subFlow")),
|
||||
jSubFlowClass → (graphHelpers ++ Set("groupBy", "splitAfter", "splitWhen", "subFlow", "watch", "ask")),
|
||||
jSubSourceClass → (graphHelpers ++ Set("groupBy", "splitAfter", "splitWhen", "subFlow", "watch", "ask")),
|
||||
|
||||
sFlowClass → Set("of"),
|
||||
sSourceClass → Set("adapt", "from"),
|
||||
sSourceClass → Set("adapt", "from", "watch"),
|
||||
sSinkClass → Set("adapt"),
|
||||
sSubFlowClass → Set(),
|
||||
sSubSourceClass → Set(),
|
||||
|
|
|
|||
|
|
@ -0,0 +1,243 @@
|
|||
/**
|
||||
* Copyright (C) 2014-2017 Lightbend Inc. <http://www.lightbend.com>
|
||||
*/
|
||||
package akka.stream.scaladsl
|
||||
|
||||
import java.util.concurrent.{ CountDownLatch, LinkedBlockingQueue, ThreadLocalRandom }
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
|
||||
import akka.actor.{ Actor, ActorRef, PoisonPill, Props }
|
||||
import akka.stream.ActorAttributes.{ SupervisionStrategy, supervisionStrategy }
|
||||
import akka.stream.{ ActorAttributes, ActorMaterializer, Supervision }
|
||||
import akka.stream.Supervision.resumingDecider
|
||||
import akka.stream.impl.ReactiveStreamsCompliance
|
||||
import akka.stream.testkit.Utils._
|
||||
import akka.stream.testkit._
|
||||
import akka.stream.testkit.scaladsl.TestSink
|
||||
import akka.testkit.{ TestActors, TestLatch, TestProbe }
|
||||
import org.scalatest.concurrent.PatienceConfiguration.Timeout
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import scala.concurrent.{ Await, Future, Promise }
|
||||
import scala.concurrent.duration._
|
||||
import scala.reflect.ClassTag
|
||||
import scala.util.control.NoStackTrace
|
||||
|
||||
object FlowAskSpec {
|
||||
case class Reply(payload: Int)
|
||||
|
||||
class Replier extends Actor {
|
||||
override def receive: Receive = {
|
||||
case msg: Int ⇒ sender() ! Reply(msg)
|
||||
}
|
||||
}
|
||||
|
||||
class ReplyAndProxy(to: ActorRef) extends Actor {
|
||||
override def receive: Receive = {
|
||||
case msg: Int ⇒
|
||||
to ! msg
|
||||
sender() ! Reply(msg)
|
||||
}
|
||||
}
|
||||
|
||||
class RandomDelaysReplier extends Actor {
|
||||
override def receive: Receive = {
|
||||
case msg: Int ⇒
|
||||
import context.dispatcher
|
||||
|
||||
val replyTo = sender()
|
||||
Future {
|
||||
Thread.sleep(ThreadLocalRandom.current().nextInt(1, 10))
|
||||
replyTo ! Reply(msg)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class StatusReplier extends Actor {
|
||||
override def receive: Receive = {
|
||||
case msg: Int ⇒ sender() ! akka.actor.Status.Success(Reply(msg))
|
||||
}
|
||||
}
|
||||
|
||||
class FailOn(n: Int) extends Actor {
|
||||
override def receive: Receive = {
|
||||
case `n` ⇒ sender() ! akka.actor.Status.Failure(new Exception(s"Booming for $n!"))
|
||||
case msg: Int ⇒ sender() ! akka.actor.Status.Success(Reply(msg))
|
||||
}
|
||||
}
|
||||
|
||||
class FailOnAllExcept(n: Int) extends Actor {
|
||||
override def receive: Receive = {
|
||||
case `n` ⇒ sender() ! akka.actor.Status.Success(Reply(n))
|
||||
case msg: Int ⇒ sender() ! akka.actor.Status.Failure(new Exception(s"Booming for $n!"))
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
class FlowAskSpec extends StreamSpec {
|
||||
import FlowAskSpec._
|
||||
|
||||
implicit val materializer = ActorMaterializer()
|
||||
|
||||
"A Flow with ask" must {
|
||||
|
||||
implicit val timeout = akka.util.Timeout(10.seconds)
|
||||
|
||||
val replyOnInts = system.actorOf(Props(classOf[Replier]).withDispatcher("akka.test.stream-dispatcher"), "replyOnInts")
|
||||
|
||||
val dontReply = system.actorOf(TestActors.blackholeProps.withDispatcher("akka.test.stream-dispatcher"), "dontReply")
|
||||
|
||||
val replyRandomDelays = system.actorOf(Props(classOf[RandomDelaysReplier]).withDispatcher("akka.test.stream-dispatcher"), "replyRandomDelays")
|
||||
|
||||
val statusReplier = system.actorOf(Props(new StatusReplier).withDispatcher("akka.test.stream-dispatcher"), "statusReplier")
|
||||
|
||||
def replierAndProxyTo(ref: ActorRef) = system.actorOf(Props(new ReplyAndProxy(ref)).withDispatcher("akka.test.stream-dispatcher"), s"reply-and-proxy-${ref.hashCode}")
|
||||
|
||||
def replierFailOn(n: Int) = system.actorOf(Props(new FailOn(n)).withDispatcher("akka.test.stream-dispatcher"), s"failureReplier-$n")
|
||||
val failsOn1 = replierFailOn(1)
|
||||
val failsOn3 = replierFailOn(3)
|
||||
|
||||
def replierFailAllExceptOn(n: Int) = system.actorOf(Props(new FailOnAllExcept(n)).withDispatcher("akka.test.stream-dispatcher"), s"failureReplier-$n")
|
||||
val failAllExcept6 = replierFailAllExceptOn(6)
|
||||
|
||||
"produce asked elements" in assertAllStagesStopped {
|
||||
val c = TestSubscriber.manualProbe[Reply]()
|
||||
implicit val ec = system.dispatcher
|
||||
val p = Source(1 to 3).ask[Reply](4)(replyOnInts).runWith(Sink.fromSubscriber(c))
|
||||
val sub = c.expectSubscription()
|
||||
sub.request(2)
|
||||
c.expectNext(Reply(1))
|
||||
c.expectNext(Reply(2))
|
||||
c.expectNoMessage(200.millis)
|
||||
sub.request(2)
|
||||
c.expectNext(Reply(3))
|
||||
c.expectComplete()
|
||||
}
|
||||
"produce asked elements, when replies are akka.actor.Status.Success" in assertAllStagesStopped {
|
||||
val c = TestSubscriber.manualProbe[Reply]()
|
||||
implicit val ec = system.dispatcher
|
||||
val p = Source(1 to 3).ask[Reply](4)(statusReplier).runWith(Sink.fromSubscriber(c))
|
||||
val sub = c.expectSubscription()
|
||||
sub.request(2)
|
||||
c.expectNext(Reply(1))
|
||||
c.expectNext(Reply(2))
|
||||
c.expectNoMessage(200.millis)
|
||||
sub.request(2)
|
||||
c.expectNext(Reply(3))
|
||||
c.expectComplete()
|
||||
}
|
||||
|
||||
"produce future elements in order" in {
|
||||
val c = TestSubscriber.manualProbe[Reply]()
|
||||
implicit val ec = system.dispatcher
|
||||
val p = Source(1 to 50).ask[Reply](4)(replyRandomDelays).to(Sink.fromSubscriber(c)).run()
|
||||
val sub = c.expectSubscription()
|
||||
sub.request(1000)
|
||||
for (n ← 1 to 50) c.expectNext(Reply(n))
|
||||
c.expectComplete()
|
||||
}
|
||||
|
||||
"signal ask timeout failure" in assertAllStagesStopped {
|
||||
val c = TestSubscriber.manualProbe[Reply]()
|
||||
implicit val ec = system.dispatcher
|
||||
Source(1 to 5).map(_ + " nope")
|
||||
.ask[Reply](4)(dontReply)(akka.util.Timeout(10.millis), implicitly[ClassTag[Reply]])
|
||||
.to(Sink.fromSubscriber(c)).run()
|
||||
c.expectSubscription().request(10)
|
||||
c.expectError().getMessage should startWith("Ask timed out on [Actor[akka://FlowAskSpec/user/dontReply#")
|
||||
}
|
||||
|
||||
"signal ask failure" in assertAllStagesStopped {
|
||||
val c = TestSubscriber.manualProbe[Reply]()
|
||||
val ref = failsOn1
|
||||
implicit val ec = system.dispatcher
|
||||
val p = Source(1 to 5).ask[Reply](4)(ref).to(Sink.fromSubscriber(c)).run()
|
||||
val sub = c.expectSubscription()
|
||||
sub.request(10)
|
||||
c.expectError().getMessage should be("Booming for 1!")
|
||||
}
|
||||
|
||||
"signal failure when target actor is terminated" in assertAllStagesStopped {
|
||||
val r = system.actorOf(Props(classOf[Replier]).withDispatcher("akka.test.stream-dispatcher"), "wanna-fail")
|
||||
val done = Source.maybe[Int]
|
||||
.ask[Reply](4)(r).runWith(Sink.ignore)
|
||||
|
||||
intercept[RuntimeException] {
|
||||
r ! PoisonPill
|
||||
Await.result(done, remainingOrDefault)
|
||||
}.getMessage should startWith("Actor watched by [ask()] has terminated! Was: Actor[akka://FlowAskSpec/user/wanna-fail#")
|
||||
}
|
||||
|
||||
"a failure mid-stream must skip element with resume strategy" in assertAllStagesStopped {
|
||||
val p = TestProbe()
|
||||
|
||||
val input = "a" :: "b" :: "c" :: "d" :: "e" :: "f" :: Nil
|
||||
|
||||
val elements = Source.fromIterator(() ⇒ input.iterator)
|
||||
.ask[String](5)(p.ref)
|
||||
.withAttributes(ActorAttributes.supervisionStrategy(Supervision.resumingDecider))
|
||||
.runWith(Sink.seq)
|
||||
|
||||
// the problematic ordering:
|
||||
p.expectMsg("a")
|
||||
p.lastSender ! "a"
|
||||
|
||||
p.expectMsg("b")
|
||||
p.lastSender ! "b"
|
||||
|
||||
p.expectMsg("c")
|
||||
val cSender = p.lastSender
|
||||
|
||||
p.expectMsg("d")
|
||||
p.lastSender ! "d"
|
||||
|
||||
p.expectMsg("e")
|
||||
p.lastSender ! "e"
|
||||
|
||||
p.expectMsg("f")
|
||||
p.lastSender ! "f"
|
||||
|
||||
cSender ! akka.actor.Status.Failure(new Exception("Booom!"))
|
||||
|
||||
elements.futureValue should ===(List("a", "b", /* no c */ "d", "e", "f"))
|
||||
}
|
||||
|
||||
"resume after ask failure" in assertAllStagesStopped {
|
||||
val c = TestSubscriber.manualProbe[Reply]()
|
||||
implicit val ec = system.dispatcher
|
||||
val ref = failsOn3
|
||||
val p = Source(1 to 5)
|
||||
.ask[Reply](4)(ref)
|
||||
.withAttributes(supervisionStrategy(resumingDecider))
|
||||
.to(Sink.fromSubscriber(c)).run()
|
||||
val sub = c.expectSubscription()
|
||||
sub.request(10)
|
||||
for (n ← List(1, 2, 4, 5)) c.expectNext(Reply(n))
|
||||
c.expectComplete()
|
||||
}
|
||||
|
||||
"resume after multiple failures" in assertAllStagesStopped {
|
||||
Await.result(
|
||||
Source(1 to 6)
|
||||
.ask[Reply](2)(failAllExcept6).withAttributes(supervisionStrategy(resumingDecider))
|
||||
.runWith(Sink.head), 3.seconds) should ===(Reply(6))
|
||||
}
|
||||
|
||||
"should handle cancel properly" in assertAllStagesStopped {
|
||||
val pub = TestPublisher.manualProbe[Int]()
|
||||
val sub = TestSubscriber.manualProbe[Reply]()
|
||||
|
||||
Source.fromPublisher(pub).ask[Reply](4)(dontReply).runWith(Sink.fromSubscriber(sub))
|
||||
|
||||
val upstream = pub.expectSubscription()
|
||||
upstream.expectRequest()
|
||||
|
||||
sub.expectSubscription().cancel()
|
||||
|
||||
upstream.expectCancellation()
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,79 @@
|
|||
/**
|
||||
* Copyright (C) 2014-2017 Lightbend Inc. <http://www.lightbend.com>
|
||||
*/
|
||||
package akka.stream.scaladsl
|
||||
|
||||
import akka.actor.{ Actor, PoisonPill, Props }
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.testkit.Utils._
|
||||
import akka.stream.testkit._
|
||||
import akka.testkit.TestActors
|
||||
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
|
||||
object FlowWatchSpec {
|
||||
case class Reply(payload: Int)
|
||||
|
||||
class Replier extends Actor {
|
||||
override def receive: Receive = {
|
||||
case msg: Int ⇒ sender() ! Reply(msg)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
class FlowWatchSpec extends StreamSpec {
|
||||
import FlowWatchSpec._
|
||||
|
||||
implicit val materializer = ActorMaterializer()
|
||||
|
||||
"A Flow with watch" must {
|
||||
|
||||
implicit val timeout = akka.util.Timeout(10.seconds)
|
||||
|
||||
val replyOnInts = system.actorOf(Props(classOf[Replier]).withDispatcher("akka.test.stream-dispatcher"), "replyOnInts")
|
||||
|
||||
val dontReply = system.actorOf(TestActors.blackholeProps.withDispatcher("akka.test.stream-dispatcher"), "dontReply")
|
||||
|
||||
"pass through elements while actor is alive" in assertAllStagesStopped {
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
implicit val ec = system.dispatcher
|
||||
val p = Source(1 to 3).watch(replyOnInts).runWith(Sink.fromSubscriber(c))
|
||||
val sub = c.expectSubscription()
|
||||
sub.request(2)
|
||||
c.expectNext(1)
|
||||
c.expectNext(2)
|
||||
c.expectNoMessage(200.millis)
|
||||
sub.request(2)
|
||||
c.expectNext(3)
|
||||
c.expectComplete()
|
||||
}
|
||||
|
||||
"signal failure when target actor is terminated" in assertAllStagesStopped {
|
||||
val r = system.actorOf(Props(classOf[Replier]).withDispatcher("akka.test.stream-dispatcher"), "wanna-fail")
|
||||
val done = Source.maybe[Int].watch(r).runWith(Sink.ignore)
|
||||
|
||||
intercept[RuntimeException] {
|
||||
r ! PoisonPill
|
||||
Await.result(done, remainingOrDefault)
|
||||
}.getMessage should startWith("Actor watched by [Watch] has terminated! Was: Actor[akka://FlowWatchSpec/user/wanna-fail#")
|
||||
}
|
||||
|
||||
"should handle cancel properly" in assertAllStagesStopped {
|
||||
val pub = TestPublisher.manualProbe[Int]()
|
||||
val sub = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
Source.fromPublisher(pub).watch(dontReply).runWith(Sink.fromSubscriber(sub))
|
||||
|
||||
val upstream = pub.expectSubscription()
|
||||
upstream.expectRequest()
|
||||
|
||||
sub.expectSubscription().cancel()
|
||||
|
||||
upstream.expectCancellation()
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
@ -56,7 +56,7 @@ class FlowWatchTerminationSpec extends StreamSpec {
|
|||
sinkProbe.request(5)
|
||||
sourceProbe.sendNext(1)
|
||||
sinkProbe.expectNext(1)
|
||||
expectNoMsg(300.millis)
|
||||
expectNoMessage(300.millis)
|
||||
|
||||
sourceProbe.sendComplete()
|
||||
expectMsg(Done)
|
||||
|
|
|
|||
|
|
@ -15,3 +15,9 @@ ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.ActorRefSou
|
|||
|
||||
# #24254 add collectType
|
||||
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.scaladsl.FlowOps.collectType")
|
||||
|
||||
# #24325 ask stage
|
||||
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.scaladsl.FlowOps.ask")
|
||||
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.scaladsl.FlowOps.watch")
|
||||
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.javadsl.FlowOps.ask")
|
||||
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.javadsl.FlowOps.watch")
|
||||
|
|
|
|||
|
|
@ -0,0 +1,13 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
package akka.stream
|
||||
|
||||
import akka.actor.ActorRef
|
||||
|
||||
/**
|
||||
* Used as failure exception by an `ask` stage if the target actor terminates.
|
||||
* See `Flow.ask` and `Flow.watch`.
|
||||
*/
|
||||
final class WatchedActorTerminatedException(val watchingStageName: String, val ref: ActorRef)
|
||||
extends RuntimeException(s"Actor watched by [$watchingStageName] has terminated! Was: $ref")
|
||||
|
|
@ -28,6 +28,7 @@ import akka.stream._
|
|||
val recover = name("recover")
|
||||
val mapAsync = name("mapAsync")
|
||||
val mapAsyncUnordered = name("mapAsyncUnordered")
|
||||
val ask = name("ask")
|
||||
val grouped = name("grouped")
|
||||
val groupedWithin = name("groupedWithin")
|
||||
val groupedWeightedWithin = name("groupedWeightedWithin")
|
||||
|
|
@ -89,6 +90,7 @@ import akka.stream._
|
|||
val delay = name("delay")
|
||||
|
||||
val terminationWatcher = name("terminationWatcher")
|
||||
val watch = name("watch")
|
||||
|
||||
val publisherSource = name("publisherSource")
|
||||
val iterableSource = name("iterableSource")
|
||||
|
|
|
|||
|
|
@ -5,10 +5,12 @@ package akka.stream.impl.fusing
|
|||
|
||||
import java.util.concurrent.TimeUnit.NANOSECONDS
|
||||
|
||||
import akka.actor.{ ActorRef, Terminated }
|
||||
import akka.annotation.{ DoNotInherit, InternalApi }
|
||||
import akka.dispatch.ExecutionContexts
|
||||
import akka.event.Logging.LogLevel
|
||||
import akka.event.{ LogSource, Logging, LoggingAdapter }
|
||||
import akka.pattern.AskSupport
|
||||
import akka.stream.Attributes.{ InputBuffer, LogLevels }
|
||||
import akka.stream.OverflowStrategies._
|
||||
import akka.stream.impl.fusing.GraphStages.SimpleLinearGraphStage
|
||||
|
|
@ -27,7 +29,9 @@ import akka.stream.ActorAttributes.SupervisionStrategy
|
|||
|
||||
import scala.concurrent.duration.{ FiniteDuration, _ }
|
||||
import akka.stream.impl.Stages.DefaultAttributes
|
||||
import akka.util.OptionVal
|
||||
import akka.util.{ OptionVal, Timeout }
|
||||
|
||||
import scala.reflect.ClassTag
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -1325,6 +1329,33 @@ private[stream] object Collect {
|
|||
}
|
||||
}
|
||||
|
||||
@InternalApi private[akka] final case class Watch[T](targetRef: ActorRef) extends SimpleLinearGraphStage[T] {
|
||||
|
||||
override def initialAttributes = DefaultAttributes.watch
|
||||
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic =
|
||||
new GraphStageLogic(shape) with InHandler with OutHandler with StageLogging {
|
||||
|
||||
private lazy val self = getStageActor {
|
||||
case (_, Terminated(`targetRef`)) ⇒
|
||||
failStage(new WatchedActorTerminatedException("Watch", targetRef))
|
||||
}
|
||||
|
||||
override def preStart(): Unit = {
|
||||
// initialize self, and watch the target
|
||||
self.watch(targetRef)
|
||||
}
|
||||
|
||||
override def onPull(): Unit =
|
||||
pull(in)
|
||||
|
||||
override def onPush(): Unit =
|
||||
push(out, grab(in))
|
||||
|
||||
setHandlers(in, out, this)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
|
|
|
|||
|
|
@ -3,7 +3,7 @@
|
|||
*/
|
||||
package akka.stream.javadsl
|
||||
|
||||
import akka.util.ConstantFun
|
||||
import akka.util.{ ConstantFun, Timeout }
|
||||
import akka.{ Done, NotUsed }
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.japi.{ Pair, function }
|
||||
|
|
@ -16,6 +16,7 @@ import akka.japi.Util
|
|||
import java.util.Comparator
|
||||
import java.util.concurrent.CompletionStage
|
||||
|
||||
import akka.actor.ActorRef
|
||||
import akka.dispatch.ExecutionContexts
|
||||
import akka.stream.impl.fusing.LazyFlow
|
||||
|
||||
|
|
@ -528,10 +529,10 @@ final class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends
|
|||
*
|
||||
* '''Emits when''' the CompletionStage returned by the provided function finishes for the next element in sequence
|
||||
*
|
||||
* '''Backpressures when''' the number of futures reaches the configured parallelism and the downstream
|
||||
* '''Backpressures when''' the number of CompletionStages reaches the configured parallelism and the downstream
|
||||
* backpressures or the first future is not completed
|
||||
*
|
||||
* '''Completes when''' upstream completes and all futures have been completed and all elements have been emitted
|
||||
* '''Completes when''' upstream completes and all CompletionStages have been completed and all elements have been emitted
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*
|
||||
|
|
@ -563,9 +564,9 @@ final class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends
|
|||
*
|
||||
* '''Emits when''' any of the CompletionStages returned by the provided function complete
|
||||
*
|
||||
* '''Backpressures when''' the number of futures reaches the configured parallelism and the downstream backpressures
|
||||
* '''Backpressures when''' the number of CompletionStages reaches the configured parallelism and the downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes and all futures have been completed and all elements have been emitted
|
||||
* '''Completes when''' upstream completes and all CompletionStages have been completed and all elements have been emitted
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*
|
||||
|
|
@ -574,6 +575,52 @@ final class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends
|
|||
def mapAsyncUnordered[T](parallelism: Int, f: function.Function[Out, CompletionStage[T]]): javadsl.Flow[In, T, Mat] =
|
||||
new Flow(delegate.mapAsyncUnordered(parallelism)(x ⇒ f(x).toScala))
|
||||
|
||||
/**
|
||||
* Use the `ask` pattern to send a request-reply message to the target `ref` actor.
|
||||
* If any of the asks times out it will fail the stream with a [[akka.pattern.AskTimeoutException]].
|
||||
*
|
||||
* The `mapTo` class parameter is used to cast the incoming responses to the expected response type.
|
||||
*
|
||||
* Similar to the plain ask pattern, the target actor is allowed to reply with `akka.util.Status`.
|
||||
* An `akka.util.Status#Failure` will cause the stage to fail with the cause carried in the `Failure` message.
|
||||
*
|
||||
* Parallelism limits the number of how many asks can be "in flight" at the same time.
|
||||
* Please note that the elements emitted by this stage are in-order with regards to the asks being issued
|
||||
* (i.e. same behaviour as mapAsync).
|
||||
*
|
||||
* The stage fails with an [[akka.stream.WatchedActorTerminatedException]] if the target actor is terminated.
|
||||
*
|
||||
* Adheres to the [[ActorAttributes.SupervisionStrategy]] attribute.
|
||||
*
|
||||
* '''Emits when''' any of the CompletionStages returned by the provided function complete
|
||||
*
|
||||
* '''Backpressures when''' the number of futures reaches the configured parallelism and the downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes and all futures have been completed and all elements have been emitted
|
||||
*
|
||||
* '''Fails when''' the passed in actor terminates, or a timeout is exceeded in any of the asks performed
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
def ask[S](parallelism: Int, ref: ActorRef, mapTo: Class[S], timeout: Timeout): javadsl.Flow[In, S, Mat] =
|
||||
new Flow(delegate.ask[S](parallelism)(ref)(timeout, ClassTag(mapTo)))
|
||||
|
||||
/**
|
||||
* The stage fails with an [[akka.stream.WatchedActorTerminatedException]] if the target actor is terminated.
|
||||
*
|
||||
* '''Emits when''' upstream emits
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes
|
||||
*
|
||||
* '''Fails when''' the watched actor terminates
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
def watch(ref: ActorRef): javadsl.Flow[In, Out, Mat] =
|
||||
new Flow(delegate.watch(ref))
|
||||
|
||||
/**
|
||||
* Only pass on those elements that satisfy the given predicate.
|
||||
*
|
||||
|
|
|
|||
|
|
@ -6,7 +6,7 @@ package akka.stream.javadsl
|
|||
import java.util
|
||||
import java.util.Optional
|
||||
|
||||
import akka.util.ConstantFun
|
||||
import akka.util.{ ConstantFun, Timeout }
|
||||
import akka.{ Done, NotUsed }
|
||||
import akka.actor.{ ActorRef, Cancellable, Props }
|
||||
import akka.event.LoggingAdapter
|
||||
|
|
@ -1253,6 +1253,52 @@ final class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Grap
|
|||
def mapAsyncUnordered[T](parallelism: Int, f: function.Function[Out, CompletionStage[T]]): javadsl.Source[T, Mat] =
|
||||
new Source(delegate.mapAsyncUnordered(parallelism)(x ⇒ f(x).toScala))
|
||||
|
||||
/**
|
||||
* Use the `ask` pattern to send a request-reply message to the target `ref` actor.
|
||||
* If any of the asks times out it will fail the stream with a [[akka.pattern.AskTimeoutException]].
|
||||
*
|
||||
* The `mapTo` class parameter is used to cast the incoming responses to the expected response type.
|
||||
*
|
||||
* Similar to the plain ask pattern, the target actor is allowed to reply with `akka.util.Status`.
|
||||
* An `akka.util.Status#Failure` will cause the stage to fail with the cause carried in the `Failure` message.
|
||||
*
|
||||
* Parallelism limits the number of how many asks can be "in flight" at the same time.
|
||||
* Please note that the elements emitted by this stage are in-order with regards to the asks being issued
|
||||
* (i.e. same behaviour as mapAsync).
|
||||
*
|
||||
* The stage fails with an [[akka.stream.WatchedActorTerminatedException]] if the target actor is terminated.
|
||||
*
|
||||
* Adheres to the [[ActorAttributes.SupervisionStrategy]] attribute.
|
||||
*
|
||||
* '''Emits when''' any of the CompletionStages returned by the provided function complete
|
||||
*
|
||||
* '''Backpressures when''' the number of futures reaches the configured parallelism and the downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes and all futures have been completed and all elements have been emitted
|
||||
*
|
||||
* '''Fails when''' the passed in actor terminates, or a timeout is exceeded in any of the asks performed
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
def ask[S](parallelism: Int, ref: ActorRef, mapTo: Class[S], timeout: Timeout): javadsl.Source[S, Mat] =
|
||||
new Source(delegate.ask[S](parallelism)(ref)(timeout, ClassTag(mapTo)))
|
||||
|
||||
/**
|
||||
* The stage fails with an [[akka.stream.WatchedActorTerminatedException]] if the target actor is terminated.
|
||||
*
|
||||
* '''Emits when''' upstream emits
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes
|
||||
*
|
||||
* '''Fails when''' the watched actor terminates
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
def watch(ref: ActorRef): javadsl.Source[Out, Mat] =
|
||||
new Source(delegate.watch(ref))
|
||||
|
||||
/**
|
||||
* Only pass on those elements that satisfy the given predicate.
|
||||
*
|
||||
|
|
|
|||
|
|
@ -9,7 +9,7 @@ import akka.Done
|
|||
import akka.stream.impl._
|
||||
import akka.stream.impl.fusing._
|
||||
import akka.stream.stage._
|
||||
import akka.util.ConstantFun
|
||||
import akka.util.{ ConstantFun, Timeout }
|
||||
import org.reactivestreams.{ Processor, Publisher, Subscriber, Subscription }
|
||||
|
||||
import scala.annotation.unchecked.uncheckedVariance
|
||||
|
|
@ -19,8 +19,10 @@ import scala.concurrent.duration.FiniteDuration
|
|||
import scala.language.higherKinds
|
||||
import akka.stream.impl.fusing.FlattenMerge
|
||||
import akka.NotUsed
|
||||
import akka.actor.ActorRef
|
||||
import akka.annotation.DoNotInherit
|
||||
|
||||
import scala.annotation.implicitNotFound
|
||||
import scala.reflect.ClassTag
|
||||
|
||||
/**
|
||||
|
|
@ -845,6 +847,75 @@ trait FlowOps[+Out, +Mat] {
|
|||
*/
|
||||
def mapAsyncUnordered[T](parallelism: Int)(f: Out ⇒ Future[T]): Repr[T] = via(MapAsyncUnordered(parallelism, f))
|
||||
|
||||
/**
|
||||
* Use the `ask` pattern to send a request-reply message to the target `ref` actor.
|
||||
* If any of the asks times out it will fail the stream with a [[akka.pattern.AskTimeoutException]].
|
||||
*
|
||||
* Do not forget to include the expected response type in the method call, like so:
|
||||
*
|
||||
* '''
|
||||
* flow.ask[ExpectedReply](ref)
|
||||
* '''
|
||||
*
|
||||
* otherwise `Nothing` will be assumed, which is most likely not what you want.
|
||||
*
|
||||
* Parallelism limits the number of how many asks can be "in flight" at the same time.
|
||||
* Please note that the elements emitted by this stage are in-order with regards to the asks being issued
|
||||
* (i.e. same behaviour as mapAsync).
|
||||
*
|
||||
* The mapTo class parameter is used to cast the incoming responses to the expected response type.
|
||||
*
|
||||
* Similar to the plain ask pattern, the target actor is allowed to reply with `akka.util.Status`.
|
||||
* An `akka.util.Status#Failure` will cause the stage to fail with the cause carried in the `Failure` message.
|
||||
*
|
||||
* The stage fails with an [[akka.stream.WatchedActorTerminatedException]] if the target actor is terminated.
|
||||
*
|
||||
* Adheres to the [[ActorAttributes.SupervisionStrategy]] attribute.
|
||||
*
|
||||
* '''Emits when''' the futures (in submission order) created by the ask pattern internally are completed
|
||||
*
|
||||
* '''Backpressures when''' the number of futures reaches the configured parallelism and the downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes and all futures have been completed and all elements have been emitted
|
||||
*
|
||||
* '''Fails when''' the passed in actor terminates, or a timeout is exceeded in any of the asks performed
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@implicitNotFound("Missing an implicit akka.util.Timeout for the ask() stage")
|
||||
def ask[S](parallelism: Int)(ref: ActorRef)(implicit timeout: Timeout, tag: ClassTag[S]): Repr[S] = {
|
||||
val askFlow = Flow[Out]
|
||||
.watch(ref)
|
||||
.mapAsync(parallelism) { el ⇒
|
||||
akka.pattern.ask(ref).?(el)(timeout).mapTo[S](tag)
|
||||
}
|
||||
.recover[S] {
|
||||
// the purpose of this recovery is to change the name of the stage in that exception
|
||||
// we do so in order to help users find which stage caused the failure -- "the ask stage"
|
||||
case ex: WatchedActorTerminatedException ⇒
|
||||
throw new WatchedActorTerminatedException("ask()", ex.ref)
|
||||
}
|
||||
.named("ask")
|
||||
|
||||
via(askFlow)
|
||||
}
|
||||
|
||||
/**
|
||||
* The stage fails with an [[akka.stream.WatchedActorTerminatedException]] if the target actor is terminated.
|
||||
*
|
||||
* '''Emits when''' upstream emits
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes
|
||||
*
|
||||
* '''Fails when''' the watched actor terminates
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
def watch(ref: ActorRef): Repr[Out] =
|
||||
via(Watch(ref))
|
||||
|
||||
/**
|
||||
* Only pass on those elements that satisfy the given predicate.
|
||||
*
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue