* mapAsync + ask should be the first choice * add missing Source.queue * prefer actorRefWithAck * move ActorPublisher and ActorSubscriber to the end with additional warning * fix wrong doc of SourceQueue offer * and add missing java api
This commit is contained in:
parent
bcf4de5b2c
commit
aa8c253d14
9 changed files with 457 additions and 232 deletions
|
|
@ -11,6 +11,7 @@ import akka.stream.*;
|
|||
import akka.stream.javadsl.*;
|
||||
import akka.testkit.JavaTestKit;
|
||||
import akka.testkit.TestProbe;
|
||||
import akka.util.Timeout;
|
||||
|
||||
import com.typesafe.config.Config;
|
||||
import com.typesafe.config.ConfigFactory;
|
||||
|
|
@ -20,6 +21,8 @@ import docs.stream.TwitterStreamQuickstartDocTest.Model.Tweet;
|
|||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import scala.Option;
|
||||
import scala.PartialFunction;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.Optional;
|
||||
|
|
@ -27,6 +30,7 @@ import java.util.Set;
|
|||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import static akka.pattern.PatternsCS.ask;
|
||||
import static docs.stream.TwitterStreamQuickstartDocTest.Model.AKKA;
|
||||
|
|
@ -39,6 +43,7 @@ public class IntegrationDocTest extends AbstractJavaTest {
|
|||
|
||||
static ActorSystem system;
|
||||
static Materializer mat;
|
||||
static ActorRef ref;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
|
|
@ -54,6 +59,7 @@ public class IntegrationDocTest extends AbstractJavaTest {
|
|||
|
||||
system = ActorSystem.create("ActorPublisherDocTest", config);
|
||||
mat = ActorMaterializer.create(system);
|
||||
ref = system.actorOf(Props.create(Translator.class));
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
|
|
@ -61,6 +67,7 @@ public class IntegrationDocTest extends AbstractJavaTest {
|
|||
JavaTestKit.shutdownActorSystem(system);
|
||||
system = null;
|
||||
mat = null;
|
||||
ref = null;
|
||||
}
|
||||
|
||||
|
||||
|
|
@ -281,6 +288,40 @@ public class IntegrationDocTest extends AbstractJavaTest {
|
|||
}
|
||||
}
|
||||
//#sometimes-slow-service
|
||||
|
||||
//#ask-actor
|
||||
static class Translator extends UntypedActor {
|
||||
@Override
|
||||
public void onReceive(Object message) {
|
||||
if (message instanceof String) {
|
||||
String word = (String) message;
|
||||
// ... process message
|
||||
String reply = word.toUpperCase();
|
||||
// reply to the ask
|
||||
getSender().tell(reply, getSelf());
|
||||
} else {
|
||||
unhandled(message);
|
||||
}
|
||||
}
|
||||
}
|
||||
//#ask-actor
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Test
|
||||
public void mapAsyncPlusAsk() throws Exception {
|
||||
//#mapAsync-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)
|
||||
// continue processing of the replies from the actor
|
||||
.map(elem -> elem.toLowerCase())
|
||||
.runWith(Sink.ignore(), mat);
|
||||
//#mapAsync-ask
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
|
|
|
|||
|
|
@ -7,27 +7,88 @@ Integration
|
|||
Integrating with Actors
|
||||
=======================
|
||||
|
||||
For piping the elements of a stream as messages to an ordinary actor you can use the
|
||||
``Sink.actorRef``. Messages can be sent to a stream via the :class:`ActorRef` that is
|
||||
For piping the elements of a stream as messages to an ordinary actor you can use
|
||||
``ask`` in a ``mapAsync`` or use ``Sink.actorRefWithAck``.
|
||||
|
||||
Messages can be sent to a stream with ``Source.queue`` or via the :class:`ActorRef` that is
|
||||
materialized by ``Source.actorRef``.
|
||||
|
||||
For more advanced use cases the :class:`ActorPublisher` and :class:`ActorSubscriber` traits are
|
||||
provided to support implementing Reactive Streams :class:`Publisher` and :class:`Subscriber` with
|
||||
an :class:`Actor`.
|
||||
mapAsync + ask
|
||||
^^^^^^^^^^^^^^
|
||||
|
||||
These can be consumed by other Reactive Stream libraries or used as an
|
||||
Akka Streams :class:`Source` or :class:`Sink`.
|
||||
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 ``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.
|
||||
|
||||
.. warning::
|
||||
.. includecode:: ../code/docs/stream/IntegrationDocTest.java#mapAsync-ask
|
||||
|
||||
:class:`AbstractActorPublisher` and :class:`AbstractActorSubscriber` cannot be used with remote actors,
|
||||
because if signals of the Reactive Streams protocol (e.g. ``request``) are lost the
|
||||
the stream may deadlock.
|
||||
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
|
||||
of the messages. There is a performance advantage of using parallelism > 1
|
||||
even though the actor will only process one message at a time because then there
|
||||
is already a message in the mailbox when the actor has completed previous
|
||||
message.
|
||||
|
||||
The actor must reply to the ``getSender()`` for each message from the stream. That
|
||||
reply will complete the ``CompletionStage`` of the ``ask`` and it will be the element that
|
||||
is emitted downstreams from ``mapAsync``.
|
||||
|
||||
.. includecode:: ../code/docs/stream/IntegrationDocTest.java#ask-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`` :class:`CompletionStage`.
|
||||
|
||||
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
|
||||
of the stream.
|
||||
|
||||
The same pattern can be used with :ref:`Actor routers <routing-java>`. Then you
|
||||
can use ``mapAsyncUnordered`` for better efficiency if you don't care about the
|
||||
order of the emitted downstream elements (the replies).
|
||||
|
||||
Sink.actorRefWithAck
|
||||
^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
The sink sends the elements of the stream to the given :class:`ActorRef` that sends back back-pressure signal.
|
||||
First element is always `onInitMessage`, then stream is waiting for the given acknowledgement message
|
||||
from the given actor which means that it is ready to process elements. It also requires the given acknowledgement
|
||||
message after each stream element to make back-pressure work.
|
||||
|
||||
If the target actor terminates the stream will be cancelled. When the stream is completed successfully the
|
||||
given ``onCompleteMessage`` will be sent to the destination actor. When the stream is completed with
|
||||
failure a ``akka.actor.Status.Failure`` message will be sent to the destination actor.
|
||||
|
||||
.. note::
|
||||
These Actors are designed to be implemented using Java 8 lambda expressions. In case you need to stay on a JVM
|
||||
prior to 8, Akka provides :class:`UntypedActorPublisher` and :class:`UntypedActorSubscriber` which can be used
|
||||
easily from any language level.
|
||||
|
||||
Using ``Sink.actorRef`` or ordinary ``tell`` from a ``map`` or ``foreach`` stage means that there is
|
||||
no back-pressure signal from the destination actor, i.e. if the actor is not consuming the messages
|
||||
fast enough the mailbox of the actor will grow, unless you use a bounded mailbox with zero
|
||||
`mailbox-push-timeout-time` or use a rate limiting stage in front. It's often better to
|
||||
use ``Sink.actorRefWithAck`` or ``ask`` in ``mapAsync``, though.
|
||||
|
||||
Source.queue
|
||||
^^^^^^^^^^^^
|
||||
|
||||
``Source.queue`` can be used for emitting elements to a stream from an actor (or from anything running outside
|
||||
the stream). The elements will be buffered until the stream can process them. You can ``offer`` elements to
|
||||
the queue and they will be emitted to the stream if there is demand from downstream, otherwise they will
|
||||
be buffered until request for demand is received.
|
||||
|
||||
Use overflow strategy ``akka.stream.OverflowStrategy.backpressure`` to avoid dropping of elements if the
|
||||
buffer is full.
|
||||
|
||||
``SourceQueue.offer`` returns ``CompletionStage<QueueOfferResult>`` which completes with
|
||||
``QueueOfferResult.enqueued`` if element was added to buffer or sent downstream. It completes with
|
||||
``QueueOfferResult.dropped`` if element was dropped. Can also complete with ``QueueOfferResult.Failure`` -
|
||||
when stream failed or ``QueueOfferResult.QueueClosed`` when downstream is completed.
|
||||
|
||||
When used from an actor you typically ``pipe`` the result of the ``CompletionStage`` back to the actor to
|
||||
continue processing.
|
||||
|
||||
Source.actorRef
|
||||
^^^^^^^^^^^^^^^
|
||||
|
|
@ -37,9 +98,10 @@ stream if there is demand from downstream, otherwise they will be buffered until
|
|||
demand is received.
|
||||
|
||||
Depending on the defined :class:`OverflowStrategy` it might drop elements if there is no space
|
||||
available in the buffer. The strategy ``OverflowStrategy.backpressure()`` is not supported
|
||||
for this Source type, you should consider using ``ActorPublisher`` if you want a backpressured
|
||||
actor interface.
|
||||
available in the buffer. The strategy ``OverflowStrategy.backpressure`` is not supported
|
||||
for this Source type, i.e. elements will be dropped if the buffer is filled by sending
|
||||
at a rate that is faster than the stream can consume. You should consider using ``Source.queue``
|
||||
if you want a backpressured actor interface.
|
||||
|
||||
The stream can be completed successfully by sending ``akka.actor.PoisonPill`` or
|
||||
``akka.actor.Status.Success`` to the actor reference.
|
||||
|
|
@ -50,96 +112,6 @@ actor reference.
|
|||
The actor will be stopped when the stream is completed, failed or cancelled from downstream,
|
||||
i.e. you can watch it to get notified when that happens.
|
||||
|
||||
Sink.actorRef
|
||||
^^^^^^^^^^^^^
|
||||
|
||||
The sink sends the elements of the stream to the given `ActorRef`. If the target actor terminates
|
||||
the stream will be cancelled. When the stream is completed successfully the given ``onCompleteMessage``
|
||||
will be sent to the destination actor. When the stream is completed with failure a ``akka.actor.Status.Failure``
|
||||
message will be sent to the destination actor.
|
||||
|
||||
.. warning::
|
||||
|
||||
There is no back-pressure signal from the destination actor, i.e. if the actor is not consuming
|
||||
the messages fast enough the mailbox of the actor will grow. For potentially slow consumer actors
|
||||
it is recommended to use a bounded mailbox with zero `mailbox-push-timeout-time` or use a rate
|
||||
limiting stage in front of this stage.
|
||||
|
||||
ActorPublisher
|
||||
^^^^^^^^^^^^^^
|
||||
|
||||
Extend :class:`akka.stream.actor.AbstractActorPublisher` to implement a
|
||||
stream publisher that keeps track of the subscription life cycle and requested elements.
|
||||
|
||||
Here is an example of such an actor. It dispatches incoming jobs to the attached subscriber:
|
||||
|
||||
.. includecode:: ../code/docs/stream/ActorPublisherDocTest.java#job-manager
|
||||
|
||||
You send elements to the stream by calling ``onNext``. You are allowed to send as many
|
||||
elements as have been requested by the stream subscriber. This amount can be inquired with
|
||||
``totalDemand``. It is only allowed to use ``onNext`` when ``isActive`` and ``totalDemand>0``,
|
||||
otherwise ``onNext`` will throw ``IllegalStateException``.
|
||||
|
||||
When the stream subscriber requests more elements the ``ActorPublisherMessage.Request`` message
|
||||
is delivered to this actor, and you can act on that event. The ``totalDemand``
|
||||
is updated automatically.
|
||||
|
||||
When the stream subscriber cancels the subscription the ``ActorPublisherMessage.Cancel`` message
|
||||
is delivered to this actor. After that subsequent calls to ``onNext`` will be ignored.
|
||||
|
||||
You can complete the stream by calling ``onComplete``. After that you are not allowed to
|
||||
call ``onNext``, ``onError`` and ``onComplete``.
|
||||
|
||||
You can terminate the stream with failure by calling ``onError``. After that you are not allowed to
|
||||
call ``onNext``, ``onError`` and ``onComplete``.
|
||||
|
||||
If you suspect that this ``AbstractActorPublisher`` may never get subscribed to, you can override the ``subscriptionTimeout``
|
||||
method to provide a timeout after which this Publisher should be considered canceled. The actor will be notified when
|
||||
the timeout triggers via an ``ActorPublisherMessage.SubscriptionTimeoutExceeded`` message and MUST then perform
|
||||
cleanup and stop itself.
|
||||
|
||||
If the actor is stopped the stream will be completed, unless it was not already terminated with
|
||||
failure, completed or canceled.
|
||||
|
||||
More detailed information can be found in the API documentation.
|
||||
|
||||
This is how it can be used as input :class:`Source` to a :class:`Flow`:
|
||||
|
||||
.. includecode:: ../code/docs/stream/ActorPublisherDocTest.java#actor-publisher-usage
|
||||
|
||||
You can only attach one subscriber to this publisher. Use a ``Broadcast``-element or
|
||||
attach a ``Sink.asPublisher(AsPublisher.WITH_FANOUT)`` to enable multiple subscribers.
|
||||
|
||||
ActorSubscriber
|
||||
^^^^^^^^^^^^^^^
|
||||
|
||||
Extend :class:`akka.stream.actor.AbstractActorSubscriber` to make your class a stream subscriber with
|
||||
full control of stream back pressure. It will receive
|
||||
``ActorSubscriberMessage.OnNext``, ``ActorSubscriberMessage.OnComplete`` and ``ActorSubscriberMessage.OnError``
|
||||
messages from the stream. It can also receive other, non-stream messages, in the same way as any actor.
|
||||
|
||||
Here is an example of such an actor. It dispatches incoming jobs to child worker actors:
|
||||
|
||||
.. includecode:: ../code/docs/stream/ActorSubscriberDocTest.java#worker-pool
|
||||
|
||||
Subclass must define the ``RequestStrategy`` to control stream back pressure.
|
||||
After each incoming message the ``AbstractActorSubscriber`` will automatically invoke
|
||||
the ``RequestStrategy.requestDemand`` and propagate the returned demand to the stream.
|
||||
|
||||
* The provided ``WatermarkRequestStrategy`` is a good strategy if the actor performs work itself.
|
||||
* The provided ``MaxInFlightRequestStrategy`` is useful if messages are queued internally or
|
||||
delegated to other actors.
|
||||
* You can also implement a custom ``RequestStrategy`` or call ``request`` manually together with
|
||||
``ZeroRequestStrategy`` or some other strategy. In that case
|
||||
you must also call ``request`` when the actor is started or when it is ready, otherwise
|
||||
it will not receive any elements.
|
||||
|
||||
More detailed information can be found in the API documentation.
|
||||
|
||||
This is how it can be used as output :class:`Sink` to a :class:`Flow`:
|
||||
|
||||
.. includecode:: ../code/docs/stream/ActorSubscriberDocTest.java#actor-subscriber-usage
|
||||
|
||||
Integrating with External Services
|
||||
==================================
|
||||
|
||||
|
|
@ -438,3 +410,101 @@ passing a factory function that will create the :class:`Processor` instances:
|
|||
.. includecode:: ../code/docs/stream/ReactiveStreamsDocTest.java#use-processor
|
||||
|
||||
Please note that a factory is necessary to achieve reusability of the resulting :class:`Flow`.
|
||||
|
||||
Implementing Reactive Streams Publisher or Subscriber
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
As described above any Akka Streams ``Source`` can be exposed as a Reactive Streams ``Publisher`` and
|
||||
any ``Sink`` can be exposed as a Reactive Streams ``Subscriber``. Therefore we recommend that you
|
||||
implement Reactive Streams integrations with built-in stages or :ref:`custom stages <stream-customize-java>`.
|
||||
|
||||
For historical reasons the :class:`ActorPublisher` and :class:`ActorSubscriber` traits are
|
||||
provided to support implementing Reactive Streams :class:`Publisher` and :class:`Subscriber` with
|
||||
an :class:`Actor`.
|
||||
|
||||
These can be consumed by other Reactive Stream libraries or used as an Akka Streams :class:`Source` or :class:`Sink`.
|
||||
|
||||
.. warning::
|
||||
|
||||
:class:`ActorPublisher` and :class:`ActorSubscriber` will probably be deprecated in future versions of Akka.
|
||||
|
||||
.. warning::
|
||||
|
||||
:class:`ActorPublisher` and :class:`ActorSubscriber` cannot be used with remote actors,
|
||||
because if signals of the Reactive Streams protocol (e.g. ``request``) are lost the
|
||||
the stream may deadlock.
|
||||
|
||||
ActorPublisher
|
||||
--------------
|
||||
|
||||
Extend :class:`akka.stream.actor.AbstractActorPublisher` to implement a
|
||||
stream publisher that keeps track of the subscription life cycle and requested elements.
|
||||
|
||||
Here is an example of such an actor. It dispatches incoming jobs to the attached subscriber:
|
||||
|
||||
.. includecode:: ../code/docs/stream/ActorPublisherDocTest.java#job-manager
|
||||
|
||||
You send elements to the stream by calling ``onNext``. You are allowed to send as many
|
||||
elements as have been requested by the stream subscriber. This amount can be inquired with
|
||||
``totalDemand``. It is only allowed to use ``onNext`` when ``isActive`` and ``totalDemand>0``,
|
||||
otherwise ``onNext`` will throw ``IllegalStateException``.
|
||||
|
||||
When the stream subscriber requests more elements the ``ActorPublisherMessage.Request`` message
|
||||
is delivered to this actor, and you can act on that event. The ``totalDemand``
|
||||
is updated automatically.
|
||||
|
||||
When the stream subscriber cancels the subscription the ``ActorPublisherMessage.Cancel`` message
|
||||
is delivered to this actor. After that subsequent calls to ``onNext`` will be ignored.
|
||||
|
||||
You can complete the stream by calling ``onComplete``. After that you are not allowed to
|
||||
call ``onNext``, ``onError`` and ``onComplete``.
|
||||
|
||||
You can terminate the stream with failure by calling ``onError``. After that you are not allowed to
|
||||
call ``onNext``, ``onError`` and ``onComplete``.
|
||||
|
||||
If you suspect that this ``AbstractActorPublisher`` may never get subscribed to, you can override the ``subscriptionTimeout``
|
||||
method to provide a timeout after which this Publisher should be considered canceled. The actor will be notified when
|
||||
the timeout triggers via an ``ActorPublisherMessage.SubscriptionTimeoutExceeded`` message and MUST then perform
|
||||
cleanup and stop itself.
|
||||
|
||||
If the actor is stopped the stream will be completed, unless it was not already terminated with
|
||||
failure, completed or canceled.
|
||||
|
||||
More detailed information can be found in the API documentation.
|
||||
|
||||
This is how it can be used as input :class:`Source` to a :class:`Flow`:
|
||||
|
||||
.. includecode:: ../code/docs/stream/ActorPublisherDocTest.java#actor-publisher-usage
|
||||
|
||||
You can only attach one subscriber to this publisher. Use a ``Broadcast``-element or
|
||||
attach a ``Sink.asPublisher(AsPublisher.WITH_FANOUT)`` to enable multiple subscribers.
|
||||
|
||||
ActorSubscriber
|
||||
---------------
|
||||
|
||||
Extend :class:`akka.stream.actor.AbstractActorSubscriber` to make your class a stream subscriber with
|
||||
full control of stream back pressure. It will receive
|
||||
``ActorSubscriberMessage.OnNext``, ``ActorSubscriberMessage.OnComplete`` and ``ActorSubscriberMessage.OnError``
|
||||
messages from the stream. It can also receive other, non-stream messages, in the same way as any actor.
|
||||
|
||||
Here is an example of such an actor. It dispatches incoming jobs to child worker actors:
|
||||
|
||||
.. includecode:: ../code/docs/stream/ActorSubscriberDocTest.java#worker-pool
|
||||
|
||||
Subclass must define the ``RequestStrategy`` to control stream back pressure.
|
||||
After each incoming message the ``AbstractActorSubscriber`` will automatically invoke
|
||||
the ``RequestStrategy.requestDemand`` and propagate the returned demand to the stream.
|
||||
|
||||
* The provided ``WatermarkRequestStrategy`` is a good strategy if the actor performs work itself.
|
||||
* The provided ``MaxInFlightRequestStrategy`` is useful if messages are queued internally or
|
||||
delegated to other actors.
|
||||
* You can also implement a custom ``RequestStrategy`` or call ``request`` manually together with
|
||||
``ZeroRequestStrategy`` or some other strategy. In that case
|
||||
you must also call ``request`` when the actor is started or when it is ready, otherwise
|
||||
it will not receive any elements.
|
||||
|
||||
More detailed information can be found in the API documentation.
|
||||
|
||||
This is how it can be used as output :class:`Sink` to a :class:`Flow`:
|
||||
|
||||
.. includecode:: ../code/docs/stream/ActorSubscriberDocTest.java#actor-subscriber-usage
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue