!str #19005 make groupBy et al return a SubFlow

A SubFlow (or SubSource) is not a Graph, it is an unfinished builder
that accepts transformations. This allows us to capture the substreams’
transformations before materializing the flow, which will be very
helpful in fully fusing all operators.

Another change is that groupBy now requires a maxSubstreams parameter in
order to bound its resource usage. In exchange the matching merge can be
unbounded. This trades silent deadlock for explicit stream failure.

This commit also changes all uses of Predef.identity to use `conforms`
and removes the HTTP impl.util.identityFunc.
This commit is contained in:
Roland Kuhn 2015-11-25 19:58:48 +01:00
parent 654fa41443
commit 1500d1f36d
56 changed files with 3484 additions and 720 deletions

View file

@ -322,6 +322,62 @@ Update procedure
*There is no simple update procedure. The affected stages must be ported to the new ``GraphStage`` DSL manually. Please
read the* ``GraphStage`` *documentation (TODO) for details.*
GroupBy, SplitWhen and SplitAfter now return SubFlow or SubSource
=================================================================
Previously the ``groupBy``, ``splitWhen``, and ``splitAfter`` combinators
returned a type that included a :class:`Source` within its elements.
Transforming these substreams was only possible by nesting the respective
combinators inside a ``map`` of the outer stream. This has been made more
convenient and also safer by dropping down into transforming the substreams
instead: the return type is now a :class:`SubSource` (for sources) or a
:class:`SubFlow` (for flows) that does not implement the :class:`Graph`
interface and therefore only represents an unfinished intermediate builder
step.
Update Procedure
----------------
The transformations that were done on the substreams need to be lifted up one
level. This only works for cases where the processing topology is homogenous
for all substreams.
Example
^^^^^^^
::
Flow.<Integer> create()
// This no longer works!
.groupBy(i -> i % 2)
// This no longer works!
.map(pair -> pair.second().map(i -> i + 3))
// This no longer works!
.flatten(FlattenStrategy.concat())
This is implemented now as
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/MigrationsJava.java#group-flatten
Example 2
^^^^^^^^^
::
Flow.<String> create()
// This no longer works!
.groupBy(i -> i)
// This no longer works!
.map(pair ->
pair.second().runFold(new Pair<>(pair.first(), 0),
(pair, word) -> new Pair<>(word, pair.second() + 1)))
// This no longer works!
.mapAsyncUnordered(4, i -> i)
This is implemented now as
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/MigrationsJava.java#group-fold
Semantic change in ``isHoldingUpstream`` in the DetachedStage DSL
=================================================================
@ -437,7 +493,7 @@ should be replaced by:
.. includecode:: code/docs/MigrationsJava.java#query-param
SynchronousFileSource and SynchronousFileSink
============================================
=============================================
Both have been replaced by ``Source.file(…)`` and ``Sink.file(…)`` due to discoverability issues
paired with names which leaked internal implementation details.
@ -553,4 +609,4 @@ Example
should be replaced by
.. includecode:: code/docs/MigrationsJava.java#output-input-stream-source-sink
.. includecode:: code/docs/MigrationsJava.java#output-input-stream-source-sink

View file

@ -107,23 +107,28 @@ Implementing reduce-by-key
elements.
The "hello world" of reduce-by-key style operations is *wordcount* which we demonstrate below. Given a stream of words
we first create a new stream ``wordStreams`` that groups the words according to the ``i -> i`` function, i.e. now
we first create a new stream that groups the words according to the ``i -> i`` function, i.e. now
we have a stream of streams, where every substream will serve identical words.
To count the words, we need to process the stream of streams (the actual groups containing identical words). By mapping
over the groups and using ``fold`` (remember that ``fold`` automatically materializes and runs the stream it is used
on) we get a stream with elements of ``Future[String,Int]``. Now all we need is to flatten this stream, which
can be achieved by calling ``mapAsync`` with ``i -> i`` identity function.
To count the words, we need to process the stream of streams (the actual groups
containing identical words). ``groupBy`` returns a :class:`SubSource`, which
means that we transform the resulting substreams directly. In this case we use
the ``fold`` combinator to aggregate the word itself and the number of its
occurrences within a :class:`Pair<String, Integer>`. Each substream will then
emit one final value—precisely such a pair—when the overall input completes. As
a last step we merge back these values from the substreams into one single
output stream.
There is one tricky issue to be noted here. The careful reader probably noticed that we put a ``buffer`` between the
``mapAsync()`` operation that flattens the stream of futures and the actual stream of futures. The reason for this is
that the substreams produced by ``groupBy()`` can only complete when the original upstream source completes. This means
that ``mapAsync()`` cannot pull for more substreams because it still waits on folding futures to finish, but these
futures never finish if the additional group streams are not consumed. This typical deadlock situation is resolved by
this buffer which either able to contain all the group streams (which ensures that they are already running and folding)
or fails with an explicit failure instead of a silent deadlock.
One noteworthy detail pertains to the ``MAXIMUM_DISTINCT_WORDS`` parameter:
this defines the breadth of the merge operation. Akka Streams is focused on
bounded resource consumption and the number of concurrently open inputs to the
merge operator describes the amount of resources needed by the merge itself.
Therefore only a finite number of substreams can be active at any given time.
If the ``groupBy`` operator encounters more keys than this number then the
stream cannot continue without violating its resource bound, in this case
``groupBy`` will terminate with a failure.
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/cookbook/RecipeReduceByKey.java#word-count
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/cookbook/RecipeReduceByKeyTest.java#word-count
By extracting the parts specific to *wordcount* into
@ -133,13 +138,14 @@ By extracting the parts specific to *wordcount* into
we get a generalized version below:
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/cookbook/RecipeReduceByKey.java#reduce-by-key-general
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/cookbook/RecipeReduceByKeyTest.java#reduce-by-key-general
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/cookbook/RecipeReduceByKey.java#reduce-by-key-general2
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/cookbook/RecipeReduceByKeyTest.java#reduce-by-key-general2
.. note::
Please note that the reduce-by-key version we discussed above is sequential, in other words it is **NOT** a
parallelization pattern like mapReduce and similar frameworks.
Please note that the reduce-by-key version we discussed above is sequential
in reading the overall input stream, in other words it is **NOT** a
parallelization pattern like MapReduce and similar frameworks.
Sorting elements to multiple groups with groupBy
------------------------------------------------
@ -150,12 +156,12 @@ Sometimes we want to map elements into multiple groups simultaneously.
To achieve the desired result, we attack the problem in two steps:
* first, using a function ``topicMapper`` that gives a list of topics (groups) a message belongs to, we transform our
stream of ``Message`` to a stream of ``(Message, Topic)`` where for each topic the message belongs to a separate pair
stream of ``Message`` to a stream of :class:`Pair<Message, Topic>`` where for each topic the message belongs to a separate pair
will be emitted. This is achieved by using ``mapConcat``
* Then we take this new stream of message topic pairs (containing a separate pair for each topic a given message
belongs to) and feed it into groupBy, using the topic as the group key.
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/cookbook/RecipeMultiGroupBy.java#multi-groupby
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/cookbook/RecipeMultiGroupByTest.java#multi-groupby
Working with Graphs
===================

View file

@ -231,8 +231,8 @@ resulting values. Some examples of using these combiners are illustrated in the
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/FlowDocTest.java#flow-mat-combine
.. note::
In Graphs it is possible to access the materialized value from inside the stream processing graph. For details see
:ref:`graph-matvalue-java`
In Graphs it is possible to access the materialized value from inside the stream processing graph. For details see :ref:`graph-matvalue-java`.
Stream ordering
===============

View file

@ -4,60 +4,89 @@
Testing streams
###############
Verifying behaviour of Akka Stream sources, flows and sinks can be done using various code patterns and libraries. Here we will discuss testing these elements using:
Verifying behaviour of Akka Stream sources, flows and sinks can be done using
various code patterns and libraries. Here we will discuss testing these
elements using:
- simple sources, sinks and flows;
- sources and sinks in combination with :class:`TestProbe` from the :mod:`akka-testkit` module;
- sources and sinks specifically crafted for writing tests from the :mod:`akka-stream-testkit` module.
It is important to keep your data processing pipeline as separate sources, flows and sinks. This makes them easily testable by wiring them up to other sources or sinks, or some test harnesses that :mod:`akka-testkit` or :mod:`akka-stream-testkit` provide.
It is important to keep your data processing pipeline as separate sources,
flows and sinks. This makes them easily testable by wiring them up to other
sources or sinks, or some test harnesses that :mod:`akka-testkit` or
:mod:`akka-stream-testkit` provide.
Built in sources, sinks and combinators
=======================================
Testing a custom sink can be as simple as attaching a source that emits elements from a predefined collection, running a constructed test flow and asserting on the results that sink produced. Here is an example of a test for a sink:
Testing a custom sink can be as simple as attaching a source that emits
elements from a predefined collection, running a constructed test flow and
asserting on the results that sink produced. Here is an example of a test for a
sink:
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/StreamTestKitDocTest.java#strict-collection
The same strategy can be applied for sources as well. In the next example we have a source that produces an infinite stream of elements. Such source can be tested by asserting that first arbitrary number of elements hold some condition. Here the :code:`grouped` combinator and :code:`Sink.head` are very useful.
The same strategy can be applied for sources as well. In the next example we
have a source that produces an infinite stream of elements. Such source can be
tested by asserting that first arbitrary number of elements hold some
condition. Here the ``grouped`` combinator and ``Sink.head`` are very useful.
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/StreamTestKitDocTest.java#grouped-infinite
When testing a flow we need to attach a source and a sink. As both stream ends are under our control, we can choose sources that tests various edge cases of the flow and sinks that ease assertions.
When testing a flow we need to attach a source and a sink. As both stream ends
are under our control, we can choose sources that tests various edge cases of
the flow and sinks that ease assertions.
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/StreamTestKitDocTest.java#folded-stream
TestKit
=======
Akka Stream offers integration with Actors out of the box. This support can be used for writing stream tests that use familiar :class:`TestProbe` from the :mod:`akka-testkit` API.
Akka Stream offers integration with Actors out of the box. This support can be
used for writing stream tests that use familiar :class:`TestProbe` from the
:mod:`akka-testkit` API.
One of the more straightforward tests would be to materialize stream to a :class:`Future` and then use :code:`pipe` pattern to pipe the result of that future to the probe.
One of the more straightforward tests would be to materialize stream to a
:class:`Future` and then use ``pipe`` pattern to pipe the result of that future
to the probe.
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/StreamTestKitDocTest.java#pipeto-testprobe
Instead of materializing to a future, we can use a :class:`Sink.actorRef` that sends all incoming elements to the given :class:`ActorRef`. Now we can use assertion methods on :class:`TestProbe` and expect elements one by one as they arrive. We can also assert stream completion by expecting for :code:`onCompleteMessage` which was given to :class:`Sink.actorRef`.
Instead of materializing to a future, we can use a :class:`Sink.actorRef` that
sends all incoming elements to the given :class:`ActorRef`. Now we can use
assertion methods on :class:`TestProbe` and expect elements one by one as they
arrive. We can also assert stream completion by expecting for
``onCompleteMessage`` which was given to :class:`Sink.actorRef`.
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/StreamTestKitDocTest.java#sink-actorref
Similarly to :class:`Sink.actorRef` that provides control over received elements, we can use :class:`Source.actorRef` and have full control over elements to be sent.
Similarly to :class:`Sink.actorRef` that provides control over received
elements, we can use :class:`Source.actorRef` and have full control over
elements to be sent.
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/StreamTestKitDocTest.java#source-actorref
Streams TestKit
===============
You may have noticed various code patterns that emerge when testing stream pipelines. Akka Stream has a separate :mod:`akka-stream-testkit` module that provides tools specifically for writing stream tests. This module comes with two main components that are :class:`TestSource` and :class:`TestSink` which provide sources and sinks that materialize to probes that allow fluent API.
You may have noticed various code patterns that emerge when testing stream
pipelines. Akka Stream has a separate :mod:`akka-stream-testkit` module that
provides tools specifically for writing stream tests. This module comes with
two main components that are :class:`TestSource` and :class:`TestSink` which
provide sources and sinks that materialize to probes that allow fluent API.
.. note::
Be sure to add the module :mod:`akka-stream-testkit` to your dependencies.
A sink returned by :code:`TestSink.probe` allows manual control over demand and assertions over elements coming downstream.
A sink returned by ``TestSink.probe`` allows manual control over demand and
assertions over elements coming downstream.
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/StreamTestKitDocTest.java#test-sink-probe
A source returned by :code:`TestSource.probe` can be used for asserting demand or controlling when stream is completed or ended with an error.
A source returned by ``TestSource.probe`` can be used for asserting demand or
controlling when stream is completed or ended with an error.
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/StreamTestKitDocTest.java#test-source-probe
@ -68,3 +97,4 @@ You can also inject exceptions and test sink behaviour on error conditions.
Test source and sink can be used together in combination when testing flows.
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/StreamTestKitDocTest.java#test-source-and-sink

View file

@ -116,6 +116,21 @@ class MigrationsScala extends AkkaSpec {
Flow[Source[Int, Any]].flatMapConcat(identity)
//#flatMapConcat
//#group-flatten
Flow[Int]
.groupBy(2, _ % 2) // the first parameter sets max number of substreams
.map(_ + 3)
.concatSubstreams
//#group-flatten
val MaxDistinctWords = 1000
//#group-fold
Flow[String]
.groupBy(MaxDistinctWords, identity)
.fold(("", 0))((pair, word) => (word, pair._2 + 1))
.mergeSubstreams
//#group-fold
//#port-async
class MapAsyncOne[In, Out](f: In Future[Out])(implicit ec: ExecutionContext)
extends GraphStage[FlowShape[In, Out]] {

View file

@ -30,17 +30,22 @@ class RecipeMultiGroupBy extends RecipeSpec {
topicsForMessage.map(msg -> _)
}
val multiGroups: Source[(Topic, Source[String, Unit]), Unit] = messageAndTopic
.groupBy(_._2).map {
case (topic, topicStream) =>
// chopping of the topic from the (Message, Topic) pairs
(topic, topicStream.map(_._1))
val multiGroups = messageAndTopic
.groupBy(2, _._2).map {
case (msg, topic) =>
// do what needs to be done
//#multi-groupby
(msg, topic)
//#multi-groupby
}
//#multi-groupby
val result = multiGroups.map {
case (topic, topicMessages) => topicMessages.grouped(10).map(topic.name + _.mkString("[", ", ", "]")).runWith(Sink.head)
}.mapAsync(4)(identity).grouped(10).runWith(Sink.head)
val result = multiGroups
.grouped(10)
.mergeSubstreams
.map(g => g.head._2.name + g.map(_._1).mkString("[", ", ", "]"))
.grouped(10)
.runWith(Sink.head)
Await.result(result, 3.seconds).toSet should be(Set(
"1[1: a, 1: b, all: c, all: d, 1: e]",

View file

@ -1,10 +1,10 @@
package docs.stream.cookbook
import akka.stream.OverflowStrategy
import akka.stream.{ Graph, FlowShape, Inlet, Outlet, Attributes, OverflowStrategy }
import akka.stream.scaladsl._
import scala.concurrent.{ Await, Future }
import scala.concurrent.duration._
import akka.stream.stage.{ GraphStage, GraphStageLogic }
class RecipeReduceByKey extends RecipeSpec {
@ -17,22 +17,15 @@ class RecipeReduceByKey extends RecipeSpec {
def words = Source(List("hello", "world", "and", "hello", "universe", "akka") ++ List.fill(1000)("rocks!"))
//#word-count
// split the words into separate streams first
val wordStreams: Source[(String, Source[String, Unit]), Unit] = words.groupBy(identity)
// add counting logic to the streams
val countedWords: Source[Future[(String, Int)], Unit] = wordStreams.map {
case (word, wordStream) =>
wordStream.runFold((word, 0)) {
case ((w, count), _) => (w, count + 1)
}
}
// get a stream of word counts
val counts: Source[(String, Int), Unit] =
countedWords
.buffer(MaximumDistinctWords, OverflowStrategy.fail)
.mapAsync(4)(identity)
val counts: Source[(String, Int), Unit] = words
// split the words into separate streams first
.groupBy(MaximumDistinctWords, identity)
// add counting logic to the streams
.fold(("", 0)) {
case ((_, count), word) => (word, count + 1)
}
// get a stream of word counts
.mergeSubstreams
//#word-count
Await.result(counts.grouped(10).runWith(Sink.head), 3.seconds).toSet should be(Set(
@ -54,15 +47,17 @@ class RecipeReduceByKey extends RecipeSpec {
groupKey: (In) => K,
foldZero: (K) => Out)(fold: (Out, In) => Out): Flow[In, (K, Out), Unit] = {
val groupStreams = Flow[In].groupBy(groupKey)
val reducedValues = groupStreams.map {
case (key, groupStream) =>
groupStream.runFold((key, foldZero(key))) {
case ((key, aggregated), elem) => (key, fold(aggregated, elem))
}
}
reducedValues.buffer(maximumGroupSize, OverflowStrategy.fail).mapAsync(4)(identity)
Flow[In]
.groupBy(maximumGroupSize, groupKey)
.fold(Option.empty[(K, Out)]) {
case (None, elem) =>
val key = groupKey(elem)
Some((key, fold(foldZero(key), elem)))
case (Some((key, out)), elem) =>
Some((key, fold(out, elem)))
}
.map(_.get)
.mergeSubstreams
}
val wordCounts = words.via(reduceByKey(

View file

@ -292,6 +292,64 @@ Update procedure
*There is no simple update procedure. The affected stages must be ported to the new ``GraphStage`` DSL manually. Please
read the* ``GraphStage`` *documentation (TODO) for details.*
GroupBy, SplitWhen and SplitAfter now return SubFlow
====================================================
Previously the ``groupBy``, ``splitWhen``, and ``splitAfter`` combinators
returned a type that included a :class:`Source` within its elements.
Transforming these substreams was only possible by nesting the respective
combinators inside a ``map`` of the outer stream. This has been made more
convenient and also safer by dropping down into transforming the substreams
instead: the return type is now a :class:`SubFlow` that does not implement the
:class:`Graph` interface and therefore only represents an unfinished
intermediate builder step. The substream mode can be ended by closing the
substreams (i.e. attaching a :class:`Sink`) or merging them back together.
Update Procedure
----------------
The transformations that were done on the substreams need to be lifted up one
level. This only works for cases where the processing topology is homogenous
for all substreams.
Example
^^^^^^^
::
Flow[Int]
// This no longer works!
.groupBy(_ % 2)
// This no longer works!
.map {
case (key, source) => source.map(_ + 3)
}
// This no longer works!
.flatten(FlattenStrategy.concat)
This is implemented now as
.. includecode:: code/docs/MigrationsScala.scala#group-flatten
Example 2
^^^^^^^^^
::
Flow[String]
// This no longer works!
.groupBy(identity)
// This no longer works!
.map {
case (key, source) => source.runFold((key, 0))((pair, word) => (key, pair._2 + 1))
}
// This no longer works!
.mapAsyncUnordered(4, identity)
This is implemented now as
.. includecode:: code/docs/MigrationsScala.scala#group-fold
Variance of Inlet and Outlet
============================
@ -468,7 +526,7 @@ should be replaced by:
.. includecode:: code/docs/MigrationsScala.scala#query-param
SynchronousFileSource and SynchronousFileSink
============================================
=============================================
Both have been replaced by ``Source.file(…)`` and ``Sink.file(…)`` due to discoverability issues
paired with names which leaked internal implementation details.
@ -559,4 +617,4 @@ Example
should be replaced by
.. includecode:: code/docs/MigrationsScala.scala#output-input-stream-source-sink
.. includecode:: code/docs/MigrationsScala.scala#output-input-stream-source-sink

View file

@ -105,21 +105,26 @@ Implementing reduce-by-key
elements.
The "hello world" of reduce-by-key style operations is *wordcount* which we demonstrate below. Given a stream of words
we first create a new stream ``wordStreams`` that groups the words according to the ``identity`` function, i.e. now
we first create a new stream that groups the words according to the ``identity`` function, i.e. now
we have a stream of streams, where every substream will serve identical words.
To count the words, we need to process the stream of streams (the actual groups containing identical words). By mapping
over the groups and using ``fold`` (remember that ``fold`` automatically materializes and runs the stream it is used
on) we get a stream with elements of ``Future[String,Int]``. Now all we need is to flatten this stream, which
can be achieved by calling ``mapAsync`` with ``identity`` function.
To count the words, we need to process the stream of streams (the actual groups
containing identical words). ``groupBy`` returns a :class:`SubFlow`, which
means that we transform the resulting substreams directly. In this case we use
the ``fold`` combinator to aggregate the word itself and the number of its
occurrences within a tuple :class:`(String, Integer)`. Each substream will then
emit one final value—precisely such a pair—when the overall input completes. As
a last step we merge back these values from the substreams into one single
output stream.
There is one tricky issue to be noted here. The careful reader probably noticed that we put a ``buffer`` between the
``mapAsync()`` operation that flattens the stream of futures and the actual stream of futures. The reason for this is
that the substreams produced by ``groupBy()`` can only complete when the original upstream source completes. This means
that ``mapAsync()`` cannot pull for more substreams because it still waits on folding futures to finish, but these
futures never finish if the additional group streams are not consumed. This typical deadlock situation is resolved by
this buffer which either able to contain all the group streams (which ensures that they are already running and folding)
or fails with an explicit failure instead of a silent deadlock.
One noteworthy detail pertains to the ``MaximumDistinctWords`` parameter: this
defines the breadth of the groupBy and merge operations. Akka Streams is
focused on bounded resource consumption and the number of concurrently open
inputs to the merge operator describes the amount of resources needed by the
merge itself. Therefore only a finite number of substreams can be active at
any given time. If the ``groupBy`` operator encounters more keys than this
number then the stream cannot continue without violating its resource bound, in
this case ``groupBy`` will terminate with a failure.
.. includecode:: code/docs/stream/cookbook/RecipeReduceByKey.scala#word-count
@ -134,8 +139,9 @@ we get a generalized version below:
.. includecode:: code/docs/stream/cookbook/RecipeReduceByKey.scala#reduce-by-key-general
.. note::
Please note that the reduce-by-key version we discussed above is sequential, in other words it is **NOT** a
parallelization pattern like mapReduce and similar frameworks.
Please note that the reduce-by-key version we discussed above is sequential
in reading the overall input stream, in other words it is **NOT** a
parallelization pattern like MapReduce and similar frameworks.
Sorting elements to multiple groups with groupBy
------------------------------------------------

View file

@ -235,8 +235,8 @@ resulting values. Some examples of using these combiners are illustrated in the
.. includecode:: code/docs/stream/FlowDocSpec.scala#flow-mat-combine
.. note::
In Graphs it is possible to access the materialized value from inside the stream processing graph. For details see
:ref:`graph-matvalue-scala`
In Graphs it is possible to access the materialized value from inside the stream processing graph. For details see :ref:`graph-matvalue-scala`.
Stream ordering
===============

View file

@ -4,60 +4,89 @@
Testing streams
###############
Verifying behaviour of Akka Stream sources, flows and sinks can be done using various code patterns and libraries. Here we will discuss testing these elements using:
Verifying behaviour of Akka Stream sources, flows and sinks can be done using
various code patterns and libraries. Here we will discuss testing these
elements using:
- simple sources, sinks and flows;
- sources and sinks in combination with :class:`TestProbe` from the :mod:`akka-testkit` module;
- sources and sinks specifically crafted for writing tests from the :mod:`akka-stream-testkit` module.
It is important to keep your data processing pipeline as separate sources, flows and sinks. This makes them easily testable by wiring them up to other sources or sinks, or some test harnesses that :mod:`akka-testkit` or :mod:`akka-stream-testkit` provide.
It is important to keep your data processing pipeline as separate sources,
flows and sinks. This makes them easily testable by wiring them up to other
sources or sinks, or some test harnesses that :mod:`akka-testkit` or
:mod:`akka-stream-testkit` provide.
Built in sources, sinks and combinators
=======================================
Testing a custom sink can be as simple as attaching a source that emits elements from a predefined collection, running a constructed test flow and asserting on the results that sink produced. Here is an example of a test for a sink:
Testing a custom sink can be as simple as attaching a source that emits
elements from a predefined collection, running a constructed test flow and
asserting on the results that sink produced. Here is an example of a test for a
sink:
.. includecode:: code/docs/stream/StreamTestKitDocSpec.scala#strict-collection
The same strategy can be applied for sources as well. In the next example we have a source that produces an infinite stream of elements. Such source can be tested by asserting that first arbitrary number of elements hold some condition. Here the :code:`grouped` combinator and :code:`Sink.head` are very useful.
The same strategy can be applied for sources as well. In the next example we
have a source that produces an infinite stream of elements. Such source can be
tested by asserting that first arbitrary number of elements hold some
condition. Here the ``grouped`` combinator and ``Sink.head`` are very useful.
.. includecode:: code/docs/stream/StreamTestKitDocSpec.scala#grouped-infinite
When testing a flow we need to attach a source and a sink. As both stream ends are under our control, we can choose sources that tests various edge cases of the flow and sinks that ease assertions.
When testing a flow we need to attach a source and a sink. As both stream ends
are under our control, we can choose sources that tests various edge cases of
the flow and sinks that ease assertions.
.. includecode:: code/docs/stream/StreamTestKitDocSpec.scala#folded-stream
TestKit
=======
Akka Stream offers integration with Actors out of the box. This support can be used for writing stream tests that use familiar :class:`TestProbe` from the :mod:`akka-testkit` API.
Akka Stream offers integration with Actors out of the box. This support can be
used for writing stream tests that use familiar :class:`TestProbe` from the
:mod:`akka-testkit` API.
One of the more straightforward tests would be to materialize stream to a :class:`Future` and then use :code:`pipe` pattern to pipe the result of that future to the probe.
One of the more straightforward tests would be to materialize stream to a
:class:`Future` and then use ``pipe`` pattern to pipe the result of that future
to the probe.
.. includecode:: code/docs/stream/StreamTestKitDocSpec.scala#pipeto-testprobe
Instead of materializing to a future, we can use a :class:`Sink.actorRef` that sends all incoming elements to the given :class:`ActorRef`. Now we can use assertion methods on :class:`TestProbe` and expect elements one by one as they arrive. We can also assert stream completion by expecting for :code:`onCompleteMessage` which was given to :class:`Sink.actorRef`.
Instead of materializing to a future, we can use a :class:`Sink.actorRef` that
sends all incoming elements to the given :class:`ActorRef`. Now we can use
assertion methods on :class:`TestProbe` and expect elements one by one as they
arrive. We can also assert stream completion by expecting for
``onCompleteMessage`` which was given to :class:`Sink.actorRef`.
.. includecode:: code/docs/stream/StreamTestKitDocSpec.scala#sink-actorref
Similarly to :class:`Sink.actorRef` that provides control over received elements, we can use :class:`Source.actorRef` and have full control over elements to be sent.
Similarly to :class:`Sink.actorRef` that provides control over received
elements, we can use :class:`Source.actorRef` and have full control over
elements to be sent.
.. includecode:: code/docs/stream/StreamTestKitDocSpec.scala#source-actorref
Streams TestKit
===============
You may have noticed various code patterns that emerge when testing stream pipelines. Akka Stream has a separate :mod:`akka-stream-testkit` module that provides tools specifically for writing stream tests. This module comes with two main components that are :class:`TestSource` and :class:`TestSink` which provide sources and sinks that materialize to probes that allow fluent API.
You may have noticed various code patterns that emerge when testing stream
pipelines. Akka Stream has a separate :mod:`akka-stream-testkit` module that
provides tools specifically for writing stream tests. This module comes with
two main components that are :class:`TestSource` and :class:`TestSink` which
provide sources and sinks that materialize to probes that allow fluent API.
.. note::
Be sure to add the module :mod:`akka-stream-testkit` to your dependencies.
A sink returned by :code:`TestSink.probe` allows manual control over demand and assertions over elements coming downstream.
A sink returned by ``TestSink.probe`` allows manual control over demand and
assertions over elements coming downstream.
.. includecode:: code/docs/stream/StreamTestKitDocSpec.scala#test-sink-probe
A source returned by :code:`TestSource.probe` can be used for asserting demand or controlling when stream is completed or ended with an error.
A source returned by ``TestSource.probe`` can be used for asserting demand or
controlling when stream is completed or ended with an error.
.. includecode:: code/docs/stream/StreamTestKitDocSpec.scala#test-source-probe

View file

@ -66,15 +66,15 @@ private[http] object OutgoingConnectionBlueprint {
import ParserOutput._
val responsePrep = Flow[List[ResponseOutput]]
.mapConcat(identityFunc)
.mapConcat(conforms)
.splitWhen(x x.isInstanceOf[MessageStart] || x == MessageEnd)
.via(headAndTailFlow)
.prefixAndTail(1)
.collect {
case (ResponseStart(statusCode, protocol, headers, createEntity, _), entityParts)
case (Seq(ResponseStart(statusCode, protocol, headers, createEntity, _)), entityParts)
val entity = createEntity(entityParts) withSizeLimit parserSettings.maxContentLength
HttpResponse(statusCode, headers, entity, protocol)
case (MessageStartError(_, info), _) throw IllegalResponseException(info)
}
case (Seq(MessageStartError(_, info)), _) throw IllegalResponseException(info)
}.concatSubstreams
val core = BidiFlow.fromGraph(GraphDSL.create() { implicit b
import GraphDSL.Implicits._

View file

@ -41,15 +41,15 @@ private object PoolSlot {
Stream Setup
============
Request- +-----------+ +-------------+ +-------------+ +------------+
Request- +-----------+ +-------------+ +-------------+ +------------+
Context |Slot-| List[ | flatten | Processor- | doubler | |SlotEvent-| Response-
+--------->|Processor+------------->| (MapConcat) +------------->| (MapConcat) +---->|Split+------------->
|| Processor- | | Out | | || Context
+-----------+ Out] +-------------+ +-------------+ +-----+------+
|RawSlotEvent
|| Processor- | | Out | | || Context
+-----------+ Out] +-------------+ +-------------+ +-----+------+
|RawSlotEvent
|(toConductor
| via slotEventMerge)
v
v
*/
def apply(slotIx: Int, connectionFlow: Flow[HttpRequest, HttpResponse, Any],
remoteAddress: InetSocketAddress, // TODO: remove after #16168 is cleared
@ -65,7 +65,7 @@ private object PoolSlot {
val actor = system.actorOf(Props(new SlotProcessor(slotIx, connectionFlow, settings)).withDeploy(Deploy.local),
name)
ActorProcessor[RequestContext, List[ProcessorOut]](actor)
}.mapConcat(identity)
}.mapConcat(conforms)
}
val split = b.add(Broadcast[ProcessorOut](2))
@ -196,7 +196,7 @@ private object PoolSlot {
} else {
inflightRequests.map { rc
if (rc.retriesLeft == 0) {
val reason = error.fold[Throwable](new RuntimeException("Unexpected disconnect"))(identityFunc)
val reason = error.fold[Throwable](new RuntimeException("Unexpected disconnect"))(conforms)
connInport ! ActorPublisherMessage.Cancel
ResponseDelivery(ResponseContext(rc, Failure(reason)))
} else SlotEvent.RetryRequest(rc.copy(retriesLeft = rc.retriesLeft - 1))

View file

@ -87,7 +87,9 @@ private[http] object HttpServerBluePrint {
BidiFlow.fromFlows(Flow[HttpResponse],
Flow[RequestOutput]
.splitWhen(x x.isInstanceOf[MessageStart] || x == MessageEnd)
.via(headAndTailFlow)
.prefixAndTail(1)
.map(p p._1.head -> p._2)
.concatSubstreams
.via(requestStartOrRunIgnore(settings)))
def requestStartOrRunIgnore(settings: ServerSettings)(implicit mat: Materializer): Flow[(ParserOutput.RequestOutput, Source[ParserOutput.RequestOutput, Unit]), HttpRequest, Unit] =

View file

@ -21,7 +21,7 @@ private[http] object MessageToFrameRenderer {
// FIXME: fragment?
Source.single(FrameEvent.fullFrame(opcode, None, data, fin = true))
def streamedFrames(opcode: Opcode, data: Source[ByteString, _]): Source[FrameStart, _] =
def streamedFrames[M](opcode: Opcode, data: Source[ByteString, M]): Source[FrameStart, Unit] =
Source.single(FrameEvent.empty(opcode, fin = false)) ++
data.map(FrameEvent.fullFrame(Opcode.Continuation, None, _, fin = false)) ++
Source.single(FrameEvent.emptyLastContinuationFrame)

View file

@ -85,36 +85,39 @@ private[http] object Websocket {
}
/** Collects user-level API messages from MessageDataParts */
val collectMessage: Flow[Source[MessageDataPart, Unit], Message, Unit] =
Flow[Source[MessageDataPart, Unit]]
.via(headAndTailFlow)
val collectMessage: Flow[MessageDataPart, Message, Unit] =
Flow[MessageDataPart]
.prefixAndTail(1)
.map {
case (TextMessagePart(text, true), remaining)
TextMessage.Strict(text)
case (first @ TextMessagePart(text, false), remaining)
TextMessage(
(Source.single(first) ++ remaining)
.collect {
case t: TextMessagePart if t.data.nonEmpty t.data
})
case (BinaryMessagePart(data, true), remaining)
BinaryMessage.Strict(data)
case (first @ BinaryMessagePart(data, false), remaining)
BinaryMessage(
(Source.single(first) ++ remaining)
.collect {
case t: BinaryMessagePart if t.data.nonEmpty t.data
})
case (seq, remaining) seq.head match {
case TextMessagePart(text, true)
TextMessage.Strict(text)
case first @ TextMessagePart(text, false)
TextMessage(
(Source.single(first) ++ remaining)
.collect {
case t: TextMessagePart if t.data.nonEmpty t.data
})
case BinaryMessagePart(data, true)
BinaryMessage.Strict(data)
case first @ BinaryMessagePart(data, false)
BinaryMessage(
(Source.single(first) ++ remaining)
.collect {
case t: BinaryMessagePart if t.data.nonEmpty t.data
})
}
}
def prepareMessages: Flow[MessagePart, Message, Unit] =
Flow[MessagePart]
.transform(() new PrepareForUserHandler)
.splitWhen(_.isMessageEnd) // FIXME using splitAfter from #16885 would simplify protocol a lot
.map(_.collect {
.collect {
case m: MessageDataPart m
})
}
.via(collectMessage)
.concatSubstreams
.named("ws-prepare-messages")
def renderMessages: Flow[Message, FrameStart, Unit] =

View file

@ -47,14 +47,6 @@ package object util {
private[http] implicit def enhanceByteStringsMat[Mat](byteStrings: Source[ByteString, Mat]): EnhancedByteStringSource[Mat] =
new EnhancedByteStringSource(byteStrings)
private[http] def headAndTailFlow[T]: Flow[Source[T, Any], (T, Source[T, Unit]), Unit] =
Flow[Source[T, Any]]
.flatMapConcat {
_.prefixAndTail(1)
.filter(_._1.nonEmpty)
.map { case (prefix, tail) (prefix.head, tail) }
}
private[http] def printEvent[T](marker: String): Flow[T, T, Unit] =
Flow[T].transform(() new PushPullStage[T, T] {
override def onPush(element: T, ctx: Context[T]): SyncDirective = {
@ -110,10 +102,6 @@ package object util {
}
}
private[this] val _identityFunc: Any Any = x x
/** Returns a constant identity function to avoid allocating the closure */
private[http] def identityFunc[T]: T T = _identityFunc.asInstanceOf[T T]
private[http] def humanReadableByteCount(bytes: Long, si: Boolean): String = {
val unit = if (si) 1000 else 1024
if (bytes >= unit) {

View file

@ -475,13 +475,14 @@ class RequestParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
.map(ByteString.apply)
.transform(() parser.stage).named("parser")
.splitWhen(x x.isInstanceOf[MessageStart] || x.isInstanceOf[EntityStreamError])
.via(headAndTailFlow)
.prefixAndTail(1)
.collect {
case (RequestStart(method, uri, protocol, headers, createEntity, _, close), entityParts)
case (Seq(RequestStart(method, uri, protocol, headers, createEntity, _, close)), entityParts)
closeAfterResponseCompletion :+= close
Right(HttpRequest(method, uri, headers, createEntity(entityParts), protocol))
case (x @ (MessageStartError(_, _) | EntityStreamError(_)), _) Left(x)
case (Seq(x @ (MessageStartError(_, _) | EntityStreamError(_))), _) Left(x)
}
.concatSubstreams
.flatMapConcat { x
Source {
x match {

View file

@ -293,13 +293,13 @@ class ResponseParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
.map(ByteString.apply)
.transform(() newParserStage(requestMethod)).named("parser")
.splitWhen(x x.isInstanceOf[MessageStart] || x.isInstanceOf[EntityStreamError])
.via(headAndTailFlow)
.prefixAndTail(1)
.collect {
case (ResponseStart(statusCode, protocol, headers, createEntity, close), entityParts)
case (Seq(ResponseStart(statusCode, protocol, headers, createEntity, close)), entityParts)
closeAfterResponseCompletion :+= close
Right(HttpResponse(statusCode, headers, createEntity(entityParts), protocol))
case (x @ (MessageStartError(_, _) | EntityStreamError(_)), _) Left(x)
}
case (Seq(x @ (MessageStartError(_, _) | EntityStreamError(_))), _) Left(x)
}.concatSubstreams
def collectBlocking[T](source: Source[T, Any]): Seq[T] =
Await.result(source.grouped(100000).runWith(Sink.head), 500.millis)

View file

@ -110,7 +110,7 @@ trait RouteTest extends RequestBuilding with WSTestRequestBuilding with RouteTes
* The result of the pipeline is the result that can later be checked with `check`. See the
* "separate running route from checking" example from ScalatestRouteTestSpec.scala.
*/
def runRoute: RouteTestResult RouteTestResult = akka.http.impl.util.identityFunc
def runRoute: RouteTestResult RouteTestResult = conforms
// there is already an implicit class WithTransformation in scope (inherited from akka.http.scaladsl.testkit.TransformerPipelineSupport)
// however, this one takes precedence

View file

@ -170,7 +170,7 @@ abstract class CoderSpec extends WordSpec with CodecSpecSupport with Inspectors
ByteString(output.toByteArray)
}
def decodeChunks(input: Source[ByteString, _]): ByteString =
def decodeChunks(input: Source[ByteString, Unit]): ByteString =
input.via(Coder.decoderFlow).join.awaitResult(3.seconds)
def decodeFromIterator(iterator: () Iterator[ByteString]): ByteString =

View file

@ -11,7 +11,7 @@ import akka.http.scaladsl.util.FastFuture._
trait PredefinedToRequestMarshallers {
private type TRM[T] = ToRequestMarshaller[T] // brevity alias
implicit val fromRequest: TRM[HttpRequest] = Marshaller.opaque(identity)
implicit val fromRequest: TRM[HttpRequest] = Marshaller.opaque(conforms)
implicit def fromUri: TRM[Uri] =
Marshaller strict { uri Marshalling.Opaque(() HttpRequest(uri = uri)) }
@ -26,4 +26,3 @@ trait PredefinedToRequestMarshallers {
}
object PredefinedToRequestMarshallers extends PredefinedToRequestMarshallers

View file

@ -18,7 +18,7 @@ trait PredefinedToResponseMarshallers extends LowPriorityToResponseMarshallerImp
implicit m: ToEntityMarshaller[T]): ToResponseMarshaller[T] =
fromStatusCodeAndHeadersAndValue compose (t (status, headers, t))
implicit val fromResponse: TRM[HttpResponse] = Marshaller.opaque(identity)
implicit val fromResponse: TRM[HttpResponse] = Marshaller.opaque(conforms)
implicit val fromStatusCode: TRM[StatusCode] =
Marshaller.withOpenCharset(`text/plain`) { (status, charset)
@ -46,4 +46,3 @@ trait LowPriorityToResponseMarshallerImplicits {
}
object PredefinedToResponseMarshallers extends PredefinedToResponseMarshallers

View file

@ -35,7 +35,7 @@ trait BasicDirectives {
Directive { inner ctx inner(())(ctx).fast.flatMap(f)(ctx.executionContext) }
def mapRouteResultPF(f: PartialFunction[RouteResult, RouteResult]): Directive0 =
mapRouteResult(f.applyOrElse(_, akka.http.impl.util.identityFunc[RouteResult]))
mapRouteResult(f.applyOrElse(_, conforms[RouteResult]))
def mapRouteResultWithPF(f: PartialFunction[RouteResult, Future[RouteResult]]): Directive0 =
mapRouteResultWith(f.applyOrElse(_, FastFuture.successful[RouteResult]))
@ -195,5 +195,5 @@ object BasicDirectives extends BasicDirectives {
private val _extractMaterializer: Directive1[Materializer] = extract(_.materializer)
private val _extractLog: Directive1[LoggingAdapter] = extract(_.log)
private val _extractSettings: Directive1[RoutingSettings] = extract(_.settings)
private val _extractRequestContext: Directive1[RequestContext] = extract(akka.http.impl.util.identityFunc)
}
private val _extractRequestContext: Directive1[RequestContext] = extract(conforms)
}

View file

@ -88,11 +88,12 @@ trait MultipartUnmarshallers {
val bodyParts = entity.dataBytes
.transform(() parser)
.splitWhen(_.isInstanceOf[PartStart])
.via(headAndTailFlow)
.prefixAndTail(1)
.collect {
case (BodyPartStart(headers, createEntity), entityParts) createBodyPart(createEntity(entityParts), headers)
case (ParseError(errorInfo), _) throw ParsingException(errorInfo)
case (Seq(BodyPartStart(headers, createEntity)), entityParts) createBodyPart(createEntity(entityParts), headers)
case (Seq(ParseError(errorInfo)), _) throw ParsingException(errorInfo)
}
.concatSubstreams
createStreamed(entity.contentType.mediaType.asInstanceOf[MediaType.Multipart], bodyParts)
}
}

View file

@ -17,7 +17,12 @@ class GroupByTest extends AkkaPublisherVerification[Int] {
if (elements == 0) EmptyPublisher[Int]
else {
val futureGroupSource =
Source(iterable(elements)).groupBy(elem "all").map { case (_, group) group }.runWith(Sink.head)
Source(iterable(elements))
.groupBy(1, elem "all")
.prefixAndTail(0)
.map(_._2)
.concatSubstreams
.runWith(Sink.head)
val groupSource = Await.result(futureGroupSource, 3.seconds)
groupSource.runWith(Sink.publisher(false))

View file

@ -16,7 +16,13 @@ class SplitWhenTest extends AkkaPublisherVerification[Int] {
def createPublisher(elements: Long): Publisher[Int] =
if (elements == 0) EmptyPublisher[Int]
else {
val futureSource = Source(iterable(elements)).splitWhen(elem false).runWith(Sink.head)
val futureSource =
Source(iterable(elements))
.splitWhen(elem false)
.prefixAndTail(0)
.map(_._2)
.concatSubstreams
.runWith(Sink.head)
val source = Await.result(futureSource, 3.seconds)
source.runWith(Sink.publisher(false))
}

View file

@ -42,7 +42,7 @@ object TestPublisher {
}
/**
* Publisher that signals error to subscribers immediately, before handing out subscription.
* Publisher that signals error to subscribers immediately after handing out subscription.
*/
def error[T](cause: Throwable): Publisher[T] = ErrorPublisher(cause, "error").asInstanceOf[Publisher[T]]
@ -251,7 +251,10 @@ object TestSubscriber {
/**
* Expect and return a stream element.
*/
def expectNext(): I = probe.expectMsgType[OnNext[I]].element
def expectNext(): I = probe.receiveOne(probe.remaining) match {
case OnNext(elem) elem.asInstanceOf[I]
case other throw new AssertionError("expected OnNext, found " + other)
}
/**
* Fluent DSL

View file

@ -7,16 +7,16 @@ import org.reactivestreams.Publisher
import akka.stream.ActorMaterializer
import akka.stream.Attributes
class ChainSetup[In, Out](
stream: Flow[In, In, _] Flow[In, Out, _],
class ChainSetup[In, Out, M](
stream: Flow[In, In, Unit] Flow[In, Out, M],
val settings: ActorMaterializerSettings,
materializer: ActorMaterializer,
toPublisher: (Source[Out, _], ActorMaterializer) Publisher[Out])(implicit val system: ActorSystem) {
def this(stream: Flow[In, In, _] Flow[In, Out, _], settings: ActorMaterializerSettings, toPublisher: (Source[Out, _], ActorMaterializer) Publisher[Out])(implicit system: ActorSystem) =
def this(stream: Flow[In, In, Unit] Flow[In, Out, M], settings: ActorMaterializerSettings, toPublisher: (Source[Out, _], ActorMaterializer) Publisher[Out])(implicit system: ActorSystem) =
this(stream, settings, ActorMaterializer(settings)(system), toPublisher)(system)
def this(stream: Flow[In, In, _] Flow[In, Out, _],
def this(stream: Flow[In, In, Unit] Flow[In, Out, M],
settings: ActorMaterializerSettings,
materializerCreator: (ActorMaterializerSettings, ActorRefFactory) ActorMaterializer,
toPublisher: (Source[Out, _], ActorMaterializer) Publisher[Out])(implicit system: ActorSystem) =

View file

@ -86,8 +86,8 @@ trait ScriptedTest extends Matchers {
def debug: String = s"Script(pending=($pendingIns in, $pendingOuts out), remainingIns=${providedInputs.drop(inputCursor).mkString("/")}, remainingOuts=${expectedOutputs.drop(outputCursor).mkString("/")})"
}
class ScriptRunner[In, Out](
op: Flow[In, In, _] Flow[In, Out, _],
class ScriptRunner[In, Out, M](
op: Flow[In, In, Unit] Flow[In, Out, M],
settings: ActorMaterializerSettings,
script: Script[In, Out],
maximumOverrun: Int,
@ -196,8 +196,8 @@ trait ScriptedTest extends Matchers {
}
def runScript[In, Out](script: Script[In, Out], settings: ActorMaterializerSettings, maximumOverrun: Int = 3, maximumRequest: Int = 3, maximumBuffer: Int = 3)(
op: Flow[In, In, _] Flow[In, Out, _])(implicit system: ActorSystem): Unit = {
def runScript[In, Out, M](script: Script[In, Out], settings: ActorMaterializerSettings, maximumOverrun: Int = 3, maximumRequest: Int = 3, maximumBuffer: Int = 3)(
op: Flow[In, In, Unit] Flow[In, Out, M])(implicit system: ActorSystem): Unit = {
new ScriptRunner(op, settings, script, maximumOverrun, maximumRequest, maximumBuffer).run()
}

View file

@ -31,8 +31,7 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import static akka.stream.testkit.StreamTestKit.PublisherProbeSubscription;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
import static org.junit.Assert.*;
@SuppressWarnings("serial")
public class FlowTest extends StreamTest {
@ -237,122 +236,71 @@ public class FlowTest extends StreamTest {
probe.expectMsgEquals(6);
}
@SuppressWarnings("unchecked")
@Test
public void mustBeAbleToUseGroupBy() {
final JavaTestKit probe = new JavaTestKit(system);
public void mustBeAbleToUseGroupBy() throws Exception {
final Iterable<String> input = Arrays.asList("Aaa", "Abb", "Bcc", "Cdd", "Cee");
final Flow<String, Pair<String, Source<String, BoxedUnit>>, BoxedUnit> slsFlow = Flow
.of(String.class).groupBy(new Function<String, String>() {
final Flow<String, List<String>, BoxedUnit> flow = Flow
.of(String.class)
.groupBy(3, new Function<String, String>() {
public String apply(String elem) {
return elem.substring(0, 1);
}
});
Source.from(input).via(slsFlow).runForeach(new Procedure<Pair<String, Source<String, BoxedUnit>>>() {
})
.grouped(10)
.mergeSubstreams();
final Future<List<List<String>>> future =
Source.from(input).via(flow).grouped(10).runWith(Sink.<List<List<String>>> head(), materializer);
final Object[] result = Await.result(future, Duration.create(1, TimeUnit.SECONDS)).toArray();
Arrays.sort(result, (Comparator<Object>)(Object) new Comparator<List<String>>() {
@Override
public void apply(final Pair<String, Source<String, BoxedUnit>> pair) throws Exception {
pair.second().runForeach(new Procedure<String>() {
@Override
public void apply(String elem) throws Exception {
probe.getRef().tell(new Pair<String, String>(pair.first(), elem), ActorRef.noSender());
}
}, materializer);
}
}, materializer);
Map<String, List<String>> grouped = new HashMap<String, List<String>>();
for (Object o : probe.receiveN(5)) {
@SuppressWarnings("unchecked")
Pair<String, String> p = (Pair<String, String>) o;
List<String> g = grouped.get(p.first());
if (g == null) {
g = new ArrayList<String>();
}
g.add(p.second());
grouped.put(p.first(), g);
}
assertEquals(Arrays.asList("Aaa", "Abb"), grouped.get("A"));
}
@Test
public void mustBeAbleToUseSplitWhen() {
final JavaTestKit probe = new JavaTestKit(system);
final Iterable<String> input = Arrays.asList("A", "B", "C", ".", "D", ".", "E", "F");
final Flow<String, Source<String, BoxedUnit>, ?> flow = Flow.of(String.class).splitWhen(new Predicate<String>() {
public boolean test(String elem) {
return elem.equals(".");
}
});
Source.from(input).via(flow).runForeach(new Procedure<Source<String, BoxedUnit>>() {
@Override
public void apply(Source<String, BoxedUnit> subStream) throws Exception {
subStream.filter(new Predicate<String>() {
@Override
public boolean test(String elem) {
return !elem.equals(".");
}
}).grouped(10).runForeach(new Procedure<List<String>>() {
@Override
public void apply(List<String> chunk) throws Exception {
probe.getRef().tell(chunk, ActorRef.noSender());
}
}, materializer);
}
}, materializer);
for (Object o : probe.receiveN(3)) {
@SuppressWarnings("unchecked")
List<String> chunk = (List<String>) o;
if (chunk.get(0).equals("A")) {
assertEquals(Arrays.asList("A", "B", "C"), chunk);
} else if (chunk.get(0).equals("D")) {
assertEquals(Arrays.asList("D"), chunk);
} else if (chunk.get(0).equals("E")) {
assertEquals(Arrays.asList("E", "F"), chunk);
} else {
assertEquals("[A, B, C] or [D] or [E, F]", chunk);
}
}
}
@Test
public void mustBeAbleToUseSplitAfter() {
final JavaTestKit probe = new JavaTestKit(system);
final Iterable<String> input = Arrays.asList("A", "B", "C", ".", "D", ".", "E", "F");
final Flow<String, Source<String, BoxedUnit>, ?> flow = Flow.of(String.class).splitAfter(new Predicate<String>() {
public boolean test(String elem) {
return elem.equals(".");
public int compare(List<String> o1, List<String> o2) {
return o1.get(0).charAt(0) - o2.get(0).charAt(0);
}
});
Source.from(input).via(flow).runForeach(new Procedure<Source<String, BoxedUnit>>() {
@Override
public void apply(Source<String, BoxedUnit> subStream) throws Exception {
subStream.grouped(10).runForeach(new Procedure<List<String>>() {
@Override
public void apply(List<String> chunk) throws Exception {
probe.getRef().tell(chunk, ActorRef.noSender());
assertArrayEquals(new Object[] { Arrays.asList("Aaa", "Abb"), Arrays.asList("Bcc"), Arrays.asList("Cdd", "Cee") }, result);
}
@Test
public void mustBeAbleToUseSplitWhen() throws Exception {
final Iterable<String> input = Arrays.asList("A", "B", "C", ".", "D", ".", "E", "F");
final Flow<String, List<String>, BoxedUnit> flow = Flow
.of(String.class)
.splitWhen(new Predicate<String>() {
public boolean test(String elem) {
return elem.equals(".");
}
}, materializer);
}
}, materializer);
})
.grouped(10)
.concatSubstreams();
for (Object o : probe.receiveN(3)) {
@SuppressWarnings("unchecked")
List<String> chunk = (List<String>) o;
if (chunk.get(0).equals("A")) {
assertEquals(Arrays.asList("A", "B", "C", "."), chunk);
} else if (chunk.get(0).equals("D")) {
assertEquals(Arrays.asList("D", "."), chunk);
} else if (chunk.get(0).equals("E")) {
assertEquals(Arrays.asList("E", "F"), chunk);
} else {
assertEquals("[A, B, C, .] or [D, .] or [E, F]", chunk);
}
}
final Future<List<List<String>>> future =
Source.from(input).via(flow).grouped(10).runWith(Sink.<List<List<String>>> head(), materializer);
final List<List<String>> result = Await.result(future, Duration.create(1, TimeUnit.SECONDS));
assertEquals(Arrays.asList(Arrays.asList("A", "B", "C"), Arrays.asList(".", "D"), Arrays.asList(".", "E", "F")), result);
}
@Test
public void mustBeAbleToUseSplitAfter() throws Exception {
final Iterable<String> input = Arrays.asList("A", "B", "C", ".", "D", ".", "E", "F");
final Flow<String, List<String>, BoxedUnit> flow = Flow
.of(String.class)
.splitAfter(new Predicate<String>() {
public boolean test(String elem) {
return elem.equals(".");
}
})
.grouped(10)
.concatSubstreams();
final Future<List<List<String>>> future =
Source.from(input).via(flow).grouped(10).runWith(Sink.<List<List<String>>> head(), materializer);
final List<List<String>> result = Await.result(future, Duration.create(1, TimeUnit.SECONDS));
assertEquals(Arrays.asList(Arrays.asList("A", "B", "C", "."), Arrays.asList("D", "."), Arrays.asList("E", "F")), result);
}
public <T> Creator<Stage<T, T>> op() {

View file

@ -36,8 +36,7 @@ import java.util.concurrent.TimeoutException;
import static akka.stream.testkit.StreamTestKit.PublisherProbeSubscription;
import static akka.stream.testkit.TestPublisher.ManualProbe;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
import static org.junit.Assert.*;
@SuppressWarnings("serial")
public class SourceTest extends StreamTest {
@ -54,7 +53,7 @@ public class SourceTest extends StreamTest {
final JavaTestKit probe = new JavaTestKit(system);
final String[] lookup = {"a", "b", "c", "d", "e", "f"};
final java.lang.Iterable<Integer> input = Arrays.asList(0, 1, 2, 3, 4, 5);
final Source<Integer, ?> ints = Source.from(input);
final Source<Integer, BoxedUnit> ints = Source.from(input);
ints.drop(2).take(3).takeWithin(FiniteDuration.create(10, TimeUnit.SECONDS)).map(new Function<Integer, String>() {
public String apply(Integer elem) {
@ -164,81 +163,71 @@ public class SourceTest extends StreamTest {
probe.expectMsgEquals(6);
}
@SuppressWarnings("unchecked")
@Test
public void mustBeAbleToUseGroupBy() {
final JavaTestKit probe = new JavaTestKit(system);
public void mustBeAbleToUseGroupBy() throws Exception {
final Iterable<String> input = Arrays.asList("Aaa", "Abb", "Bcc", "Cdd", "Cee");
Source.from(input).groupBy(new Function<String, String>() {
public String apply(String elem) {
return elem.substring(0, 1);
}
}).runForeach(new Procedure<Pair<String, Source<String, BoxedUnit>>>() {
@Override
public void apply(final Pair<String, Source<String, BoxedUnit>> pair) throws Exception {
pair.second().runForeach(new Procedure<String>() {
@Override
public void apply(String elem) throws Exception {
probe.getRef().tell(new Pair<String, String>(pair.first(), elem), ActorRef.noSender());
final Source<List<String>, BoxedUnit> source = Source
.from(input)
.groupBy(3, new Function<String, String>() {
public String apply(String elem) {
return elem.substring(0, 1);
}
}, materializer);
})
.grouped(10)
.mergeSubstreams();
final Future<List<List<String>>> future =
source.grouped(10).runWith(Sink.<List<List<String>>> head(), materializer);
final Object[] result = Await.result(future, Duration.create(1, TimeUnit.SECONDS)).toArray();
Arrays.sort(result, (Comparator<Object>)(Object) new Comparator<List<String>>() {
@Override
public int compare(List<String> o1, List<String> o2) {
return o1.get(0).charAt(0) - o2.get(0).charAt(0);
}
}, materializer);
Map<String, List<String>> grouped = new HashMap<String, List<String>>();
for (Object o : probe.receiveN(5)) {
@SuppressWarnings("unchecked")
Pair<String, String> p = (Pair<String, String>) o;
List<String> g = grouped.get(p.first());
if (g == null) {
g = new ArrayList<String>();
}
g.add(p.second());
grouped.put(p.first(), g);
}
assertEquals(Arrays.asList("Aaa", "Abb"), grouped.get("A"));
});
assertArrayEquals(new Object[] { Arrays.asList("Aaa", "Abb"), Arrays.asList("Bcc"), Arrays.asList("Cdd", "Cee") }, result);
}
@Test
public void mustBeAbleToUseSplitWhen() {
final JavaTestKit probe = new JavaTestKit(system);
public void mustBeAbleToUseSplitWhen() throws Exception {
final Iterable<String> input = Arrays.asList("A", "B", "C", ".", "D", ".", "E", "F");
Source.from(input).splitWhen(new Predicate<String>() {
public boolean test(String elem) {
return elem.equals(".");
}
}).runForeach(new Procedure<Source<String, BoxedUnit>>() {
@Override
public void apply(Source<String, BoxedUnit> subStream) throws Exception {
subStream.filter(new Predicate<String>() {
@Override
public boolean test(String elem) {
return !elem.equals(".");
final Source<List<String>, BoxedUnit> source = Source
.from(input)
.splitWhen(new Predicate<String>() {
public boolean test(String elem) {
return elem.equals(".");
}
}).grouped(10).runForeach(new Procedure<List<String>>() {
@Override
public void apply(List<String> chunk) throws Exception {
probe.getRef().tell(chunk, ActorRef.noSender());
}
}, materializer);
}
}, materializer);
})
.grouped(10)
.concatSubstreams();
for (Object o : probe.receiveN(3)) {
@SuppressWarnings("unchecked")
List<String> chunk = (List<String>) o;
if (chunk.get(0).equals("A")) {
assertEquals(Arrays.asList("A", "B", "C"), chunk);
} else if (chunk.get(0).equals("D")) {
assertEquals(Arrays.asList("D"), chunk);
} else if (chunk.get(0).equals("E")) {
assertEquals(Arrays.asList("E", "F"), chunk);
} else {
assertEquals("[A, B, C] or [D] or [E, F]", chunk);
}
}
final Future<List<List<String>>> future =
source.grouped(10).runWith(Sink.<List<List<String>>> head(), materializer);
final List<List<String>> result = Await.result(future, Duration.create(1, TimeUnit.SECONDS));
assertEquals(Arrays.asList(Arrays.asList("A", "B", "C"), Arrays.asList(".", "D"), Arrays.asList(".", "E", "F")), result);
}
@Test
public void mustBeAbleToUseSplitAfter() throws Exception {
final Iterable<String> input = Arrays.asList("A", "B", "C", ".", "D", ".", "E", "F");
final Source<List<String>, BoxedUnit> source = Source
.from(input)
.splitAfter(new Predicate<String>() {
public boolean test(String elem) {
return elem.equals(".");
}
})
.grouped(10)
.concatSubstreams();
final Future<List<List<String>>> future =
source.grouped(10).runWith(Sink.<List<List<String>>> head(), materializer);
final List<List<String>> result = Await.result(future, Duration.create(1, TimeUnit.SECONDS));
assertEquals(Arrays.asList(Arrays.asList("A", "B", "C", "."), Arrays.asList("D", "."), Arrays.asList("E", "F")), result);
}
@Test

View file

@ -8,19 +8,30 @@ import java.lang.reflect.Modifier
import org.scalatest.Matchers
import org.scalatest.WordSpec
object DslConsistencySpec {
class ScalaSubSource[Out, Mat] extends impl.SubFlowImpl[Out, Out, Mat, scaladsl.Source[Out, Mat]#Repr, scaladsl.RunnableGraph[Mat]](null, null, null)
class ScalaSubFlow[In, Out, Mat] extends impl.SubFlowImpl[Out, Out, Mat, scaladsl.Flow[In, Out, Mat]#Repr, scaladsl.Sink[In, Mat]](null, null, null)
}
class DslConsistencySpec extends WordSpec with Matchers {
val sFlowClass = classOf[akka.stream.scaladsl.Flow[_, _, _]]
val jFlowClass = classOf[akka.stream.javadsl.Flow[_, _, _]]
val sFlowClass: Class[_] = classOf[akka.stream.scaladsl.Flow[_, _, _]]
val jFlowClass: Class[_] = classOf[akka.stream.javadsl.Flow[_, _, _]]
val sSourceClass = classOf[akka.stream.scaladsl.Source[_, _]]
val jSourceClass = classOf[akka.stream.javadsl.Source[_, _]]
val sSubFlowClass: Class[_] = classOf[DslConsistencySpec.ScalaSubFlow[_, _, _]]
val jSubFlowClass: Class[_] = classOf[akka.stream.javadsl.SubFlow[_, _, _]]
val sSinkClass = classOf[akka.stream.scaladsl.Sink[_, _]]
val jSinkClass = classOf[akka.stream.javadsl.Sink[_, _]]
val sSourceClass: Class[_] = classOf[akka.stream.scaladsl.Source[_, _]]
val jSourceClass: Class[_] = classOf[akka.stream.javadsl.Source[_, _]]
val jRunnableGraphClass = classOf[akka.stream.javadsl.RunnableGraph[_]]
val sRunnableGraphClass = classOf[akka.stream.scaladsl.RunnableGraph[_]]
val sSubSourceClass: Class[_] = classOf[DslConsistencySpec.ScalaSubSource[_, _]]
val jSubSourceClass: Class[_] = classOf[akka.stream.javadsl.SubSource[_, _]]
val sSinkClass: Class[_] = classOf[akka.stream.scaladsl.Sink[_, _]]
val jSinkClass: Class[_] = classOf[akka.stream.javadsl.Sink[_, _]]
val jRunnableGraphClass: Class[_] = classOf[akka.stream.javadsl.RunnableGraph[_]]
val sRunnableGraphClass: Class[_] = classOf[akka.stream.scaladsl.RunnableGraph[_]]
val ignore =
Set("equals", "hashCode", "notify", "notifyAll", "wait", "toString", "getClass") ++
@ -28,13 +39,16 @@ class DslConsistencySpec extends WordSpec with Matchers {
Set("create", "apply", "ops", "appendJava", "andThen", "andThenMat", "isIdentity", "withAttributes", "transformMaterializing") ++
Set("asScala", "asJava", "deprecatedAndThen", "deprecatedAndThenMat")
val graphHelpers = Set("zipGraph", "zipWithGraph", "mergeGraph", "concatGraph", "alsoToGraph")
val allowMissing: Map[Class[_], Set[String]] = Map(
jFlowClass -> graphHelpers,
jSourceClass -> graphHelpers,
// 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")),
sFlowClass -> Set("of"),
sSourceClass -> Set("adapt", "from"),
sSinkClass -> Set("adapt"),
jSinkClass -> Set(),
sRunnableGraphClass -> Set("builder"))
def materializing(m: Method): Boolean = m.getParameterTypes.contains(classOf[ActorMaterializer])
@ -48,7 +62,9 @@ class DslConsistencySpec extends WordSpec with Matchers {
"Java and Scala DSLs" must {
("Source" -> List(sSourceClass, jSourceClass)) ::
("SubSource" -> List(sSubSourceClass, jSubSourceClass)) ::
("Flow" -> List(sFlowClass, jFlowClass)) ::
("SubFlow" -> List(sSubFlowClass, jSubFlowClass)) ::
("Sink" -> List(sSinkClass, jSinkClass)) ::
("RunanbleFlow" -> List(sRunnableGraphClass, jRunnableGraphClass)) ::
Nil foreach {

View file

@ -44,7 +44,13 @@ class FlowConcatAllSpec extends AkkaSpec {
"work together with SplitWhen" in {
val subscriber = TestSubscriber.manualProbe[Int]()
Source(1 to 10).splitWhen(_ % 2 == 0).flatMapConcat(ConstantFun.scalaIdentityFunction).runWith(Sink(subscriber))
Source(1 to 10)
.splitWhen(_ % 2 == 0)
.prefixAndTail(0)
.map(_._2)
.concatSubstreams
.flatMapConcat(ConstantFun.scalaIdentityFunction)
.runWith(Sink(subscriber))
val subscription = subscriber.expectSubscription()
subscription.request(10)
for (i (1 to 10))

View file

@ -73,13 +73,10 @@ class FlowConcatSpec extends BaseTwoStreamsSetup {
"work with one immediately failed and one nonempty publisher" in assertAllStagesStopped {
val subscriber1 = setup(failedPublisher, nonemptyPublisher(1 to 4))
subscriber1.expectSubscriptionAndError(TestException)
val subscriber2 = setup(nonemptyPublisher(1 to 4), failedPublisher)
subscriber2.expectSubscriptionAndError(TestException)
}
"work with one nonempty and one delayed failed publisher" in assertAllStagesStopped {
val subscriber = setup(nonemptyPublisher(1 to 4), soonToFailPublisher)
"work with one nonempty and one immediately failed publisher" in assertAllStagesStopped {
val subscriber = setup(nonemptyPublisher(1 to 4), failedPublisher)
subscriber.expectSubscription().request(5)
val errorSignalled = (1 to 4).foldLeft(false)((errorSignalled, e)
@ -89,6 +86,11 @@ class FlowConcatSpec extends BaseTwoStreamsSetup {
"work with one delayed failed and one nonempty publisher" in assertAllStagesStopped {
val subscriber = setup(soonToFailPublisher, nonemptyPublisher(1 to 4))
subscriber.expectSubscriptionAndError(TestException)
}
"work with one nonempty and one delayed failed publisher" in assertAllStagesStopped {
val subscriber = setup(nonemptyPublisher(1 to 4), soonToFailPublisher)
subscriber.expectSubscription().request(5)
val errorSignalled = (1 to 4).foldLeft(false)((errorSignalled, e)

View file

@ -13,8 +13,23 @@ import akka.stream.testkit.Utils._
import org.reactivestreams.Publisher
import akka.stream.Attributes
import akka.stream.ActorAttributes
import org.scalatest.concurrent.ScalaFutures
import org.scalactic.ConversionCheckedTripleEquals
import org.scalatest.concurrent.PatienceConfiguration.Timeout
import akka.stream.testkit.scaladsl.TestSource
import akka.stream.testkit.scaladsl.TestSink
class FlowGroupBySpec extends AkkaSpec {
object FlowGroupBySpec {
import language.higherKinds
implicit class Lift[M](val f: SubFlow[Int, M, Source[Int, M]#Repr, RunnableGraph[M]]) extends AnyVal {
def lift(key: Int Int) = f.prefixAndTail(1).map(p key(p._1.head) -> (Source.single(p._1.head) ++ p._2)).mergeSubstreams
}
}
class FlowGroupBySpec extends AkkaSpec with ScalaFutures with ConversionCheckedTripleEquals {
import FlowGroupBySpec._
val settings = ActorMaterializerSettings(system)
.withInputBuffer(initialSize = 2, maxSize = 2)
@ -34,9 +49,10 @@ class FlowGroupBySpec extends AkkaSpec {
def cancel(): Unit = subscription.cancel()
}
class SubstreamsSupport(groupCount: Int = 2, elementCount: Int = 6) {
class SubstreamsSupport(groupCount: Int = 2, elementCount: Int = 6, maxSubstreams: Int = -1) {
val source = Source(1 to elementCount).runWith(Sink.publisher(false))
val groupStream = Source(source).groupBy(_ % groupCount).runWith(Sink.publisher(false))
val max = if (maxSubstreams > 0) maxSubstreams else groupCount
val groupStream = Source(source).groupBy(max, _ % groupCount).lift(_ % groupCount).runWith(Sink.publisher(false))
val masterSubscriber = TestSubscriber.manualProbe[(Int, Source[Int, _])]()
groupStream.subscribe(masterSubscriber)
@ -89,10 +105,21 @@ class FlowGroupBySpec extends AkkaSpec {
s1.expectNext(5)
s1.expectComplete()
masterSubscription.request(1)
masterSubscriber.expectComplete()
}
}
"work in normal user scenario" in {
Source(List("Aaa", "Abb", "Bcc", "Cdd", "Cee"))
.groupBy(3, _.substring(0, 1))
.grouped(10)
.mergeSubstreams
.grouped(10)
.runWith(Sink.head)
.futureValue(Timeout(3.seconds)) should ===(List(List("Aaa", "Abb"), List("Bcc"), List("Cdd", "Cee")))
}
"accept cancellation of substreams" in assertAllStagesStopped {
new SubstreamsSupport(groupCount = 2) {
StreamPuppet(getSubFlow(1).runWith(Sink.publisher(false))).cancel()
@ -107,13 +134,14 @@ class FlowGroupBySpec extends AkkaSpec {
substream.expectNext(6)
substream.expectComplete()
masterSubscription.request(1)
masterSubscriber.expectComplete()
}
}
"accept cancellation of master stream when not consumed anything" in assertAllStagesStopped {
val publisherProbeProbe = TestPublisher.manualProbe[Int]()
val publisher = Source(publisherProbeProbe).groupBy(_ % 2).runWith(Sink.publisher(false))
val publisher = Source(publisherProbeProbe).groupBy(2, _ % 2).lift(_ % 2).runWith(Sink.publisher(false))
val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, _])]()
publisher.subscribe(subscriber)
@ -123,28 +151,8 @@ class FlowGroupBySpec extends AkkaSpec {
upstreamSubscription.expectCancellation()
}
"accept cancellation of master stream when substreams are open" in assertAllStagesStopped {
new SubstreamsSupport(groupCount = 3, elementCount = 13) {
val substream = StreamPuppet(getSubFlow(1).runWith(Sink.publisher(false)))
substream.request(1)
substream.expectNext(1)
masterSubscription.cancel()
masterSubscriber.expectNoMsg(100.millis)
// Open substreams still work, others are discarded
substream.request(4)
substream.expectNext(4)
substream.expectNext(7)
substream.expectNext(10)
substream.expectNext(13)
substream.expectComplete()
}
}
"work with empty input stream" in assertAllStagesStopped {
val publisher = Source(List.empty[Int]).groupBy(_ % 2).runWith(Sink.publisher(false))
val publisher = Source(List.empty[Int]).groupBy(2, _ % 2).lift(_ % 2).runWith(Sink.publisher(false))
val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, _])]()
publisher.subscribe(subscriber)
@ -153,7 +161,7 @@ class FlowGroupBySpec extends AkkaSpec {
"abort on onError from upstream" in assertAllStagesStopped {
val publisherProbeProbe = TestPublisher.manualProbe[Int]()
val publisher = Source(publisherProbeProbe).groupBy(_ % 2).runWith(Sink.publisher(false))
val publisher = Source(publisherProbeProbe).groupBy(2, _ % 2).lift(_ % 2).runWith(Sink.publisher(false))
val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, _])]()
publisher.subscribe(subscriber)
@ -170,7 +178,7 @@ class FlowGroupBySpec extends AkkaSpec {
"abort on onError from upstream when substreams are running" in assertAllStagesStopped {
val publisherProbeProbe = TestPublisher.manualProbe[Int]()
val publisher = Source(publisherProbeProbe).groupBy(_ % 2).runWith(Sink.publisher(false))
val publisher = Source(publisherProbeProbe).groupBy(2, _ % 2).lift(_ % 2).runWith(Sink.publisher(false))
val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, _])]()
publisher.subscribe(subscriber)
@ -199,7 +207,8 @@ class FlowGroupBySpec extends AkkaSpec {
val publisherProbeProbe = TestPublisher.manualProbe[Int]()
val exc = TE("test")
val publisher = Source(publisherProbeProbe)
.groupBy(elem if (elem == 2) throw exc else elem % 2)
.groupBy(2, elem if (elem == 2) throw exc else elem % 2)
.lift(_ % 2)
.runWith(Sink.publisher(false))
val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, Unit])]()
publisher.subscribe(subscriber)
@ -228,7 +237,8 @@ class FlowGroupBySpec extends AkkaSpec {
val publisherProbeProbe = TestPublisher.manualProbe[Int]()
val exc = TE("test")
val publisher = Source(publisherProbeProbe)
.groupBy(elem if (elem == 2) throw exc else elem % 2)
.groupBy(2, elem if (elem == 2) throw exc else elem % 2)
.lift(_ % 2)
.withAttributes(ActorAttributes.supervisionStrategy(resumingDecider))
.runWith(Sink.publisher(false))
val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, Unit])]()
@ -270,7 +280,7 @@ class FlowGroupBySpec extends AkkaSpec {
val up = TestPublisher.manualProbe[Int]()
val down = TestSubscriber.manualProbe[(Int, Source[Int, Unit])]()
val flowSubscriber = Source.subscriber[Int].groupBy(_ % 2).to(Sink(down)).run()
val flowSubscriber = Source.subscriber[Int].groupBy(2, _ % 2).lift(_ % 2).to(Sink(down)).run()
val downstream = down.expectSubscription()
downstream.cancel()
@ -279,6 +289,26 @@ class FlowGroupBySpec extends AkkaSpec {
upsub.expectCancellation()
}
"fail when exceeding maxSubstreams" in assertAllStagesStopped {
val (up, down) = Flow[Int]
.groupBy(1, _ % 2).prefixAndTail(0).mergeSubstreams
.runWith(TestSource.probe[Int], TestSink.probe)
down.request(2)
up.sendNext(1)
val first = down.expectNext()
val s1 = StreamPuppet(first._2.runWith(Sink.publisher(false)))
s1.request(1)
s1.expectNext(1)
up.sendNext(2)
val ex = down.expectError()
ex.getMessage should include("too many substreams")
s1.expectError(ex)
}
}
}

View file

@ -38,8 +38,8 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
implicit val mat = ActorMaterializer(settings)
val identity: Flow[Any, Any, _] Flow[Any, Any, _] = in in.map(e e)
val identity2: Flow[Any, Any, _] Flow[Any, Any, _] = in identity(in)
val identity: Flow[Any, Any, Unit] Flow[Any, Any, Unit] = in in.map(e e)
val identity2: Flow[Any, Any, Unit] Flow[Any, Any, Unit] = in identity(in)
class BrokenActorInterpreter(
_assembly: GraphAssembly,
@ -63,7 +63,7 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
}
}
val faultyFlow: Flow[Any, Any, _] Flow[Any, Any, _] = in in.via({
val faultyFlow: Flow[Any, Any, Unit] Flow[Any, Any, Unit] = in in.via({
val stage = new PushPullGraphStage((_) fusing.Map({ x: Any x }, stoppingDecider), Attributes.none)
val assembly = new GraphAssembly(
@ -302,11 +302,11 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
"be covariant" in {
val f1: Source[Fruit, _] = Source[Fruit](apples)
val p1: Publisher[Fruit] = Source[Fruit](apples).runWith(Sink.publisher(false))
val f2: Source[Source[Fruit, _], _] = Source[Fruit](apples).splitWhen(_ true)
val f3: Source[(Boolean, Source[Fruit, _]), _] = Source[Fruit](apples).groupBy(_ true)
val f2: SubFlow[Fruit, _, Source[Fruit, Unit]#Repr, _] = Source[Fruit](apples).splitWhen(_ true)
val f3: SubFlow[Fruit, _, Source[Fruit, Unit]#Repr, _] = Source[Fruit](apples).groupBy(2, _ true)
val f4: Source[(immutable.Seq[Fruit], Source[Fruit, _]), _] = Source[Fruit](apples).prefixAndTail(1)
val d1: Flow[String, Source[Fruit, _], _] = Flow[String].map(_ new Apple).splitWhen(_ true)
val d2: Flow[String, (Boolean, Source[Fruit, _]), _] = Flow[String].map(_ new Apple).groupBy(_ true)
val d1: SubFlow[Fruit, _, Flow[String, Fruit, Unit]#Repr, _] = Flow[String].map(_ new Apple).splitWhen(_ true)
val d2: SubFlow[Fruit, _, Flow[String, Fruit, Unit]#Repr, _] = Flow[String].map(_ new Apple).groupBy(2, _ true)
val d3: Flow[String, (immutable.Seq[Apple], Source[Fruit, _]), _] = Flow[String].map(_ new Apple).prefixAndTail(1)
}
@ -507,7 +507,7 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
}
"call future subscribers' onError should be called instead of onSubscribed after initial upstream reported an error" in {
new ChainSetup[Int, String](_.map(_ throw TestException), settings.withInputBuffer(initialSize = 1, maxSize = 1),
new ChainSetup[Int, String, Unit](_.map(_ throw TestException), settings.withInputBuffer(initialSize = 1, maxSize = 1),
toFanoutPublisher(1)) {
downstreamSubscription.request(1)
upstreamSubscription.expectRequest(1)

View file

@ -12,13 +12,25 @@ import akka.stream.testkit.TestPublisher
import akka.stream.testkit.TestSubscriber
import akka.stream.testkit.Utils._
import org.reactivestreams.Publisher
import scala.concurrent.duration._
import akka.stream.StreamSubscriptionTimeoutSettings
import akka.stream.StreamSubscriptionTimeoutTerminationMode
object FlowSplitAfterSpec {
import language.higherKinds
implicit class Lift[M](val f: SubFlow[Int, M, Source[Int, M]#Repr, RunnableGraph[M]]) extends AnyVal {
def lift = f.prefixAndTail(0).map(_._2).concatSubstreams
}
}
class FlowSplitAfterSpec extends AkkaSpec {
import FlowSplitAfterSpec._
val settings = ActorMaterializerSettings(system)
.withInputBuffer(initialSize = 2, maxSize = 2)
.withSubscriptionTimeoutSettings(StreamSubscriptionTimeoutSettings(StreamSubscriptionTimeoutTerminationMode.cancel, 1.second))
implicit val materializer = ActorMaterializer(settings)
@ -37,7 +49,7 @@ class FlowSplitAfterSpec extends AkkaSpec {
class SubstreamsSupport(splitAfter: Int = 3, elementCount: Int = 6) {
val source = Source(1 to elementCount)
val groupStream = source.splitAfter(_ == splitAfter).runWith(Sink.publisher(false))
val groupStream = source.splitAfter(_ == splitAfter).lift.runWith(Sink.publisher(false))
val masterSubscriber = TestSubscriber.manualProbe[Source[Int, _]]()
groupStream.subscribe(masterSubscriber)
@ -77,6 +89,7 @@ class FlowSplitAfterSpec extends AkkaSpec {
s2.expectNext(5)
s2.expectComplete()
masterSubscription.request(1)
masterSubscriber.expectComplete()
}
}
@ -97,6 +110,7 @@ class FlowSplitAfterSpec extends AkkaSpec {
s2.expectNext(3)
s2.expectComplete()
masterSubscription.request(1)
masterSubscriber.expectComplete()
}
}
@ -138,6 +152,7 @@ class FlowSplitAfterSpec extends AkkaSpec {
val exc = TE("test")
val publisher = Source(publisherProbeProbe)
.splitAfter(elem if (elem == 3) throw exc else elem % 3 == 0)
.lift
.runWith(Sink.publisher(false))
val subscriber = TestSubscriber.manualProbe[Source[Int, Unit]]()
publisher.subscribe(subscriber)
@ -170,6 +185,7 @@ class FlowSplitAfterSpec extends AkkaSpec {
val exc = TE("test")
val publisher = Source(publisherProbeProbe)
.splitAfter(elem if (elem == 3) throw exc else elem % 3 == 0)
.lift
.withAttributes(ActorAttributes.supervisionStrategy(resumingDecider))
.runWith(Sink.publisher(false))
val subscriber = TestSubscriber.manualProbe[Source[Int, Unit]]()
@ -216,7 +232,7 @@ class FlowSplitAfterSpec extends AkkaSpec {
val up = TestPublisher.manualProbe[Int]()
val down = TestSubscriber.manualProbe[Source[Int, Unit]]()
val flowSubscriber = Source.subscriber[Int].splitAfter(_ % 3 == 0).to(Sink(down)).run()
val flowSubscriber = Source.subscriber[Int].splitAfter(_ % 3 == 0).lift.to(Sink(down)).run()
val downstream = down.expectSubscription()
downstream.cancel()

View file

@ -10,13 +10,16 @@ import akka.stream.Supervision.resumingDecider
import akka.stream.testkit.Utils._
import akka.stream.testkit._
import org.reactivestreams.Publisher
import scala.concurrent.duration._
import akka.stream.StreamSubscriptionTimeoutSettings
import akka.stream.StreamSubscriptionTimeoutTerminationMode
class FlowSplitWhenSpec extends AkkaSpec {
import FlowSplitAfterSpec._
val settings = ActorMaterializerSettings(system)
.withInputBuffer(initialSize = 2, maxSize = 2)
.withSubscriptionTimeoutSettings(StreamSubscriptionTimeoutSettings(StreamSubscriptionTimeoutTerminationMode.cancel, 1.second))
implicit val materializer = ActorMaterializer(settings)
@ -35,7 +38,7 @@ class FlowSplitWhenSpec extends AkkaSpec {
class SubstreamsSupport(splitWhen: Int = 3, elementCount: Int = 6) {
val source = Source(1 to elementCount)
val groupStream = source.splitWhen(_ == splitWhen).runWith(Sink.publisher(false))
val groupStream = source.splitWhen(_ == splitWhen).lift.runWith(Sink.publisher(false))
val masterSubscriber = TestSubscriber.manualProbe[Source[Int, _]]()
groupStream.subscribe(masterSubscriber)
@ -77,6 +80,7 @@ class FlowSplitWhenSpec extends AkkaSpec {
s2.request(1)
s2.expectComplete()
masterSubscription.request(1)
masterSubscriber.expectComplete()
}
}
@ -84,7 +88,6 @@ class FlowSplitWhenSpec extends AkkaSpec {
"work when first element is split-by" in assertAllStagesStopped {
new SubstreamsSupport(1, elementCount = 3) {
val s1 = StreamPuppet(getSubFlow().runWith(Sink.publisher(false)))
masterSubscriber.expectNoMsg(100.millis)
s1.request(5)
s1.expectNext(1)
@ -92,6 +95,7 @@ class FlowSplitWhenSpec extends AkkaSpec {
s1.expectNext(3)
s1.expectComplete()
masterSubscription.request(1)
masterSubscriber.expectComplete()
}
}
@ -123,6 +127,7 @@ class FlowSplitWhenSpec extends AkkaSpec {
Source(inputs)
.splitWhen(_ == 2)
.lift
.map(_.runWith(Sink(substream)))
.runWith(Sink(masterStream))
@ -139,6 +144,7 @@ class FlowSplitWhenSpec extends AkkaSpec {
val inputs2 = TestPublisher.probe[Int]()
Source(inputs2)
.splitWhen(_ == 2)
.lift
.map(_.runWith(Sink.cancelled))
.runWith(Sink.cancelled)
@ -151,6 +157,7 @@ class FlowSplitWhenSpec extends AkkaSpec {
Source(inputs3)
.splitWhen(_ == 2)
.lift
.runWith(Sink(masterStream3))
masterStream3.request(1)
@ -201,6 +208,7 @@ class FlowSplitWhenSpec extends AkkaSpec {
val exc = TE("test")
val publisher = Source(publisherProbeProbe)
.splitWhen(elem if (elem == 3) throw exc else elem % 3 == 0)
.lift
.runWith(Sink.publisher(false))
val subscriber = TestSubscriber.manualProbe[Source[Int, Unit]]()
publisher.subscribe(subscriber)
@ -233,6 +241,7 @@ class FlowSplitWhenSpec extends AkkaSpec {
val exc = TE("test")
val publisher = Source(publisherProbeProbe)
.splitWhen(elem if (elem == 3) throw exc else elem % 3 == 0)
.lift
.withAttributes(ActorAttributes.supervisionStrategy(resumingDecider))
.runWith(Sink.publisher(false))
val subscriber = TestSubscriber.manualProbe[Source[Int, Unit]]()
@ -277,7 +286,7 @@ class FlowSplitWhenSpec extends AkkaSpec {
val up = TestPublisher.manualProbe[Int]()
val down = TestSubscriber.manualProbe[Source[Int, Unit]]()
val flowSubscriber = Source.subscriber[Int].splitWhen(_ % 3 == 0).to(Sink(down)).run()
val flowSubscriber = Source.subscriber[Int].splitWhen(_ % 3 == 0).lift.to(Sink(down)).run()
val downstream = down.expectSubscription()
downstream.cancel()

View file

@ -14,6 +14,7 @@ import scala.concurrent.Await
import scala.concurrent.duration._
class SubstreamSubscriptionTimeoutSpec(conf: String) extends AkkaSpec(conf) {
import FlowGroupBySpec._
def this(subscriptionTimeout: FiniteDuration) {
this(
@ -41,7 +42,7 @@ class SubstreamSubscriptionTimeoutSpec(conf: String) extends AkkaSpec(conf) {
"timeout and cancel substream publishers when no-one subscribes to them after some time (time them out)" in assertAllStagesStopped {
val publisherProbe = TestPublisher.manualProbe[Int]()
val publisher = Source(publisherProbe).groupBy(_ % 3).runWith(Sink.publisher(false))
val publisher = Source(publisherProbe).groupBy(3, _ % 3).lift(_ % 3).runWith(Sink.publisher(false))
val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, _])]()
publisher.subscribe(subscriber)
@ -83,7 +84,7 @@ class SubstreamSubscriptionTimeoutSpec(conf: String) extends AkkaSpec(conf) {
"timeout and stop groupBy parent actor if none of the substreams are actually consumed" in assertAllStagesStopped {
val publisherProbe = TestPublisher.manualProbe[Int]()
val publisher = Source(publisherProbe).groupBy(_ % 2).runWith(Sink.publisher(false))
val publisher = Source(publisherProbe).groupBy(2, _ % 2).lift(_ % 2).runWith(Sink.publisher(false))
val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, _])]()
publisher.subscribe(subscriber)
@ -99,16 +100,11 @@ class SubstreamSubscriptionTimeoutSpec(conf: String) extends AkkaSpec(conf) {
val (_, s1) = subscriber.expectNext()
val (_, s2) = subscriber.expectNext()
val groupByActor = watchGroupByActor(4) // update this number based on how many streams the test above has...
// it should be terminated after none of it's substreams are used within the timeout
expectTerminated(groupByActor, 1000.millis)
}
"not timeout and cancel substream publishers when they have been subscribed to" in {
val publisherProbe = TestPublisher.manualProbe[Int]()
val publisher = Source(publisherProbe).groupBy(_ % 2).runWith(Sink.publisher(false))
val publisher = Source(publisherProbe).groupBy(2, _ % 2).lift(_ % 2).runWith(Sink.publisher(false))
val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, _])]()
publisher.subscribe(subscriber)
@ -147,21 +143,4 @@ class SubstreamSubscriptionTimeoutSpec(conf: String) extends AkkaSpec(conf) {
}
}
private def watchGroupByActor(flowNr: Int): ActorRef = {
implicit val t = Timeout(300.millis)
import akka.pattern.ask
val path = (materializer.supervisor.path / s"flow-${flowNr}-1-groupBy").toStringWithoutAddress
val gropByPath = system.actorSelection(path)
val groupByActor = try {
Await.result((gropByPath ? Identify("")).mapTo[ActorIdentity], 300.millis).ref.get
} catch {
case ex: Exception
alert(s"Unable to find groupBy actor by path: [$path], please adjust it's flowId, here's the current actor tree:\n" +
system.asInstanceOf[ExtendedActorSystem].printTree)
throw ex
}
watch(groupByActor)
}
}

View file

@ -324,6 +324,9 @@ final class ActorMaterializerSettings(
})
}
def withSubscriptionTimeoutSettings(sub: StreamSubscriptionTimeoutSettings): ActorMaterializerSettings =
copy(subscriptionTimeoutSettings = sub)
def withFuzzing(enable: Boolean): ActorMaterializerSettings = {
if (enable == this.fuzzingMode) this
else copy(fuzzingMode = enable)
@ -379,4 +382,3 @@ object StreamSubscriptionTimeoutTerminationMode {
def cancel = CancelTermination
}

View file

@ -259,10 +259,10 @@ private[akka] object ActorProcessorFactory {
// Also, otherwise the attributes will not affect the settings properly!
val settings = materializer.effectiveSettings(att)
op match {
case GroupBy(f, _) (GroupByProcessorImpl.props(settings, f), ())
case PrefixAndTail(n, _) (PrefixAndTailImpl.props(settings, n), ())
case Split(d, _) (SplitWhereProcessorImpl.props(settings, d), ())
case DirectProcessor(p, m) throw new AssertionError("DirectProcessor cannot end up in ActorProcessorFactory")
case GroupBy(maxSubstreams, f, _) (GroupByProcessorImpl.props(settings, maxSubstreams, f), ())
case PrefixAndTail(n, _) (PrefixAndTailImpl.props(settings, n), ())
case Split(d, _) (SplitWhereProcessorImpl.props(settings, d), ())
case DirectProcessor(p, m) throw new AssertionError("DirectProcessor cannot end up in ActorProcessorFactory")
}
}

View file

@ -15,11 +15,9 @@ private[akka] object ConstantFun {
def apply(p1: AnyRef, p2: AnyRef): AnyRef JPair AnyRef = JPair(p1, p2)
}
private[this] val ScalaIdentityFunction = (a: Any) a
def javaCreatePairFunction[A, B]: JFun2[A, B, JPair[A, B]] = JavaPairFunction.asInstanceOf[JFun2[A, B, JPair[A, B]]]
def javaIdentityFunction[T]: JFun[T, T] = JavaIdentityFunction.asInstanceOf[JFun[T, T]]
def scalaIdentityFunction[T]: T T = ScalaIdentityFunction.asInstanceOf[T T]
def scalaIdentityFunction[T]: T T = conforms
}

View file

@ -13,8 +13,8 @@ import akka.stream.scaladsl.Source
* INTERNAL API
*/
private[akka] object GroupByProcessorImpl {
def props(settings: ActorMaterializerSettings, keyFor: Any Any): Props =
Props(new GroupByProcessorImpl(settings, keyFor)).withDeploy(Deploy.local)
def props(settings: ActorMaterializerSettings, maxSubstreams: Int, keyFor: Any Any): Props =
Props(new GroupByProcessorImpl(settings, maxSubstreams, keyFor)).withDeploy(Deploy.local)
private case object Drop
}
@ -22,7 +22,7 @@ private[akka] object GroupByProcessorImpl {
/**
* INTERNAL API
*/
private[akka] class GroupByProcessorImpl(settings: ActorMaterializerSettings, val keyFor: Any Any)
private[akka] class GroupByProcessorImpl(settings: ActorMaterializerSettings, val maxSubstreams: Int, val keyFor: Any Any)
extends MultiStreamOutputProcessor(settings) {
import MultiStreamOutputProcessor._
@ -69,9 +69,11 @@ private[akka] class GroupByProcessorImpl(settings: ActorMaterializerSettings, va
// Just drop, we do not open any more substreams
nextPhase(waitNext)
} else {
if (keyToSubstreamOutput.size == maxSubstreams)
throw new IllegalStateException(s"cannot open substream for key '$key': too many substreams open")
val substreamOutput = createSubstreamOutput()
val substreamFlow = Source(substreamOutput) // substreamOutput is a Publisher
primaryOutputs.enqueueOutputElement((key, substreamFlow))
primaryOutputs.enqueueOutputElement(substreamFlow)
keyToSubstreamOutput(key) = substreamOutput
nextPhase(dispatchToSubstream(elem, substreamOutput))
}

View file

@ -133,7 +133,7 @@ private[stream] object Stages {
def apply[T]: SymbolicStage[T, T] = this.asInstanceOf[SymbolicStage[T, T]]
override def create(attr: Attributes): Stage[Any, Any] = fusing.Map(identity, supervision(attr))
override def create(attr: Attributes): Stage[Any, Any] = fusing.Map(conforms, supervision(attr))
}
final case class Map[In, Out](f: In Out, attributes: Attributes = map) extends SymbolicStage[In, Out] {
@ -215,7 +215,7 @@ private[stream] object Stages {
// FIXME: These are not yet proper stages, therefore they use the deprecated StageModule infrastructure
final case class GroupBy(f: Any Any, attributes: Attributes = groupBy) extends StageModule {
final case class GroupBy(maxSubstreams: Int, f: Any Any, attributes: Attributes = groupBy) extends StageModule {
override def withAttributes(attributes: Attributes) = copy(attributes = attributes)
}

View file

@ -0,0 +1,37 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.stream.impl
import akka.stream._
import akka.stream.scaladsl._
import language.higherKinds
object SubFlowImpl {
trait MergeBack[In, F[+_]] {
def apply[T](f: Flow[In, T, Unit], breadth: Int): F[T]
}
}
class SubFlowImpl[In, Out, Mat, F[+_], C](val subFlow: Flow[In, Out, Unit],
mergeBackFunction: SubFlowImpl.MergeBack[In, F],
finishFunction: Sink[In, Unit] C)
extends SubFlow[Out, Mat, F, C] {
override def deprecatedAndThen[U](op: Stages.StageModule): SubFlow[U, Mat, F, C] =
new SubFlowImpl[In, U, Mat, F, C](subFlow.deprecatedAndThen(op), mergeBackFunction, finishFunction)
override def via[T, Mat2](flow: Graph[FlowShape[Out, T], Mat2]): Repr[T] =
new SubFlowImpl[In, T, Mat, F, C](subFlow.via(flow), mergeBackFunction, finishFunction)
override def withAttributes(attr: Attributes): SubFlow[Out, Mat, F, C] =
new SubFlowImpl[In, Out, Mat, F, C](subFlow.withAttributes(attr), mergeBackFunction, finishFunction)
override def mergeSubstreams: F[Out] = mergeBackFunction(subFlow, Int.MaxValue)
override def concatSubstreams: F[Out] = mergeBackFunction(subFlow, 1)
def to[M](sink: Graph[SinkShape[Out], M]): C = finishFunction(subFlow.to(sink))
}

View file

@ -15,8 +15,8 @@ import java.{ util ⇒ ju }
import scala.concurrent._
import akka.dispatch.ExecutionContexts.sameThreadExecutionContext
final class FlattenMerge[T, M](breadth: Int) extends GraphStage[FlowShape[Source[T, M], T]] {
private val in = Inlet[Source[T, M]]("flatten.in")
final class FlattenMerge[T, M](breadth: Int) extends GraphStage[FlowShape[Graph[SourceShape[T], M], T]] {
private val in = Inlet[Graph[SourceShape[T], M]]("flatten.in")
private val out = Outlet[T]("flatten.out")
override val shape = FlowShape(in, out)
@ -96,25 +96,26 @@ final class FlattenMerge[T, M](breadth: Int) extends GraphStage[FlowShape[Source
override def onPull(): Unit = if (q.hasData && isAvailable(out)) pushOut()
}
def addSource(source: Source[T, M]): Unit = {
def addSource(source: Graph[SourceShape[T], M]): Unit = {
val localSource = new LocalSource[T]()
sources += localSource
val subF = source.runWith(new LocalSink(getAsyncCallback[ActorSubscriberMessage] {
case OnNext(elem)
val elemT = elem.asInstanceOf[T]
if (isAvailable(out)) {
push(out, elemT)
localSource.pull()
} else {
localSource.elem = elemT
q.enqueue(localSource)
}
case OnComplete
localSource.deactivate()
if (localSource.elem == null) removeSource(localSource)
case OnError(ex)
failStage(ex)
}.invoke))(interpreter.materializer)
val subF = Source.fromGraph(source)
.runWith(new LocalSink(getAsyncCallback[ActorSubscriberMessage] {
case OnNext(elem)
val elemT = elem.asInstanceOf[T]
if (isAvailable(out)) {
push(out, elemT)
localSource.pull()
} else {
localSource.elem = elemT
q.enqueue(localSource)
}
case OnComplete
localSource.deactivate()
if (localSource.elem == null) removeSource(localSource)
case OnError(ex)
failStage(ex)
}.invoke))(interpreter.materializer)
localSource.activate(subF)
}

View file

@ -10,11 +10,11 @@ import akka.stream.impl.{ ConstantFun, StreamLayout }
import akka.stream.{ scaladsl, _ }
import akka.stream.stage.Stage
import org.reactivestreams.Processor
import scala.annotation.unchecked.uncheckedVariance
import scala.collection.immutable
import scala.concurrent.Future
import scala.concurrent.duration.FiniteDuration
import akka.japi.Util
object Flow {
@ -78,7 +78,6 @@ final class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends
/**
* Transform this [[Flow]] by appending the given processing steps.
*
* {{{
* +----------------------------+
* | Resulting Flow |
@ -90,10 +89,9 @@ final class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends
* | +------+ +------+ |
* +----------------------------+
* }}}
*
* The materialized value of the combined [[Flow]] will be the materialized
* value of the current flow (ignoring the other Flows value), use
* [[Flow#viaMat viaMat]] if a different strategy is needed.
* `viaMat` if a different strategy is needed.
*/
def via[T, M](flow: Graph[FlowShape[Out, T], M]): javadsl.Flow[In, T, Mat] =
new Flow(delegate.via(flow))
@ -119,24 +117,70 @@ final class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends
/**
* Connect this [[Flow]] to a [[Sink]], concatenating the processing steps of both.
* {{{
* +----------------------------+
* | Resulting Sink |
* | |
* | +------+ +------+ |
* | | | | | |
* In ~~> | flow | ~Out~> | sink | |
* | | | | | |
* | +------+ +------+ |
* +----------------------------+
* }}}
* The materialized value of the combined [[Sink]] will be the materialized
* value of the current flow (ignoring the given Sinks value), use
* `toMat` if a different strategy is needed.
*/
def to(sink: Graph[SinkShape[Out], _]): javadsl.Sink[In, Mat] =
new Sink(delegate.to(sink))
/**
* Connect this [[Flow]] to a [[Sink]], concatenating the processing steps of both.
* {{{
* +----------------------------+
* | Resulting Sink |
* | |
* | +------+ +------+ |
* | | | | | |
* In ~~> | flow | ~Out~> | sink | |
* | | | | | |
* | +------+ +------+ |
* +----------------------------+
* }}}
* The `combine` function is used to compose the materialized values of this flow and that
* Sink into the materialized value of the resulting Sink.
*/
def toMat[M, M2](sink: Graph[SinkShape[Out], M], combine: function.Function2[Mat, M, M2]): javadsl.Sink[In, M2] =
new Sink(delegate.toMat(sink)(combinerToScala(combine)))
/**
* Join this [[Flow]] to another [[Flow]], by cross connecting the inputs and outputs, creating a [[RunnableGraph]]
* Join this [[Flow]] to another [[Flow]], by cross connecting the inputs and outputs, creating a [[RunnableGraph]].
* {{{
* +------+ +-------+
* | | ~Out~> | |
* | this | | other |
* | | <~In~ | |
* +------+ +-------+
* }}}
* The materialized value of the combined [[Flow]] will be the materialized
* value of the current flow (ignoring the other Flows value), use
* `joinMat` if a different strategy is needed.
*/
def join[M](flow: Graph[FlowShape[Out, In], M]): javadsl.RunnableGraph[Mat] =
RunnableGraph.fromGraph(delegate.join(flow))
/**
* Join this [[Flow]] to another [[Flow]], by cross connecting the inputs and outputs, creating a [[RunnableGraph]]
* {{{
* +------+ +-------+
* | | ~Out~> | |
* | this | | other |
* | | <~In~ | |
* +------+ +-------+
* }}}
* The `combine` function is used to compose the materialized values of this flow and that
* Flow into the materialized value of the resulting Flow.
*/
def joinMat[M, M2](flow: Graph[FlowShape[Out, In], M], combine: function.Function2[Mat, M, M2]): javadsl.RunnableGraph[M2] =
RunnableGraph.fromGraph(delegate.joinMat(flow)(combinerToScala(combine)))
@ -231,13 +275,7 @@ final class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends
* '''Cancels when''' downstream cancels
*/
def mapConcat[T](f: function.Function[Out, java.lang.Iterable[T]]): javadsl.Flow[In, T, Mat] =
new Flow(delegate.mapConcat { elem
val scalaIterable = new immutable.Iterable[T] {
import collection.JavaConverters._
override def iterator: Iterator[T] = f(elem).iterator().asScala
}
scalaIterable
})
new Flow(delegate.mapConcat { elem Util.immutableSeq(f(elem)) })
/**
* Transform this stream by applying the given function to each of the elements
@ -577,7 +615,6 @@ final class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends
* '''Completes when''' upstream completes or upstream failed with exception pf can handle
*
* '''Cancels when''' downstream cancels
*
*/
def recover[T >: Out](pf: PartialFunction[Throwable, T]): javadsl.Flow[In, T, Mat] =
new Flow(delegate.recover(pf))
@ -706,7 +743,7 @@ final class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends
new Flow(delegate.transform(() mkStage.create()))
/**
* Takes up to `n` elements from the stream (less than `n` only if the upstream completes before emitting `n` elements)
* Takes up to `n` elements from the stream (less than `n` if the upstream completes before emitting `n` elements)
* and returns a pair containing a strict sequence of the taken element
* and a stream representing the remaining elements. If ''n'' is zero or negative, then this will return a pair
* of an empty collection and a stream containing the whole upstream unchanged.
@ -733,12 +770,20 @@ final class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends
* This operation demultiplexes the incoming stream into separate output
* streams, one for each element key. The key is computed for each element
* using the given function. When a new key is encountered for the first time
* it is emitted to the downstream subscriber together with a fresh
* flow that will eventually produce all the elements of the substream
* for that key. Not consuming the elements from the created streams will
* stop this processor from processing more elements, therefore you must take
* care to unblock (or cancel) all of the produced streams even if you want
* to consume only one of them.
* a new substream is opened and subsequently fed with all elements belonging to
* that key.
*
* The object returned from this method is not a normal [[Flow]],
* it is a [[SubFlow]]. This means that after this combinator all transformations
* are applied to all encountered substreams in the same fashion. Substream mode
* is exited either by closing the substream (i.e. connecting it to a [[Sink]])
* or by merging the substreams back together; see the `to` and `mergeBack` methods
* on [[SubFlow]] for more information.
*
* It is important to note that the substreams also propagate back-pressure as
* any other stream, which means that blocking one substream will block the `groupBy`
* operator itselfand thereby all substreamsonce all internal or
* explicit buffers are filled.
*
* If the group by function `f` throws an exception and the supervision decision
* is [[akka.stream.Supervision#stop]] the stream and substreams will be completed
@ -757,9 +802,11 @@ final class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends
*
* '''Cancels when''' downstream cancels and all substreams cancel
*
* @param maxSubstreams configures the maximum number of substreams (keys)
* that are supported; if more distinct keys are encountered then the stream fails
*/
def groupBy[K](f: function.Function[Out, K]): javadsl.Flow[In, akka.japi.Pair[K, javadsl.Source[Out @uncheckedVariance, Unit]], Mat] =
new Flow(delegate.groupBy(f.apply).map { case (k, p) akka.japi.Pair(k, p.asJava) }) // TODO optimize to one step
def groupBy[K](maxSubstreams: Int, f: function.Function[Out, K]): SubFlow[In, Out @uncheckedVariance, Mat] =
new SubFlow(delegate.groupBy(maxSubstreams, f.apply))
/**
* This operation applies the given predicate to all incoming elements and
@ -782,6 +829,18 @@ final class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends
* true, false // subsequent substreams operate the same way
* }}}
*
* The object returned from this method is not a normal [[Flow]],
* it is a [[SubFlow]]. This means that after this combinator all transformations
* are applied to all encountered substreams in the same fashion. Substream mode
* is exited either by closing the substream (i.e. connecting it to a [[Sink]])
* or by merging the substreams back together; see the `to` and `mergeBack` methods
* on [[SubFlow]] for more information.
*
* It is important to note that the substreams also propagate back-pressure as
* any other stream, which means that blocking one substream will block the `splitWhen`
* operator itselfand thereby all substreamsonce all internal or
* explicit buffers are filled.
*
* If the split predicate `p` throws an exception and the supervision decision
* is [[akka.stream.Supervision#stop]] the stream and substreams will be completed
* with failure.
@ -802,8 +861,8 @@ final class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends
*
* See also [[Flow.splitAfter]].
*/
def splitWhen(p: function.Predicate[Out]): javadsl.Flow[In, Source[Out, Unit], Mat] =
new Flow(delegate.splitWhen(p.test).map(_.asJava))
def splitWhen(p: function.Predicate[Out]): SubFlow[In, Out, Mat] =
new SubFlow(delegate.splitWhen(p.test))
/**
* This operation applies the given predicate to all incoming elements and
@ -817,6 +876,18 @@ final class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends
* false, false, true // elements go into third substream
* }}}
*
* The object returned from this method is not a normal [[Flow]],
* it is a [[SubFlow]]. This means that after this combinator all transformations
* are applied to all encountered substreams in the same fashion. Substream mode
* is exited either by closing the substream (i.e. connecting it to a [[Sink]])
* or by merging the substreams back together; see the `to` and `mergeBack` methods
* on [[SubFlow]] for more information.
*
* It is important to note that the substreams also propagate back-pressure as
* any other stream, which means that blocking one substream will block the `splitAfter`
* operator itselfand thereby all substreamsonce all internal or
* explicit buffers are filled.
*
* If the split predicate `p` throws an exception and the supervision decision
* is [[akka.stream.Supervision.Stop]] the stream and substreams will be completed
* with failure.
@ -837,8 +908,8 @@ final class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends
*
* See also [[Flow.splitWhen]].
*/
def splitAfter[U >: Out](p: function.Predicate[Out]): javadsl.Flow[In, Source[Out, Unit], Mat] =
new Flow(delegate.splitAfter(p.test).map(_.asJava))
def splitAfter[U >: Out](p: function.Predicate[Out]): SubFlow[In, Out, Mat] =
new SubFlow(delegate.splitAfter(p.test))
/**
* Transform each input element into a `Source` of output elements that is
@ -853,8 +924,8 @@ final class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends
*
* '''Cancels when''' downstream cancels
*/
def flatMapConcat[T, M](f: function.Function[Out, Source[T, M]]): Flow[In, T, Mat] =
new Flow(delegate.flatMapConcat[T, M](x f(x).asScala))
def flatMapConcat[T, M](f: function.Function[Out, _ <: Graph[SourceShape[T], M]]): Flow[In, T, Mat] =
new Flow(delegate.flatMapConcat[T, M](x f(x)))
/**
* Transform each input element into a `Source` of output elements that is
@ -869,8 +940,8 @@ final class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends
*
* '''Cancels when''' downstream cancels
*/
def flatMapMerge[T, M](breadth: Int, f: function.Function[Out, Source[T, M]]): Flow[In, T, Mat] =
new Flow(delegate.flatMapMerge(breadth, o f(o).asScala))
def flatMapMerge[T, M](breadth: Int, f: function.Function[Out, _ <: Graph[SourceShape[T], M]]): Flow[In, T, Mat] =
new Flow(delegate.flatMapMerge(breadth, o f(o)))
/**
* Concatenate the given [[Source]] to this [[Flow]], meaning that once this
@ -1226,6 +1297,14 @@ final class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends
* This can be adjusted according to your needs by providing a custom [[Attributes.LogLevels]] attribute on the given Flow.
*
* Uses an internally created [[LoggingAdapter]] which uses `akka.stream.Log` as it's source (use this class to configure slf4j loggers).
*
* '''Emits when''' the mapping function returns an element
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def log(name: String): javadsl.Flow[In, Out, Mat] =
this.log(name, ConstantFun.javaIdentityFunction[Out], null)

View file

@ -206,7 +206,8 @@ object Source {
/**
* Combines several sources with fan-in strategy like `Merge` or `Concat` and returns `Source`.
*/
def combine[T, U](first: Source[T, _], second: Source[T, _], rest: java.util.List[Source[T, _]], strategy: function.Function[java.lang.Integer, Graph[UniformFanInShape[T, U], Unit]]): Source[U, Unit] = {
def combine[T, U](first: Source[T, _], second: Source[T, _], rest: java.util.List[Source[T, _]],
strategy: function.Function[java.lang.Integer, _ <: Graph[UniformFanInShape[T, U], Unit]]): Source[U, Unit] = {
import scala.collection.JavaConverters._
val seq = if (rest != null) rest.asScala.map(_.asScala) else Seq()
new Source(scaladsl.Source.combine(first.asScala, second.asScala, seq: _*)(num strategy.apply(num)))
@ -341,24 +342,78 @@ final class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Grap
/**
* Transform this [[Source]] by appending the given processing stages.
* {{{
* +----------------------------+
* | Resulting Source |
* | |
* | +------+ +------+ |
* | | | | | |
* | | this | ~Out~> | flow | ~~> T
* | | | | | |
* | +------+ +------+ |
* +----------------------------+
* }}}
* The materialized value of the combined [[Flow]] will be the materialized
* value of the current flow (ignoring the other Flows value), use
* `viaMat` if a different strategy is needed.
*/
def via[T, M](flow: Graph[FlowShape[Out, T], M]): javadsl.Source[T, Mat] =
new Source(delegate.via(flow))
/**
* Transform this [[Source]] by appending the given processing stages.
* {{{
* +----------------------------+
* | Resulting Source |
* | |
* | +------+ +------+ |
* | | | | | |
* | | this | ~Out~> | flow | ~~> T
* | | | | | |
* | +------+ +------+ |
* +----------------------------+
* }}}
* The `combine` function is used to compose the materialized values of this flow and that
* flow into the materialized value of the resulting Flow.
*/
def viaMat[T, M, M2](flow: Graph[FlowShape[Out, T], M], combine: function.Function2[Mat, M, M2]): javadsl.Source[T, M2] =
new Source(delegate.viaMat(flow)(combinerToScala(combine)))
/**
* Connect this [[Source]] to a [[Sink]], concatenating the processing steps of both.
* {{{
* +----------------------------+
* | Resulting RunnableGraph |
* | |
* | +------+ +------+ |
* | | | | | |
* | | this | ~Out~> | sink | |
* | | | | | |
* | +------+ +------+ |
* +----------------------------+
* }}}
* The materialized value of the combined [[Sink]] will be the materialized
* value of the current flow (ignoring the given Sinks value), use
* `toMat` if a different strategy is needed.
*/
def to[M](sink: Graph[SinkShape[Out], M]): javadsl.RunnableGraph[Mat] =
RunnableGraph.fromGraph(delegate.to(sink))
/**
* Connect this [[Source]] to a [[Sink]], concatenating the processing steps of both.
* {{{
* +----------------------------+
* | Resulting RunnableGraph |
* | |
* | +------+ +------+ |
* | | | | | |
* | | this | ~Out~> | sink | |
* | | | | | |
* | +------+ +------+ |
* +----------------------------+
* }}}
* The `combine` function is used to compose the materialized values of this flow and that
* Sink into the materialized value of the resulting Sink.
*/
def toMat[M, M2](sink: Graph[SinkShape[Out], M], combine: function.Function2[Mat, M, M2]): javadsl.RunnableGraph[M2] =
RunnableGraph.fromGraph(delegate.toMat(sink)(combinerToScala(combine)))
@ -533,6 +588,14 @@ final class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Grap
/**
* Transform this stream by applying the given function to each of the elements
* as they pass through this processing step.
*
* '''Emits when''' the mapping function returns an element
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def map[T](f: function.Function[Out, T]): javadsl.Source[T, Mat] =
new Source(delegate.map(f.apply))
@ -541,18 +604,40 @@ final class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Grap
* Recover allows to send last element on failure and gracefully complete the stream
* Since the underlying failure signal onError arrives out-of-band, it might jump over existing elements.
* This stage can recover the failure signal, but not the skipped elements, which will be dropped.
*
* '''Emits when''' element is available from the upstream or upstream is failed and pf returns an element
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes or upstream failed with exception pf can handle
*
* '''Cancels when''' downstream cancels
*/
def recover[T >: Out](pf: PartialFunction[Throwable, T]): javadsl.Source[T, Mat] =
new Source(delegate.recover(pf))
/**
* Transform each input element into a sequence of output elements that is
* Transform each input element into an `Iterable of output elements that is
* then flattened into the output stream.
*
* The returned list MUST NOT contain `null` values,
* Make sure that the `Iterable` is immutable or at least not modified after
* being used as an output sequence. Otherwise the stream may fail with
* `ConcurrentModificationException` or other more subtle errors may occur.
*
* The returned `Iterable` MUST NOT contain `null` values,
* as they are illegal as stream elements - according to the Reactive Streams specification.
*
* '''Emits when''' the mapping function returns an element or there are still remaining elements
* from the previously calculated collection
*
* '''Backpressures when''' downstream backpressures or there are still remaining elements from the
* previously calculated collection
*
* '''Completes when''' upstream completes and all remaining elements has been emitted
*
* '''Cancels when''' downstream cancels
*/
def mapConcat[T](f: function.Function[Out, java.util.List[T]]): javadsl.Source[T, Mat] =
def mapConcat[T](f: function.Function[Out, _ <: java.lang.Iterable[T]]): javadsl.Source[T, Mat] =
new Source(delegate.mapConcat(elem Util.immutableSeq(f.apply(elem))))
/**
@ -562,6 +647,25 @@ final class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Grap
* downstream may run in parallel and may complete in any order, but the elements that
* are emitted downstream are in the same order as received from upstream.
*
* If the function `f` throws an exception or if the `Future` is completed
* with failure and the supervision decision is [[akka.stream.Supervision#stop]]
* the stream will be completed with failure.
*
* If the function `f` throws an exception or if the `Future` is completed
* with failure and the supervision decision is [[akka.stream.Supervision#resume]] or
* [[akka.stream.Supervision#restart]] the element is dropped and the stream continues.
*
* The function `f` is always invoked on the elements in the order they arrive.
*
* '''Emits when''' the Future 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 or the first future is not completed
*
* '''Completes when''' upstream completes and all futures has been completed and all elements has been emitted
*
* '''Cancels when''' downstream cancels
*
* @see [[#mapAsyncUnordered]]
*/
def mapAsync[T](parallelism: Int, f: function.Function[Out, Future[T]]): javadsl.Source[T, Mat] =
@ -575,6 +679,25 @@ final class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Grap
* as soon as it is ready, i.e. it is possible that the elements are not emitted downstream
* in the same order as received from upstream.
*
* If the function `f` throws an exception or if the `Future` is completed
* with failure and the supervision decision is [[akka.stream.Supervision#stop]]
* the stream will be completed with failure.
*
* If the function `f` throws an exception or if the `Future` is completed
* with failure and the supervision decision is [[akka.stream.Supervision#resume]] or
* [[akka.stream.Supervision#restart]] the element is dropped and the stream continues.
*
* The function `f` is always invoked on the elements in the order they arrive (even though the result of the futures
* returned by `f` might be emitted in a different order).
*
* '''Emits when''' any of the Futures 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 has been completed and all elements has been emitted
*
* '''Cancels when''' downstream cancels
*
* @see [[#mapAsync]]
*/
def mapAsyncUnordered[T](parallelism: Int, f: function.Function[Out, Future[T]]): javadsl.Source[T, Mat] =
@ -582,12 +705,29 @@ final class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Grap
/**
* Only pass on those elements that satisfy the given predicate.
*
* '''Emits when''' the given predicate returns true for the element
*
* '''Backpressures when''' the given predicate returns true for the element and downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*
*/
def filter(p: function.Predicate[Out]): javadsl.Source[Out, Mat] =
new Source(delegate.filter(p.test))
/**
* Only pass on those elements that NOT satisfy the given predicate.
*
* '''Emits when''' the given predicate returns false for the element
*
* '''Backpressures when''' the given predicate returns false for the element and downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def filterNot(p: function.Predicate[Out]): javadsl.Source[Out, Mat] =
new Source(delegate.filterNot(p.test))
@ -596,6 +736,14 @@ final class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Grap
* Transform this stream by applying the given partial function to each of the elements
* on which the function is defined as they pass through this processing step.
* Non-matching elements are filtered out.
*
* '''Emits when''' the provided partial function is defined for the element
*
* '''Backpressures when''' the partial function is defined for the element and downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def collect[T](pf: PartialFunction[Out, T]): javadsl.Source[T, Mat] =
new Source(delegate.collect(pf))
@ -604,7 +752,15 @@ final class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Grap
* Chunk up this stream into groups of the given size, with the last group
* possibly smaller than requested due to end-of-stream.
*
* @param n must be positive, otherwise [[IllegalArgumentException]] is thrown.
* `n` must be positive, otherwise IllegalArgumentException is thrown.
*
* '''Emits when''' the specified number of elements has been accumulated or upstream completed
*
* '''Backpressures when''' a group has been assembled and downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def grouped(n: Int): javadsl.Source[java.util.List[Out @uncheckedVariance], Mat] =
new Source(delegate.grouped(n).map(_.asJava))
@ -613,8 +769,16 @@ final class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Grap
* Apply a sliding window over the stream and return the windows as groups of elements, with the last group
* possibly smaller than requested due to end-of-stream.
*
* @param n must be positive, otherwise [[IllegalArgumentException]] is thrown.
* @param step must be positive, otherwise [[IllegalArgumentException]] is thrown.
* `n` must be positive, otherwise IllegalArgumentException is thrown.
* `step` must be positive, otherwise IllegalArgumentException is thrown.
*
* '''Emits when''' enough elements have been collected within the window or upstream completed
*
* '''Backpressures when''' a window has been assembled and downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def sliding(n: Int, step: Int): javadsl.Source[java.util.List[Out @uncheckedVariance], Mat] =
new Source(delegate.sliding(n, step).map(_.asJava))
@ -623,16 +787,39 @@ final class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Grap
* Similar to `fold` but is not a terminal operation,
* emits its current value which starts at `zero` and then
* applies the current and next value to the given function `f`,
* yielding the next current value.
* emitting the next current value.
*
* If the function `f` throws an exception and the supervision decision is
* [[akka.stream.Supervision#restart]] current value starts at `zero` again
* the stream will continue.
*
* '''Emits when''' the function scanning the element returns a new element
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def scan[T](zero: T)(f: function.Function2[T, Out, T]): javadsl.Source[T, Mat] =
new Source(delegate.scan(zero)(f.apply))
/**
* Similar to `scan` but only emits the current value once, when completing.
* Its current value which starts at `zero` and then
* applies the current and next value to the given function `f`,
* Similar to `scan` but only emits its result when the upstream completes,
* after which it also completes. Applies the given function `f` towards its current and next value,
* yielding the next current value.
*
* If the function `f` throws an exception and the supervision decision is
* [[akka.stream.Supervision#restart]] current value starts at `zero` again
* the stream will continue.
*
* '''Emits when''' upstream completes
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def fold[T](zero: T)(f: function.Function2[T, Out, T]): javadsl.Source[T, Mat] =
new Source(delegate.fold(zero)(f.apply))
@ -702,7 +889,16 @@ final class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Grap
* The last group before end-of-stream will contain the buffered elements
* since the previously emitted group.
*
* @param n must be positive, and `d` must be greater than 0 seconds, otherwise [[IllegalArgumentException]] is thrown.
* '''Emits when''' the configured time elapses since the last group has been emitted
*
* '''Backpressures when''' the configured time elapses since the last group has been emitted
*
* '''Completes when''' upstream completes (emits last group)
*
* '''Cancels when''' downstream completes
*
* `n` must be positive, and `d` must be greater than 0 seconds, otherwise
* IllegalArgumentException is thrown.
*/
def groupedWithin(n: Int, d: FiniteDuration): javadsl.Source[java.util.List[Out @uncheckedVariance], Mat] =
new Source(delegate.groupedWithin(n, d).map(_.asJava)) // TODO optimize to one step
@ -710,23 +906,48 @@ final class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Grap
/**
* Discard the given number of elements at the beginning of the stream.
* No elements will be dropped if `n` is zero or negative.
*
* '''Emits when''' the specified number of elements has been dropped already
*
* '''Backpressures when''' the specified number of elements has been dropped and downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def drop(n: Long): javadsl.Source[Out, Mat] =
new Source(delegate.drop(n))
/**
* Discard the elements received within the given duration at beginning of the stream.
*
* '''Emits when''' the specified time elapsed and a new upstream element arrives
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def dropWithin(d: FiniteDuration): javadsl.Source[Out, Mat] =
new Source(delegate.dropWithin(d))
/**
* Terminate processing (and cancel the upstream publisher) after predicate returned false for the first time.
* Due to input buffering some elements may have been
* Terminate processing (and cancel the upstream publisher) after predicate
* returns false for the first time. Due to input buffering some elements may have been
* requested from upstream publishers that will then not be processed downstream
* of this step.
*
* @param p predicate is evaluated for each new element until first time returns false
* The stream will be completed without producing any elements if predicate is false for
* the first stream element.
*
* '''Emits when''' the predicate is true
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' predicate returned false or upstream completes
*
* '''Cancels when''' predicate returned false or downstream cancels
*/
def takeWhile(p: function.Predicate[Out]): javadsl.Source[Out, Mat] = new Source(delegate.takeWhile(p.test))
@ -734,6 +955,14 @@ final class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Grap
* Discard elements at the beginning of the stream while predicate is true.
* No elements will be dropped after predicate first time returned false.
*
* '''Emits when''' predicate returned false and for all following stream elements
*
* '''Backpressures when''' predicate returned false and downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*
* @param p predicate is evaluated for each new element until first time returns false
*/
def dropWhile(p: function.Predicate[Out]): javadsl.Source[Out, Mat] = new Source(delegate.dropWhile(p.test))
@ -744,7 +973,16 @@ final class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Grap
* requested from upstream publishers that will then not be processed downstream
* of this step.
*
* @param n if `n` is zero or negative the stream will be completed without producing any elements.
* The stream will be completed without producing any elements if `n` is zero
* or negative.
*
* '''Emits when''' the specified number of elements to take has not yet been reached
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' the defined number of elements has been taken or upstream completes
*
* '''Cancels when''' the defined number of elements has been taken or downstream cancels
*/
def take(n: Long): javadsl.Source[Out, Mat] =
new Source(delegate.take(n))
@ -757,6 +995,14 @@ final class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Grap
*
* Note that this can be combined with [[#take]] to limit the number of elements
* within the duration.
*
* '''Emits when''' an upstream element arrives
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes or timer fires
*
* '''Cancels when''' downstream cancels or timer fires
*/
def takeWithin(d: FiniteDuration): javadsl.Source[Out, Mat] =
new Source(delegate.takeWithin(d))
@ -769,6 +1015,14 @@ final class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Grap
* This element only rolls up elements if the upstream is faster, but if the downstream is faster it will not
* duplicate elements.
*
* '''Emits when''' downstream stops backpressuring and there is a conflated element available
*
* '''Backpressures when''' never
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*
* @param seed Provides the first state for a conflated value using the first unconsumed element as a start
* @param aggregate Takes the currently aggregated value and the current pending element to produce a new aggregate
*/
@ -784,6 +1038,17 @@ final class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Grap
* This means that if the upstream is actually faster than the upstream it will be backpressured by the downstream
* subscriber.
*
* Expand does not support [[akka.stream.Supervision#restart]] and [[akka.stream.Supervision#resume]].
* Exceptions from the `seed` or `extrapolate` functions will complete the stream with failure.
*
* '''Emits when''' downstream stops backpressuring
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*
* @param seed Provides the first state for extrapolation using the first unconsumed element
* @param extrapolate Takes the current extrapolation state to produce an output element and the next extrapolation
* state.
@ -799,6 +1064,17 @@ final class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Grap
* Depending on the defined [[akka.stream.OverflowStrategy]] it might drop elements or backpressure the upstream if
* there is no space available
*
* '''Emits when''' downstream stops backpressuring and there is a pending element in the buffer
*
* '''Backpressures when''' depending on OverflowStrategy
* * Backpressure - backpressures when buffer is full
* * DropHead, DropTail, DropBuffer - never backpressures
* * Fail - fails the stream if buffer gets full
*
* '''Completes when''' upstream completes and buffered elements has been drained
*
* '''Cancels when''' downstream cancels
*
* @param size The size of the buffer in element count
* @param overflowStrategy Strategy that is used when incoming elements cannot fit inside the buffer
*/
@ -814,7 +1090,7 @@ final class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Grap
new Source(delegate.transform(() mkStage.create()))
/**
* Takes up to `n` elements from the stream (less than `n` only if the upstream completes before emitting `n` elements)
* Takes up to `n` elements from the stream (less than `n` if the upstream completes before emitting `n` elements)
* and returns a pair containing a strict sequence of the taken element
* and a stream representing the remaining elements. If ''n'' is zero or negative, then this will return a pair
* of an empty collection and a stream containing the whole upstream unchanged.
@ -824,6 +1100,15 @@ final class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Grap
* has not yet been emitted
* - the tail substream signals the error after the prefix and tail has been emitted by the main stream
* (at that point the main stream has already completed)
*
* '''Emits when''' the configured number of prefix elements are available. Emits this prefix, and the rest
* as a substream
*
* '''Backpressures when''' downstream backpressures or substream backpressures
*
* '''Completes when''' prefix elements has been consumed and substream has been consumed
*
* '''Cancels when''' downstream cancels or substream cancels
*/
def prefixAndTail(n: Int): javadsl.Source[akka.japi.Pair[java.util.List[Out @uncheckedVariance], javadsl.Source[Out @uncheckedVariance, Unit]], Mat] =
new Source(delegate.prefixAndTail(n).map { case (taken, tail) akka.japi.Pair(taken.asJava, tail.asJava) })
@ -832,15 +1117,43 @@ final class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Grap
* This operation demultiplexes the incoming stream into separate output
* streams, one for each element key. The key is computed for each element
* using the given function. When a new key is encountered for the first time
* it is emitted to the downstream subscriber together with a fresh
* flow that will eventually produce all the elements of the substream
* for that key. Not consuming the elements from the created streams will
* stop this processor from processing more elements, therefore you must take
* care to unblock (or cancel) all of the produced streams even if you want
* to consume only one of them.
* a new substream is opened and subsequently fed with all elements belonging to
* that key.
*
* The object returned from this method is not a normal [[Flow]],
* it is a [[SubSource]]. This means that after this combinator all transformations
* are applied to all encountered substreams in the same fashion. Substream mode
* is exited either by closing the substream (i.e. connecting it to a [[Sink]])
* or by merging the substreams back together; see the `to` and `mergeBack` methods
* on [[SubSource]] for more information.
*
* It is important to note that the substreams also propagate back-pressure as
* any other stream, which means that blocking one substream will block the `groupBy`
* operator itselfand thereby all substreamsonce all internal or
* explicit buffers are filled.
*
* If the group by function `f` throws an exception and the supervision decision
* is [[akka.stream.Supervision#stop]] the stream and substreams will be completed
* with failure.
*
* If the group by function `f` throws an exception and the supervision decision
* is [[akka.stream.Supervision#resume]] or [[akka.stream.Supervision#restart]]
* the element is dropped and the stream and substreams continue.
*
* '''Emits when''' an element for which the grouping function returns a group that has not yet been created.
* Emits the new group
*
* '''Backpressures when''' there is an element pending for a group whose substream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels and all substreams cancel
*
* @param maxSubstreams configures the maximum number of substreams (keys)
* that are supported; if more distinct keys are encountered then the stream fails
*/
def groupBy[K](f: function.Function[Out, K]): javadsl.Source[akka.japi.Pair[K, javadsl.Source[Out @uncheckedVariance, Unit]], Mat] =
new Source(delegate.groupBy(f.apply).map { case (k, p) akka.japi.Pair(k, p.asJava) }) // TODO optimize to one step
def groupBy[K](maxSubstreams: Int, f: function.Function[Out, K]): SubSource[Out @uncheckedVariance, Mat] =
new SubSource(delegate.groupBy(maxSubstreams, f.apply))
/**
* This operation applies the given predicate to all incoming elements and
@ -863,6 +1176,18 @@ final class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Grap
* true, false // subsequent substreams operate the same way
* }}}
*
* The object returned from this method is not a normal [[Flow]],
* it is a [[SubSource]]. This means that after this combinator all transformations
* are applied to all encountered substreams in the same fashion. Substream mode
* is exited either by closing the substream (i.e. connecting it to a [[Sink]])
* or by merging the substreams back together; see the `to` and `mergeBack` methods
* on [[SubSource]] for more information.
*
* It is important to note that the substreams also propagate back-pressure as
* any other stream, which means that blocking one substream will block the `splitWhen`
* operator itselfand thereby all substreamsonce all internal or
* explicit buffers are filled.
*
* If the split predicate `p` throws an exception and the supervision decision
* is [[akka.stream.Supervision.Stop]] the stream and substreams will be completed
* with failure.
@ -881,8 +1206,8 @@ final class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Grap
*
* See also [[Source.splitAfter]].
*/
def splitWhen(p: function.Predicate[Out]): javadsl.Source[javadsl.Source[Out, Unit], Mat] =
new Source(delegate.splitWhen(p.test).map(_.asJava))
def splitWhen(p: function.Predicate[Out]): SubSource[Out, Mat] =
new SubSource(delegate.splitWhen(p.test))
/**
* This operation applies the given predicate to all incoming elements and
@ -896,6 +1221,18 @@ final class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Grap
* false, false, true // elements go into third substream
* }}}
*
* The object returned from this method is not a normal [[Flow]],
* it is a [[SubSource]]. This means that after this combinator all transformations
* are applied to all encountered substreams in the same fashion. Substream mode
* is exited either by closing the substream (i.e. connecting it to a [[Sink]])
* or by merging the substreams back together; see the `to` and `mergeBack` methods
* on [[SubSource]] for more information.
*
* It is important to note that the substreams also propagate back-pressure as
* any other stream, which means that blocking one substream will block the `splitAfter`
* operator itselfand thereby all substreamsonce all internal or
* explicit buffers are filled.
*
* If the split predicate `p` throws an exception and the supervision decision
* is [[akka.stream.Supervision.Stop]] the stream and substreams will be completed
* with failure.
@ -916,8 +1253,8 @@ final class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Grap
*
* See also [[Source.splitWhen]].
*/
def splitAfter[U >: Out](p: function.Predicate[Out]): javadsl.Source[Source[Out, Unit], Mat] =
new Source(delegate.splitAfter(p.test).map(_.asJava))
def splitAfter[U >: Out](p: function.Predicate[Out]): SubSource[Out, Mat] =
new SubSource(delegate.splitAfter(p.test))
/**
* Transform each input element into a `Source` of output elements that is
@ -932,8 +1269,8 @@ final class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Grap
*
* '''Cancels when''' downstream cancels
*/
def flatMapConcat[T, M](f: function.Function[Out, Source[T, M]]): Source[T, Mat] =
new Source(delegate.flatMapConcat[T, M](x f(x).asScala))
def flatMapConcat[T, M](f: function.Function[Out, _ <: Graph[SourceShape[T], M]]): Source[T, Mat] =
new Source(delegate.flatMapConcat[T, M](x f(x)))
/**
* Transform each input element into a `Source` of output elements that is
@ -948,8 +1285,8 @@ final class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Grap
*
* '''Cancels when''' downstream cancels
*/
def flatMapMerge[T, M](breadth: Int, f: function.Function[Out, Source[T, M]]): Source[T, Mat] =
new Source(delegate.flatMapMerge(breadth, o f(o).asScala))
def flatMapMerge[T, M](breadth: Int, f: function.Function[Out, _ <: Graph[SourceShape[T], M]]): Source[T, Mat] =
new Source(delegate.flatMapMerge(breadth, o f(o)))
/**
* If the first element has not passed through this stage before the provided timeout, the stream is failed

View file

@ -0,0 +1,954 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.stream.javadsl
import akka.event.LoggingAdapter
import akka.japi.function
import akka.stream._
import akka.stream.impl.Stages.StageModule
import akka.stream.impl.ConstantFun
import akka.stream.stage.Stage
import scala.collection.immutable
import scala.collection.JavaConverters._
import scala.annotation.unchecked.uncheckedVariance
import scala.concurrent.Future
import scala.concurrent.duration.FiniteDuration
import akka.japi.Util
/**
* A stream of streams sub-flow of data elements, e.g. produced by `groupBy`.
* SubFlows cannot contribute to the super-flows materialized value since they
* are materialized later, during the runtime of the flow graph processing.
*/
class SubFlow[-In, +Out, +Mat](delegate: scaladsl.SubFlow[Out, Mat, scaladsl.Flow[In, Out, Mat]#Repr, scaladsl.Sink[In, Mat]]) {
/** Converts this Flow to its Scala DSL counterpart */
def asScala: scaladsl.SubFlow[Out, Mat, scaladsl.Flow[In, Out, Mat]#Repr, scaladsl.Sink[In, Mat]] @uncheckedVariance = delegate
/**
* Flatten the sub-flows back into the super-flow by performing a merge.
*/
def mergeSubstreams(): Flow[In, Out, Mat] =
new Flow(delegate.mergeSubstreams)
/**
* Flatten the sub-flows back into the super-flow by concatenating them.
* This is usually a bad idea when combined with `groupBy` since it can
* easily lead to deadlockthe concatenation does not consume from the second
* substream until the first has finished and the `groupBy` stage will get
* back-pressure from the second stream.
*/
def concatSubstreams(): Flow[In, Out, Mat] =
new Flow(delegate.concatSubstreams)
/**
* Transform this [[Flow]] by appending the given processing steps.
*
* {{{
* +----------------------------+
* | Resulting Flow |
* | |
* | +------+ +------+ |
* | | | | | |
* In ~~> | this | ~Out~> | flow | ~~> T
* | | | | | |
* | +------+ +------+ |
* +----------------------------+
* }}}
*
* The materialized value of the combined [[Flow]] will be the materialized
* value of the current flow (ignoring the other Flows value), use
* [[Flow#viaMat viaMat]] if a different strategy is needed.
*/
def via[T, M](flow: Graph[FlowShape[Out, T], M]): SubFlow[In, T, Mat] =
new SubFlow(delegate.via(flow))
/**
* Connect this [[SubFlow]] to a [[Sink]], concatenating the processing steps of both.
* This means that all sub-flows that result from the previous sub-stream operator
* will be attached to the given sink.
* {{{
* +----------------------------+
* | Resulting Sink |
* | |
* | +------+ +------+ |
* | | | | | |
* In ~~> | flow | ~Out~> | sink | |
* | | | | | |
* | +------+ +------+ |
* +----------------------------+
* }}}
*/
def to(sink: Graph[SinkShape[Out], _]): Sink[In, Mat] =
new Sink(delegate.to(sink))
/**
* Transform this stream by applying the given function to each of the elements
* as they pass through this processing step.
*
* '''Emits when''' the mapping function returns an element
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def map[T](f: function.Function[Out, T]): SubFlow[In, T, Mat] =
new SubFlow(delegate.map(f.apply))
/**
* Transform each input element into an `Iterable` of output elements that is
* then flattened into the output stream.
*
* Make sure that the `Iterable` is immutable or at least not modified after
* being used as an output sequence. Otherwise the stream may fail with
* `ConcurrentModificationException` or other more subtle errors may occur.
*
* The returned `Iterable` MUST NOT contain `null` values,
* as they are illegal as stream elements - according to the Reactive Streams specification.
*
* '''Emits when''' the mapping function returns an element or there are still remaining elements
* from the previously calculated collection
*
* '''Backpressures when''' downstream backpressures or there are still remaining elements from the
* previously calculated collection
*
* '''Completes when''' upstream completes and all remaining elements has been emitted
*
* '''Cancels when''' downstream cancels
*/
def mapConcat[T](f: function.Function[Out, java.lang.Iterable[T]]): SubFlow[In, T, Mat] =
new SubFlow(delegate.mapConcat { elem Util.immutableSeq(f(elem)) })
/**
* Transform this stream by applying the given function to each of the elements
* as they pass through this processing step. The function returns a `Future` and the
* value of that future will be emitted downstreams. As many futures as requested elements by
* downstream may run in parallel and may complete in any order, but the elements that
* are emitted downstream are in the same order as received from upstream.
*
* If the function `f` throws an exception or if the `Future` is completed
* with failure and the supervision decision is [[akka.stream.Supervision#stop]]
* the stream will be completed with failure.
*
* If the function `f` throws an exception or if the `Future` is completed
* with failure and the supervision decision is [[akka.stream.Supervision#resume]] or
* [[akka.stream.Supervision#restart]] the element is dropped and the stream continues.
*
* The function `f` is always invoked on the elements in the order they arrive.
*
* '''Emits when''' the Future 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 or the first future is not completed
*
* '''Completes when''' upstream completes and all futures has been completed and all elements has been emitted
*
* '''Cancels when''' downstream cancels
*
* @see [[#mapAsyncUnordered]]
*/
def mapAsync[T](parallelism: Int, f: function.Function[Out, Future[T]]): SubFlow[In, T, Mat] =
new SubFlow(delegate.mapAsync(parallelism)(f.apply))
/**
* Transform this stream by applying the given function to each of the elements
* as they pass through this processing step. The function returns a `Future` and the
* value of that future will be emitted downstreams. As many futures as requested elements by
* downstream may run in parallel and each processed element will be emitted downstream
* as soon as it is ready, i.e. it is possible that the elements are not emitted downstream
* in the same order as received from upstream.
*
* If the function `f` throws an exception or if the `Future` is completed
* with failure and the supervision decision is [[akka.stream.Supervision#stop]]
* the stream will be completed with failure.
*
* If the function `f` throws an exception or if the `Future` is completed
* with failure and the supervision decision is [[akka.stream.Supervision#resume]] or
* [[akka.stream.Supervision#restart]] the element is dropped and the stream continues.
*
* The function `f` is always invoked on the elements in the order they arrive (even though the result of the futures
* returned by `f` might be emitted in a different order).
*
* '''Emits when''' any of the Futures 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 has been completed and all elements has been emitted
*
* '''Cancels when''' downstream cancels
*
* @see [[#mapAsync]]
*/
def mapAsyncUnordered[T](parallelism: Int, f: function.Function[Out, Future[T]]): SubFlow[In, T, Mat] =
new SubFlow(delegate.mapAsyncUnordered(parallelism)(f.apply))
/**
* Only pass on those elements that satisfy the given predicate.
*
* '''Emits when''' the given predicate returns true for the element
*
* '''Backpressures when''' the given predicate returns true for the element and downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*
*/
def filter(p: function.Predicate[Out]): SubFlow[In, Out, Mat] =
new SubFlow(delegate.filter(p.test))
/**
* Only pass on those elements that NOT satisfy the given predicate.
*
* '''Emits when''' the given predicate returns false for the element
*
* '''Backpressures when''' the given predicate returns false for the element and downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def filterNot(p: function.Predicate[Out]): SubFlow[In, Out, Mat] =
new SubFlow(delegate.filterNot(p.test))
/**
* Transform this stream by applying the given partial function to each of the elements
* on which the function is defined as they pass through this processing step.
* Non-matching elements are filtered out.
*
* '''Emits when''' the provided partial function is defined for the element
*
* '''Backpressures when''' the partial function is defined for the element and downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def collect[T](pf: PartialFunction[Out, T]): SubFlow[In, T, Mat] =
new SubFlow(delegate.collect(pf))
/**
* Chunk up this stream into groups of the given size, with the last group
* possibly smaller than requested due to end-of-stream.
*
* `n` must be positive, otherwise IllegalArgumentException is thrown.
*
* '''Emits when''' the specified number of elements has been accumulated or upstream completed
*
* '''Backpressures when''' a group has been assembled and downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def grouped(n: Int): SubFlow[In, java.util.List[Out @uncheckedVariance], Mat] =
new SubFlow(delegate.grouped(n).map(_.asJava)) // TODO optimize to one step
/**
* Apply a sliding window over the stream and return the windows as groups of elements, with the last group
* possibly smaller than requested due to end-of-stream.
*
* `n` must be positive, otherwise IllegalArgumentException is thrown.
* `step` must be positive, otherwise IllegalArgumentException is thrown.
*
* '''Emits when''' enough elements have been collected within the window or upstream completed
*
* '''Backpressures when''' a window has been assembled and downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def sliding(n: Int, step: Int = 1): SubFlow[In, java.util.List[Out @uncheckedVariance], Mat] =
new SubFlow(delegate.sliding(n, step).map(_.asJava)) // TODO optimize to one step
/**
* Similar to `fold` but is not a terminal operation,
* emits its current value which starts at `zero` and then
* applies the current and next value to the given function `f`,
* emitting the next current value.
*
* If the function `f` throws an exception and the supervision decision is
* [[akka.stream.Supervision#restart]] current value starts at `zero` again
* the stream will continue.
*
* '''Emits when''' the function scanning the element returns a new element
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def scan[T](zero: T)(f: function.Function2[T, Out, T]): SubFlow[In, T, Mat] =
new SubFlow(delegate.scan(zero)(f.apply))
/**
* Similar to `scan` but only emits its result when the upstream completes,
* after which it also completes. Applies the given function `f` towards its current and next value,
* yielding the next current value.
*
* If the function `f` throws an exception and the supervision decision is
* [[akka.stream.Supervision#restart]] current value starts at `zero` again
* the stream will continue.
*
* '''Emits when''' upstream completes
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def fold[T](zero: T)(f: function.Function2[T, Out, T]): SubFlow[In, T, Mat] =
new SubFlow(delegate.fold(zero)(f.apply))
/**
* Intersperses stream with provided element, similar to how [[scala.collection.immutable.List.mkString]]
* injects a separator between a List's elements.
*
* Additionally can inject start and end marker elements to stream.
*
* Examples:
*
* {{{
* Source<Integer, ?> nums = Source.from(Arrays.asList(0, 1, 2, 3));
* nums.intersperse(","); // 1 , 2 , 3
* nums.intersperse("[", ",", "]"); // [ 1 , 2 , 3 ]
* }}}
*
* In case you want to only prepend or only append an element (yet still use the `intercept` feature
* to inject a separator between elements, you may want to use the following pattern instead of the 3-argument
* version of intersperse (See [[Source.concat]] for semantics details):
*
* {{{
* Source.single(">> ").concat(flow.intersperse(","))
* flow.intersperse(",").concat(Source.single("END"))
* }}}
*
* '''Emits when''' upstream emits (or before with the `start` element if provided)
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def intersperse[T >: Out](start: T, inject: T, end: T): SubFlow[In, T, Mat] =
new SubFlow(delegate.intersperse(start, inject, end))
/**
* Intersperses stream with provided element, similar to how [[scala.collection.immutable.List.mkString]]
* injects a separator between a List's elements.
*
* Additionally can inject start and end marker elements to stream.
*
* Examples:
*
* {{{
* Source<Integer, ?> nums = Source.from(Arrays.asList(0, 1, 2, 3));
* nums.intersperse(","); // 1 , 2 , 3
* nums.intersperse("[", ",", "]"); // [ 1 , 2 , 3 ]
* }}}
*
* '''Emits when''' upstream emits (or before with the `start` element if provided)
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def intersperse[T >: Out](inject: T): SubFlow[In, T, Mat] =
new SubFlow(delegate.intersperse(inject))
/**
* Chunk up this stream into groups of elements received within a time window,
* or limited by the given number of elements, whatever happens first.
* Empty groups will not be emitted if no elements are received from upstream.
* The last group before end-of-stream will contain the buffered elements
* since the previously emitted group.
*
* '''Emits when''' the configured time elapses since the last group has been emitted
*
* '''Backpressures when''' the configured time elapses since the last group has been emitted
*
* '''Completes when''' upstream completes (emits last group)
*
* '''Cancels when''' downstream completes
*
* `n` must be positive, and `d` must be greater than 0 seconds, otherwise
* IllegalArgumentException is thrown.
*/
def groupedWithin(n: Int, d: FiniteDuration): SubFlow[In, java.util.List[Out @uncheckedVariance], Mat] =
new SubFlow(delegate.groupedWithin(n, d).map(_.asJava)) // TODO optimize to one step
/**
* Discard the given number of elements at the beginning of the stream.
* No elements will be dropped if `n` is zero or negative.
*
* '''Emits when''' the specified number of elements has been dropped already
*
* '''Backpressures when''' the specified number of elements has been dropped and downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def drop(n: Long): SubFlow[In, Out, Mat] =
new SubFlow(delegate.drop(n))
/**
* Discard the elements received within the given duration at beginning of the stream.
*
* '''Emits when''' the specified time elapsed and a new upstream element arrives
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def dropWithin(d: FiniteDuration): SubFlow[In, Out, Mat] =
new SubFlow(delegate.dropWithin(d))
/**
* Terminate processing (and cancel the upstream publisher) after predicate
* returns false for the first time. Due to input buffering some elements may have been
* requested from upstream publishers that will then not be processed downstream
* of this step.
*
* The stream will be completed without producing any elements if predicate is false for
* the first stream element.
*
* '''Emits when''' the predicate is true
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' predicate returned false or upstream completes
*
* '''Cancels when''' predicate returned false or downstream cancels
*/
def takeWhile(p: function.Predicate[Out]): SubFlow[In, Out, Mat] =
new SubFlow(delegate.takeWhile(p.test))
/**
* Discard elements at the beginning of the stream while predicate is true.
* All elements will be taken after predicate returns false first time.
*
* '''Emits when''' predicate returned false and for all following stream elements
*
* '''Backpressures when''' predicate returned false and downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def dropWhile(p: function.Predicate[Out]): SubFlow[In, Out, Mat] =
new SubFlow(delegate.dropWhile(p.test))
/**
* Recover allows to send last element on failure and gracefully complete the stream
* Since the underlying failure signal onError arrives out-of-band, it might jump over existing elements.
* This stage can recover the failure signal, but not the skipped elements, which will be dropped.
*
* '''Emits when''' element is available from the upstream or upstream is failed and pf returns an element
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes or upstream failed with exception pf can handle
*
* '''Cancels when''' downstream cancels
*
*/
def recover[T >: Out](pf: PartialFunction[Throwable, T]): SubFlow[In, T, Mat] =
new SubFlow(delegate.recover(pf))
/**
* Terminate processing (and cancel the upstream publisher) after the given
* number of elements. Due to input buffering some elements may have been
* requested from upstream publishers that will then not be processed downstream
* of this step.
*
* The stream will be completed without producing any elements if `n` is zero
* or negative.
*
* '''Emits when''' the specified number of elements to take has not yet been reached
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' the defined number of elements has been taken or upstream completes
*
* '''Cancels when''' the defined number of elements has been taken or downstream cancels
*/
def take(n: Long): SubFlow[In, Out, Mat] =
new SubFlow(delegate.take(n))
/**
* Terminate processing (and cancel the upstream publisher) after the given
* duration. Due to input buffering some elements may have been
* requested from upstream publishers that will then not be processed downstream
* of this step.
*
* Note that this can be combined with [[#take]] to limit the number of elements
* within the duration.
*
* '''Emits when''' an upstream element arrives
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes or timer fires
*
* '''Cancels when''' downstream cancels or timer fires
*/
def takeWithin(d: FiniteDuration): SubFlow[In, Out, Mat] =
new SubFlow(delegate.takeWithin(d))
/**
* Allows a faster upstream to progress independently of a slower subscriber by conflating elements into a summary
* until the subscriber is ready to accept them. For example a conflate step might average incoming numbers if the
* upstream publisher is faster.
*
* This element only rolls up elements if the upstream is faster, but if the downstream is faster it will not
* duplicate elements.
*
* '''Emits when''' downstream stops backpressuring and there is a conflated element available
*
* '''Backpressures when''' never
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*
* @param seed Provides the first state for a conflated value using the first unconsumed element as a start
* @param aggregate Takes the currently aggregated value and the current pending element to produce a new aggregate
*
*/
def conflate[S](seed: function.Function[Out, S], aggregate: function.Function2[S, Out, S]): SubFlow[In, S, Mat] =
new SubFlow(delegate.conflate(seed.apply)(aggregate.apply))
/**
* Allows a faster downstream to progress independently of a slower publisher by extrapolating elements from an older
* element until new element comes from the upstream. For example an expand step might repeat the last element for
* the subscriber until it receives an update from upstream.
*
* This element will never "drop" upstream elements as all elements go through at least one extrapolation step.
* This means that if the upstream is actually faster than the upstream it will be backpressured by the downstream
* subscriber.
*
* Expand does not support [[akka.stream.Supervision#restart]] and [[akka.stream.Supervision#resume]].
* Exceptions from the `seed` or `extrapolate` functions will complete the stream with failure.
*
* '''Emits when''' downstream stops backpressuring
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*
* @param seed Provides the first state for extrapolation using the first unconsumed element
* @param extrapolate Takes the current extrapolation state to produce an output element and the next extrapolation
* state.
*/
def expand[S, U](seed: function.Function[Out, S], extrapolate: function.Function[S, akka.japi.Pair[U, S]]): SubFlow[In, U, Mat] =
new SubFlow(delegate.expand(seed(_))(s {
val p = extrapolate(s)
(p.first, p.second)
}))
/**
* Adds a fixed size buffer in the flow that allows to store elements from a faster upstream until it becomes full.
* Depending on the defined [[akka.stream.OverflowStrategy]] it might drop elements or backpressure the upstream if
* there is no space available
*
* '''Emits when''' downstream stops backpressuring and there is a pending element in the buffer
*
* '''Backpressures when''' depending on OverflowStrategy
* * Backpressure - backpressures when buffer is full
* * DropHead, DropTail, DropBuffer - never backpressures
* * Fail - fails the stream if buffer gets full
*
* '''Completes when''' upstream completes and buffered elements has been drained
*
* '''Cancels when''' downstream cancels
*
* @param size The size of the buffer in element count
* @param overflowStrategy Strategy that is used when incoming elements cannot fit inside the buffer
*/
def buffer(size: Int, overflowStrategy: OverflowStrategy): SubFlow[In, Out, Mat] =
new SubFlow(delegate.buffer(size, overflowStrategy))
/**
* Generic transformation of a stream with a custom processing [[akka.stream.stage.Stage]].
* This operator makes it possible to extend the `Flow` API when there is no specialized
* operator that performs the transformation.
*/
def transform[U](mkStage: function.Creator[Stage[Out, U]]): SubFlow[In, U, Mat] =
new SubFlow(delegate.transform(() mkStage.create()))
/**
* Takes up to `n` elements from the stream (less than `n` only if the upstream completes before emitting `n` elements)
* and returns a pair containing a strict sequence of the taken element
* and a stream representing the remaining elements. If ''n'' is zero or negative, then this will return a pair
* of an empty collection and a stream containing the whole upstream unchanged.
*
* In case of an upstream error, depending on the current state
* - the master stream signals the error if less than `n` elements has been seen, and therefore the substream
* has not yet been emitted
* - the tail substream signals the error after the prefix and tail has been emitted by the main stream
* (at that point the main stream has already completed)
*
* '''Emits when''' the configured number of prefix elements are available. Emits this prefix, and the rest
* as a substream
*
* '''Backpressures when''' downstream backpressures or substream backpressures
*
* '''Completes when''' prefix elements has been consumed and substream has been consumed
*
* '''Cancels when''' downstream cancels or substream cancels
*/
def prefixAndTail(n: Int): SubFlow[In, akka.japi.Pair[java.util.List[Out @uncheckedVariance], javadsl.Source[Out @uncheckedVariance, Unit]], Mat] =
new SubFlow(delegate.prefixAndTail(n).map { case (taken, tail) akka.japi.Pair(taken.asJava, tail.asJava) })
/**
* Transform each input element into a `Source` of output elements that is
* then flattened into the output stream by concatenation,
* fully consuming one Source after the other.
*
* '''Emits when''' a currently consumed substream has an element available
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes and all consumed substreams complete
*
* '''Cancels when''' downstream cancels
*
*/
def flatMapConcat[T, M](f: function.Function[Out, _ <: Graph[SourceShape[T], M]]): SubFlow[In, T, Mat] =
new SubFlow(delegate.flatMapConcat(x f(x)))
/**
* Transform each input element into a `Source` of output elements that is
* then flattened into the output stream by merging, where at most `breadth`
* substreams are being consumed at any given time.
*
* '''Emits when''' a currently consumed substream has an element available
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes and all consumed substreams complete
*
* '''Cancels when''' downstream cancels
*/
def flatMapMerge[T, M](breadth: Int, f: function.Function[Out, _ <: Graph[SourceShape[T], M]]): SubFlow[In, T, Mat] =
new SubFlow(delegate.flatMapMerge(breadth, o f(o)))
/**
* Concatenate the given [[Source]] to this [[Flow]], meaning that once this
* Flows input is exhausted and all result elements have been generated,
* the Sources elements will be produced.
*
* Note that the [[Source]] is materialized together with this Flow and just kept
* from producing elements by asserting back-pressure until its time comes.
*
* If this [[Flow]] gets upstream error - no elements from the given [[Source]] will be pulled.
*
* '''Emits when''' element is available from current stream or from the given [[Source]] when current is completed
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' given [[Source]] completes
*
* '''Cancels when''' downstream cancels
*/
def concat[T >: Out, M](that: Graph[SourceShape[T], M]): SubFlow[In, T, Mat] =
new SubFlow(delegate.concat(that))
/**
* Attaches the given [[Sink]] to this [[Flow]], meaning that elements that passes
* through will also be sent to the [[Sink]].
*
* '''Emits when''' element is available and demand exists both from the Sink and the downstream.
*
* '''Backpressures when''' downstream or Sink backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def alsoTo(that: Graph[SinkShape[Out], _]): SubFlow[In, Out, Mat] =
new SubFlow(delegate.alsoTo(that))
/**
* Merge the given [[Source]] to this [[Flow]], taking elements as they arrive from input streams,
* picking randomly when several elements ready.
*
* '''Emits when''' one of the inputs has an element available
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' all upstreams complete
*
* '''Cancels when''' downstream cancels
*/
def merge[T >: Out](that: Graph[SourceShape[T], _]): SubFlow[In, T, Mat] =
new SubFlow(delegate.merge(that))
/**
* Combine the elements of current [[Flow]] and the given [[Source]] into a stream of tuples.
*
* '''Emits when''' all of the inputs has an element available
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' any upstream completes
*
* '''Cancels when''' downstream cancels
*/
def zip[T](source: Graph[SourceShape[T], _]): SubFlow[In, Out @uncheckedVariance Pair T, Mat] =
new SubFlow(delegate.zip(source))
/**
* Put together the elements of current [[Flow]] and the given [[Source]]
* into a stream of combined elements using a combiner function.
*
* '''Emits when''' all of the inputs has an element available
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' any upstream completes
*
* '''Cancels when''' downstream cancels
*/
def zipWith[Out2, Out3](that: Graph[SourceShape[Out2], _],
combine: function.Function2[Out, Out2, Out3]): SubFlow[In, Out3, Mat] =
new SubFlow(delegate.zipWith[Out2, Out3](that)(combinerToScala(combine)))
/**
* If the first element has not passed through this stage before the provided timeout, the stream is failed
* with a [[java.util.concurrent.TimeoutException]].
*
* '''Emits when''' upstream emits an element
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes or fails if timeout elapses before first element arrives
*
* '''Cancels when''' downstream cancels
*/
def initialTimeout(timeout: FiniteDuration): SubFlow[In, Out, Mat] =
new SubFlow(delegate.initialTimeout(timeout))
/**
* If the completion of the stream does not happen until the provided timeout, the stream is failed
* with a [[java.util.concurrent.TimeoutException]].
*
* '''Emits when''' upstream emits an element
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes or fails if timeout elapses before upstream completes
*
* '''Cancels when''' downstream cancels
*/
def completionTimeout(timeout: FiniteDuration): SubFlow[In, Out, Mat] =
new SubFlow(delegate.completionTimeout(timeout))
/**
* If the time between two processed elements exceed the provided timeout, the stream is failed
* with a [[java.util.concurrent.TimeoutException]].
*
* '''Emits when''' upstream emits an element
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes or fails if timeout elapses between two emitted elements
*
* '''Cancels when''' downstream cancels
*/
def idleTimeout(timeout: FiniteDuration): SubFlow[In, Out, Mat] =
new SubFlow(delegate.idleTimeout(timeout))
/**
* Injects additional elements if the upstream does not emit for a configured amount of time. In other words, this
* stage attempts to maintains a base rate of emitted elements towards the downstream.
*
* If the downstream backpressures then no element is injected until downstream demand arrives. Injected elements
* do not accumulate during this period.
*
* Upstream elements are always preferred over injected elements.
*
* '''Emits when''' upstream emits an element or if the upstream was idle for the configured period
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def keepAlive[U >: Out](maxIdle: FiniteDuration, injectedElem: function.Creator[U]): SubFlow[In, U, Mat] =
new SubFlow(delegate.keepAlive(maxIdle, () injectedElem.create()))
/**
* Sends elements downstream with speed limited to `elements/per`. In other words, this stage set the maximum rate
* for emitting messages. This combinator works for streams where all elements have the same cost or length.
*
* Throttle implements the token bucket model. There is a bucket with a given token capacity (burst size or maximumBurst).
* Tokens drops into the bucket at a given rate and can be `spared` for later use up to bucket capacity
* to allow some burstyness. Whenever stream wants to send an element, it takes as many
* tokens from the bucket as number of elements. If there isn't any, throttle waits until the
* bucket accumulates enough tokens.
*
* Parameter `mode` manages behaviour when upstream is faster than throttle rate:
* - [[akka.stream.ThrottleMode.Shaping]] makes pauses before emitting messages to meet throttle rate
* - [[akka.stream.ThrottleMode.Enforcing]] fails with exception when upstream is faster than throttle rate
*
* '''Emits when''' upstream emits an element and configured time per each element elapsed
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def throttle(elements: Int, per: FiniteDuration, maximumBurst: Int,
mode: ThrottleMode): javadsl.SubFlow[In, Out, Mat] =
new SubFlow(delegate.throttle(elements, per, maximumBurst, mode))
/**
* Sends elements downstream with speed limited to `cost/per`. Cost is
* calculating for each element individually by calling `calculateCost` function.
* This combinator works for streams when elements have different cost(length).
* Streams of `ByteString` for example.
*
* Throttle implements the token bucket model. There is a bucket with a given token capacity (burst size or maximumBurst).
* Tokens drops into the bucket at a given rate and can be `spared` for later use up to bucket capacity
* to allow some burstyness. Whenever stream wants to send an element, it takes as many
* tokens from the bucket as element cost. If there isn't any, throttle waits until the
* bucket accumulates enough tokens. Elements that costs more than the allowed burst will be delayed proportionally
* to their cost minus available tokens, meeting the target rate.
*
* Parameter `mode` manages behaviour when upstream is faster than throttle rate:
* - [[akka.stream.ThrottleMode.Shaping]] makes pauses before emitting messages to meet throttle rate
* - [[akka.stream.ThrottleMode.Enforcing]] fails with exception when upstream is faster than throttle rate. Enforcing
* cannot emit elements that cost more than the maximumBurst
*
* '''Emits when''' upstream emits an element and configured time per each element elapsed
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def throttle(cost: Int, per: FiniteDuration, maximumBurst: Int,
costCalculation: function.Function[Out, Integer], mode: ThrottleMode): javadsl.SubFlow[In, Out, Mat] =
new SubFlow(delegate.throttle(cost, per, maximumBurst, costCalculation.apply, mode))
/**
* Delays the initial element by the specified duration.
*
* '''Emits when''' upstream emits an element if the initial delay already elapsed
*
* '''Backpressures when''' downstream backpressures or initial delay not yet elapsed
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def initialDelay(delay: FiniteDuration): SubFlow[In, Out, Mat] =
new SubFlow(delegate.initialDelay(delay))
def withAttributes(attr: Attributes): SubFlow[In, Out, Mat] =
new SubFlow(delegate.withAttributes(attr))
def named(name: String): SubFlow[In, Out, Mat] =
new SubFlow(delegate.named(name))
/**
* Logs elements flowing through the stream as well as completion and erroring.
*
* By default element and completion signals are logged on debug level, and errors are logged on Error level.
* This can be adjusted according to your needs by providing a custom [[Attributes.LogLevels]] attribute on the given Flow:
*
* The `extract` function will be applied to each element before logging, so it is possible to log only those fields
* of a complex object flowing through this element.
*
* Uses the given [[LoggingAdapter]] for logging.
*
* '''Emits when''' the mapping function returns an element
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def log(name: String, extract: function.Function[Out, Any], log: LoggingAdapter): SubFlow[In, Out, Mat] =
new SubFlow(delegate.log(name, e extract.apply(e))(log))
/**
* Logs elements flowing through the stream as well as completion and erroring.
*
* By default element and completion signals are logged on debug level, and errors are logged on Error level.
* This can be adjusted according to your needs by providing a custom [[Attributes.LogLevels]] attribute on the given Flow:
*
* The `extract` function will be applied to each element before logging, so it is possible to log only those fields
* of a complex object flowing through this element.
*
* Uses an internally created [[LoggingAdapter]] which uses `akka.stream.Log` as it's source (use this class to configure slf4j loggers).
*
* '''Emits when''' the mapping function returns an element
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def log(name: String, extract: function.Function[Out, Any]): SubFlow[In, Out, Mat] =
this.log(name, extract, null)
/**
* Logs elements flowing through the stream as well as completion and erroring.
*
* By default element and completion signals are logged on debug level, and errors are logged on Error level.
* This can be adjusted according to your needs by providing a custom [[Attributes.LogLevels]] attribute on the given Flow:
*
* Uses the given [[LoggingAdapter]] for logging.
*
* '''Emits when''' the mapping function returns an element
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def log(name: String, log: LoggingAdapter): SubFlow[In, Out, Mat] =
this.log(name, ConstantFun.javaIdentityFunction[Out], log)
/**
* Logs elements flowing through the stream as well as completion and erroring.
*
* By default element and completion signals are logged on debug level, and errors are logged on Error level.
* This can be adjusted according to your needs by providing a custom [[Attributes.LogLevels]] attribute on the given Flow.
*
* Uses an internally created [[LoggingAdapter]] which uses `akka.stream.Log` as it's source (use this class to configure slf4j loggers).
*
* '''Emits when''' the mapping function returns an element
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def log(name: String): SubFlow[In, Out, Mat] =
this.log(name, ConstantFun.javaIdentityFunction[Out], null)
}

View file

@ -0,0 +1,952 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.stream.javadsl
import akka.event.LoggingAdapter
import akka.japi.function
import akka.stream._
import akka.stream.impl.Stages.StageModule
import akka.stream.impl.ConstantFun
import akka.stream.stage.Stage
import scala.collection.immutable
import scala.collection.JavaConverters._
import scala.annotation.unchecked.uncheckedVariance
import scala.concurrent.Future
import scala.concurrent.duration.FiniteDuration
import akka.japi.Util
/**
* A stream of streams sub-flow of data elements, e.g. produced by `groupBy`.
* SubFlows cannot contribute to the super-flows materialized value since they
* are materialized later, during the runtime of the flow graph processing.
*/
class SubSource[+Out, +Mat](delegate: scaladsl.SubFlow[Out, Mat, scaladsl.Source[Out, Mat]#Repr, scaladsl.RunnableGraph[Mat]]) {
/** Converts this Flow to its Scala DSL counterpart */
def asScala: scaladsl.SubFlow[Out, Mat, scaladsl.Source[Out, Mat]#Repr, scaladsl.RunnableGraph[Mat]] @uncheckedVariance = delegate
/**
* Flatten the sub-flows back into the super-flow by performing a merge.
*/
def mergeSubstreams(): Source[Out, Mat] =
new Source(delegate.mergeSubstreams)
/**
* Flatten the sub-flows back into the super-flow by concatenating them.
* This is usually a bad idea when combined with `groupBy` since it can
* easily lead to deadlockthe concatenation does not consume from the second
* substream until the first has finished and the `groupBy` stage will get
* back-pressure from the second stream.
*/
def concatSubstreams(): Source[Out, Mat] =
new Source(delegate.concatSubstreams)
/**
* Transform this [[Flow]] by appending the given processing steps.
* {{{
* +----------------------------+
* | Resulting Source |
* | |
* | +------+ +------+ |
* | | | | | |
* | | this | ~Out~> | flow | ~~> T
* | | | | | |
* | +------+ +------+ |
* +----------------------------+
* }}}
* The materialized value of the combined [[Flow]] will be the materialized
* value of the current flow (ignoring the other Flows value), use
* [[Flow#viaMat viaMat]] if a different strategy is needed.
*/
def via[T, M](flow: Graph[FlowShape[Out, T], M]): SubSource[T, Mat] =
new SubSource(delegate.via(flow))
/**
* Connect this [[SubSource]] to a [[Sink]], concatenating the processing steps of both.
* This means that all sub-flows that result from the previous sub-stream operator
* will be attached to the given sink.
* {{{
* +----------------------------+
* | Resulting RunnableGraph |
* | |
* | +------+ +------+ |
* | | | | | |
* | | this | ~Out~> | sink | |
* | | | | | |
* | +------+ +------+ |
* +----------------------------+
* }}}
*/
def to(sink: Graph[SinkShape[Out], _]): RunnableGraph[Mat] =
RunnableGraph.fromGraph(delegate.to(sink))
/**
* Transform this stream by applying the given function to each of the elements
* as they pass through this processing step.
*
* '''Emits when''' the mapping function returns an element
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def map[T](f: function.Function[Out, T]): SubSource[T, Mat] =
new SubSource(delegate.map(f.apply))
/**
* Transform each input element into an `Iterable` of output elements that is
* then flattened into the output stream.
*
* Make sure that the `Iterable` is immutable or at least not modified after
* being used as an output sequence. Otherwise the stream may fail with
* `ConcurrentModificationException` or other more subtle errors may occur.
*
* The returned `Iterable` MUST NOT contain `null` values,
* as they are illegal as stream elements - according to the Reactive Streams specification.
*
* '''Emits when''' the mapping function returns an element or there are still remaining elements
* from the previously calculated collection
*
* '''Backpressures when''' downstream backpressures or there are still remaining elements from the
* previously calculated collection
*
* '''Completes when''' upstream completes and all remaining elements has been emitted
*
* '''Cancels when''' downstream cancels
*/
def mapConcat[T](f: function.Function[Out, java.lang.Iterable[T]]): SubSource[T, Mat] =
new SubSource(delegate.mapConcat { elem Util.immutableSeq(f(elem)) })
/**
* Transform this stream by applying the given function to each of the elements
* as they pass through this processing step. The function returns a `Future` and the
* value of that future will be emitted downstreams. As many futures as requested elements by
* downstream may run in parallel and may complete in any order, but the elements that
* are emitted downstream are in the same order as received from upstream.
*
* If the function `f` throws an exception or if the `Future` is completed
* with failure and the supervision decision is [[akka.stream.Supervision#stop]]
* the stream will be completed with failure.
*
* If the function `f` throws an exception or if the `Future` is completed
* with failure and the supervision decision is [[akka.stream.Supervision#resume]] or
* [[akka.stream.Supervision#restart]] the element is dropped and the stream continues.
*
* The function `f` is always invoked on the elements in the order they arrive.
*
* '''Emits when''' the Future 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 or the first future is not completed
*
* '''Completes when''' upstream completes and all futures has been completed and all elements has been emitted
*
* '''Cancels when''' downstream cancels
*
* @see [[#mapAsyncUnordered]]
*/
def mapAsync[T](parallelism: Int, f: function.Function[Out, Future[T]]): SubSource[T, Mat] =
new SubSource(delegate.mapAsync(parallelism)(f.apply))
/**
* Transform this stream by applying the given function to each of the elements
* as they pass through this processing step. The function returns a `Future` and the
* value of that future will be emitted downstreams. As many futures as requested elements by
* downstream may run in parallel and each processed element will be emitted downstream
* as soon as it is ready, i.e. it is possible that the elements are not emitted downstream
* in the same order as received from upstream.
*
* If the function `f` throws an exception or if the `Future` is completed
* with failure and the supervision decision is [[akka.stream.Supervision#stop]]
* the stream will be completed with failure.
*
* If the function `f` throws an exception or if the `Future` is completed
* with failure and the supervision decision is [[akka.stream.Supervision#resume]] or
* [[akka.stream.Supervision#restart]] the element is dropped and the stream continues.
*
* The function `f` is always invoked on the elements in the order they arrive (even though the result of the futures
* returned by `f` might be emitted in a different order).
*
* '''Emits when''' any of the Futures 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 has been completed and all elements has been emitted
*
* '''Cancels when''' downstream cancels
*
* @see [[#mapAsync]]
*/
def mapAsyncUnordered[T](parallelism: Int, f: function.Function[Out, Future[T]]): SubSource[T, Mat] =
new SubSource(delegate.mapAsyncUnordered(parallelism)(f.apply))
/**
* Only pass on those elements that satisfy the given predicate.
*
* '''Emits when''' the given predicate returns true for the element
*
* '''Backpressures when''' the given predicate returns true for the element and downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*
*/
def filter(p: function.Predicate[Out]): SubSource[Out, Mat] =
new SubSource(delegate.filter(p.test))
/**
* Only pass on those elements that NOT satisfy the given predicate.
*
* '''Emits when''' the given predicate returns false for the element
*
* '''Backpressures when''' the given predicate returns false for the element and downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def filterNot(p: function.Predicate[Out]): SubSource[Out, Mat] =
new SubSource(delegate.filterNot(p.test))
/**
* Transform this stream by applying the given partial function to each of the elements
* on which the function is defined as they pass through this processing step.
* Non-matching elements are filtered out.
*
* '''Emits when''' the provided partial function is defined for the element
*
* '''Backpressures when''' the partial function is defined for the element and downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def collect[T](pf: PartialFunction[Out, T]): SubSource[T, Mat] =
new SubSource(delegate.collect(pf))
/**
* Chunk up this stream into groups of the given size, with the last group
* possibly smaller than requested due to end-of-stream.
*
* `n` must be positive, otherwise IllegalArgumentException is thrown.
*
* '''Emits when''' the specified number of elements has been accumulated or upstream completed
*
* '''Backpressures when''' a group has been assembled and downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def grouped(n: Int): SubSource[java.util.List[Out @uncheckedVariance], Mat] =
new SubSource(delegate.grouped(n).map(_.asJava)) // TODO optimize to one step
/**
* Apply a sliding window over the stream and return the windows as groups of elements, with the last group
* possibly smaller than requested due to end-of-stream.
*
* `n` must be positive, otherwise IllegalArgumentException is thrown.
* `step` must be positive, otherwise IllegalArgumentException is thrown.
*
* '''Emits when''' enough elements have been collected within the window or upstream completed
*
* '''Backpressures when''' a window has been assembled and downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def sliding(n: Int, step: Int = 1): SubSource[java.util.List[Out @uncheckedVariance], Mat] =
new SubSource(delegate.sliding(n, step).map(_.asJava)) // TODO optimize to one step
/**
* Similar to `fold` but is not a terminal operation,
* emits its current value which starts at `zero` and then
* applies the current and next value to the given function `f`,
* emitting the next current value.
*
* If the function `f` throws an exception and the supervision decision is
* [[akka.stream.Supervision#restart]] current value starts at `zero` again
* the stream will continue.
*
* '''Emits when''' the function scanning the element returns a new element
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def scan[T](zero: T)(f: function.Function2[T, Out, T]): SubSource[T, Mat] =
new SubSource(delegate.scan(zero)(f.apply))
/**
* Similar to `scan` but only emits its result when the upstream completes,
* after which it also completes. Applies the given function `f` towards its current and next value,
* yielding the next current value.
*
* If the function `f` throws an exception and the supervision decision is
* [[akka.stream.Supervision#restart]] current value starts at `zero` again
* the stream will continue.
*
* '''Emits when''' upstream completes
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def fold[T](zero: T)(f: function.Function2[T, Out, T]): SubSource[T, Mat] =
new SubSource(delegate.fold(zero)(f.apply))
/**
* Intersperses stream with provided element, similar to how [[scala.collection.immutable.List.mkString]]
* injects a separator between a List's elements.
*
* Additionally can inject start and end marker elements to stream.
*
* Examples:
*
* {{{
* Source<Integer, ?> nums = Source.from(Arrays.asList(0, 1, 2, 3));
* nums.intersperse(","); // 1 , 2 , 3
* nums.intersperse("[", ",", "]"); // [ 1 , 2 , 3 ]
* }}}
*
* In case you want to only prepend or only append an element (yet still use the `intercept` feature
* to inject a separator between elements, you may want to use the following pattern instead of the 3-argument
* version of intersperse (See [[Source.concat]] for semantics details):
*
* {{{
* Source.single(">> ").concat(flow.intersperse(","))
* flow.intersperse(",").concat(Source.single("END"))
* }}}
*
* '''Emits when''' upstream emits (or before with the `start` element if provided)
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def intersperse[T >: Out](start: T, inject: T, end: T): SubSource[T, Mat] =
new SubSource(delegate.intersperse(start, inject, end))
/**
* Intersperses stream with provided element, similar to how [[scala.collection.immutable.List.mkString]]
* injects a separator between a List's elements.
*
* Additionally can inject start and end marker elements to stream.
*
* Examples:
*
* {{{
* Source<Integer, ?> nums = Source.from(Arrays.asList(0, 1, 2, 3));
* nums.intersperse(","); // 1 , 2 , 3
* nums.intersperse("[", ",", "]"); // [ 1 , 2 , 3 ]
* }}}
*
* '''Emits when''' upstream emits (or before with the `start` element if provided)
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def intersperse[T >: Out](inject: T): SubSource[T, Mat] =
new SubSource(delegate.intersperse(inject))
/**
* Chunk up this stream into groups of elements received within a time window,
* or limited by the given number of elements, whatever happens first.
* Empty groups will not be emitted if no elements are received from upstream.
* The last group before end-of-stream will contain the buffered elements
* since the previously emitted group.
*
* '''Emits when''' the configured time elapses since the last group has been emitted
*
* '''Backpressures when''' the configured time elapses since the last group has been emitted
*
* '''Completes when''' upstream completes (emits last group)
*
* '''Cancels when''' downstream completes
*
* `n` must be positive, and `d` must be greater than 0 seconds, otherwise
* IllegalArgumentException is thrown.
*/
def groupedWithin(n: Int, d: FiniteDuration): SubSource[java.util.List[Out @uncheckedVariance], Mat] =
new SubSource(delegate.groupedWithin(n, d).map(_.asJava)) // TODO optimize to one step
/**
* Discard the given number of elements at the beginning of the stream.
* No elements will be dropped if `n` is zero or negative.
*
* '''Emits when''' the specified number of elements has been dropped already
*
* '''Backpressures when''' the specified number of elements has been dropped and downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def drop(n: Long): SubSource[Out, Mat] =
new SubSource(delegate.drop(n))
/**
* Discard the elements received within the given duration at beginning of the stream.
*
* '''Emits when''' the specified time elapsed and a new upstream element arrives
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def dropWithin(d: FiniteDuration): SubSource[Out, Mat] =
new SubSource(delegate.dropWithin(d))
/**
* Terminate processing (and cancel the upstream publisher) after predicate
* returns false for the first time. Due to input buffering some elements may have been
* requested from upstream publishers that will then not be processed downstream
* of this step.
*
* The stream will be completed without producing any elements if predicate is false for
* the first stream element.
*
* '''Emits when''' the predicate is true
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' predicate returned false or upstream completes
*
* '''Cancels when''' predicate returned false or downstream cancels
*/
def takeWhile(p: function.Predicate[Out]): SubSource[Out, Mat] =
new SubSource(delegate.takeWhile(p.test))
/**
* Discard elements at the beginning of the stream while predicate is true.
* All elements will be taken after predicate returns false first time.
*
* '''Emits when''' predicate returned false and for all following stream elements
*
* '''Backpressures when''' predicate returned false and downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def dropWhile(p: function.Predicate[Out]): SubSource[Out, Mat] =
new SubSource(delegate.dropWhile(p.test))
/**
* Recover allows to send last element on failure and gracefully complete the stream
* Since the underlying failure signal onError arrives out-of-band, it might jump over existing elements.
* This stage can recover the failure signal, but not the skipped elements, which will be dropped.
*
* '''Emits when''' element is available from the upstream or upstream is failed and pf returns an element
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes or upstream failed with exception pf can handle
*
* '''Cancels when''' downstream cancels
*
*/
def recover[T >: Out](pf: PartialFunction[Throwable, T]): SubSource[T, Mat] =
new SubSource(delegate.recover(pf))
/**
* Terminate processing (and cancel the upstream publisher) after the given
* number of elements. Due to input buffering some elements may have been
* requested from upstream publishers that will then not be processed downstream
* of this step.
*
* The stream will be completed without producing any elements if `n` is zero
* or negative.
*
* '''Emits when''' the specified number of elements to take has not yet been reached
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' the defined number of elements has been taken or upstream completes
*
* '''Cancels when''' the defined number of elements has been taken or downstream cancels
*/
def take(n: Long): SubSource[Out, Mat] =
new SubSource(delegate.take(n))
/**
* Terminate processing (and cancel the upstream publisher) after the given
* duration. Due to input buffering some elements may have been
* requested from upstream publishers that will then not be processed downstream
* of this step.
*
* Note that this can be combined with [[#take]] to limit the number of elements
* within the duration.
*
* '''Emits when''' an upstream element arrives
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes or timer fires
*
* '''Cancels when''' downstream cancels or timer fires
*/
def takeWithin(d: FiniteDuration): SubSource[Out, Mat] =
new SubSource(delegate.takeWithin(d))
/**
* Allows a faster upstream to progress independently of a slower subscriber by conflating elements into a summary
* until the subscriber is ready to accept them. For example a conflate step might average incoming numbers if the
* upstream publisher is faster.
*
* This element only rolls up elements if the upstream is faster, but if the downstream is faster it will not
* duplicate elements.
*
* '''Emits when''' downstream stops backpressuring and there is a conflated element available
*
* '''Backpressures when''' never
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*
* @param seed Provides the first state for a conflated value using the first unconsumed element as a start
* @param aggregate Takes the currently aggregated value and the current pending element to produce a new aggregate
*
*/
def conflate[S](seed: function.Function[Out, S], aggregate: function.Function2[S, Out, S]): SubSource[S, Mat] =
new SubSource(delegate.conflate(seed.apply)(aggregate.apply))
/**
* Allows a faster downstream to progress independently of a slower publisher by extrapolating elements from an older
* element until new element comes from the upstream. For example an expand step might repeat the last element for
* the subscriber until it receives an update from upstream.
*
* This element will never "drop" upstream elements as all elements go through at least one extrapolation step.
* This means that if the upstream is actually faster than the upstream it will be backpressured by the downstream
* subscriber.
*
* Expand does not support [[akka.stream.Supervision#restart]] and [[akka.stream.Supervision#resume]].
* Exceptions from the `seed` or `extrapolate` functions will complete the stream with failure.
*
* '''Emits when''' downstream stops backpressuring
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*
* @param seed Provides the first state for extrapolation using the first unconsumed element
* @param extrapolate Takes the current extrapolation state to produce an output element and the next extrapolation
* state.
*/
def expand[S, U](seed: function.Function[Out, S], extrapolate: function.Function[S, akka.japi.Pair[U, S]]): SubSource[U, Mat] =
new SubSource(delegate.expand(seed(_))(s {
val p = extrapolate(s)
(p.first, p.second)
}))
/**
* Adds a fixed size buffer in the flow that allows to store elements from a faster upstream until it becomes full.
* Depending on the defined [[akka.stream.OverflowStrategy]] it might drop elements or backpressure the upstream if
* there is no space available
*
* '''Emits when''' downstream stops backpressuring and there is a pending element in the buffer
*
* '''Backpressures when''' depending on OverflowStrategy
* * Backpressure - backpressures when buffer is full
* * DropHead, DropTail, DropBuffer - never backpressures
* * Fail - fails the stream if buffer gets full
*
* '''Completes when''' upstream completes and buffered elements has been drained
*
* '''Cancels when''' downstream cancels
*
* @param size The size of the buffer in element count
* @param overflowStrategy Strategy that is used when incoming elements cannot fit inside the buffer
*/
def buffer(size: Int, overflowStrategy: OverflowStrategy): SubSource[Out, Mat] =
new SubSource(delegate.buffer(size, overflowStrategy))
/**
* Generic transformation of a stream with a custom processing [[akka.stream.stage.Stage]].
* This operator makes it possible to extend the `Flow` API when there is no specialized
* operator that performs the transformation.
*/
def transform[U](mkStage: function.Creator[Stage[Out, U]]): SubSource[U, Mat] =
new SubSource(delegate.transform(() mkStage.create()))
/**
* Takes up to `n` elements from the stream (less than `n` only if the upstream completes before emitting `n` elements)
* and returns a pair containing a strict sequence of the taken element
* and a stream representing the remaining elements. If ''n'' is zero or negative, then this will return a pair
* of an empty collection and a stream containing the whole upstream unchanged.
*
* In case of an upstream error, depending on the current state
* - the master stream signals the error if less than `n` elements has been seen, and therefore the substream
* has not yet been emitted
* - the tail substream signals the error after the prefix and tail has been emitted by the main stream
* (at that point the main stream has already completed)
*
* '''Emits when''' the configured number of prefix elements are available. Emits this prefix, and the rest
* as a substream
*
* '''Backpressures when''' downstream backpressures or substream backpressures
*
* '''Completes when''' prefix elements has been consumed and substream has been consumed
*
* '''Cancels when''' downstream cancels or substream cancels
*/
def prefixAndTail(n: Int): SubSource[akka.japi.Pair[java.util.List[Out @uncheckedVariance], javadsl.Source[Out @uncheckedVariance, Unit]], Mat] =
new SubSource(delegate.prefixAndTail(n).map { case (taken, tail) akka.japi.Pair(taken.asJava, tail.asJava) })
/**
* Transform each input element into a `Source` of output elements that is
* then flattened into the output stream by concatenation,
* fully consuming one Source after the other.
*
* '''Emits when''' a currently consumed substream has an element available
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes and all consumed substreams complete
*
* '''Cancels when''' downstream cancels
*
*/
def flatMapConcat[T, M](f: function.Function[Out, _ <: Graph[SourceShape[T], M]]): SubSource[T, Mat] =
new SubSource(delegate.flatMapConcat(x f(x)))
/**
* Transform each input element into a `Source` of output elements that is
* then flattened into the output stream by merging, where at most `breadth`
* substreams are being consumed at any given time.
*
* '''Emits when''' a currently consumed substream has an element available
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes and all consumed substreams complete
*
* '''Cancels when''' downstream cancels
*/
def flatMapMerge[T, M](breadth: Int, f: function.Function[Out, _ <: Graph[SourceShape[T], M]]): SubSource[T, Mat] =
new SubSource(delegate.flatMapMerge(breadth, o f(o)))
/**
* Concatenate the given [[Source]] to this [[Flow]], meaning that once this
* Flows input is exhausted and all result elements have been generated,
* the Sources elements will be produced.
*
* Note that the [[Source]] is materialized together with this Flow and just kept
* from producing elements by asserting back-pressure until its time comes.
*
* If this [[Flow]] gets upstream error - no elements from the given [[Source]] will be pulled.
*
* '''Emits when''' element is available from current stream or from the given [[Source]] when current is completed
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' given [[Source]] completes
*
* '''Cancels when''' downstream cancels
*/
def concat[T >: Out, M](that: Graph[SourceShape[T], M]): SubSource[T, Mat] =
new SubSource(delegate.concat(that))
/**
* Attaches the given [[Sink]] to this [[Flow]], meaning that elements that passes
* through will also be sent to the [[Sink]].
*
* '''Emits when''' element is available and demand exists both from the Sink and the downstream.
*
* '''Backpressures when''' downstream or Sink backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def alsoTo(that: Graph[SinkShape[Out], _]): SubSource[Out, Mat] =
new SubSource(delegate.alsoTo(that))
/**
* Merge the given [[Source]] to this [[Flow]], taking elements as they arrive from input streams,
* picking randomly when several elements ready.
*
* '''Emits when''' one of the inputs has an element available
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' all upstreams complete
*
* '''Cancels when''' downstream cancels
*/
def merge[T >: Out](that: Graph[SourceShape[T], _]): SubSource[T, Mat] =
new SubSource(delegate.merge(that))
/**
* Combine the elements of current [[Flow]] and the given [[Source]] into a stream of tuples.
*
* '''Emits when''' all of the inputs has an element available
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' any upstream completes
*
* '''Cancels when''' downstream cancels
*/
def zip[T](source: Graph[SourceShape[T], _]): SubSource[Out @uncheckedVariance Pair T, Mat] =
new SubSource(delegate.zip(source))
/**
* Put together the elements of current [[Flow]] and the given [[Source]]
* into a stream of combined elements using a combiner function.
*
* '''Emits when''' all of the inputs has an element available
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' any upstream completes
*
* '''Cancels when''' downstream cancels
*/
def zipWith[Out2, Out3](that: Graph[SourceShape[Out2], _],
combine: function.Function2[Out, Out2, Out3]): SubSource[Out3, Mat] =
new SubSource(delegate.zipWith[Out2, Out3](that)(combinerToScala(combine)))
/**
* If the first element has not passed through this stage before the provided timeout, the stream is failed
* with a [[java.util.concurrent.TimeoutException]].
*
* '''Emits when''' upstream emits an element
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes or fails if timeout elapses before first element arrives
*
* '''Cancels when''' downstream cancels
*/
def initialTimeout(timeout: FiniteDuration): SubSource[Out, Mat] =
new SubSource(delegate.initialTimeout(timeout))
/**
* If the completion of the stream does not happen until the provided timeout, the stream is failed
* with a [[java.util.concurrent.TimeoutException]].
*
* '''Emits when''' upstream emits an element
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes or fails if timeout elapses before upstream completes
*
* '''Cancels when''' downstream cancels
*/
def completionTimeout(timeout: FiniteDuration): SubSource[Out, Mat] =
new SubSource(delegate.completionTimeout(timeout))
/**
* If the time between two processed elements exceed the provided timeout, the stream is failed
* with a [[java.util.concurrent.TimeoutException]].
*
* '''Emits when''' upstream emits an element
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes or fails if timeout elapses between two emitted elements
*
* '''Cancels when''' downstream cancels
*/
def idleTimeout(timeout: FiniteDuration): SubSource[Out, Mat] =
new SubSource(delegate.idleTimeout(timeout))
/**
* Injects additional elements if the upstream does not emit for a configured amount of time. In other words, this
* stage attempts to maintains a base rate of emitted elements towards the downstream.
*
* If the downstream backpressures then no element is injected until downstream demand arrives. Injected elements
* do not accumulate during this period.
*
* Upstream elements are always preferred over injected elements.
*
* '''Emits when''' upstream emits an element or if the upstream was idle for the configured period
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def keepAlive[U >: Out](maxIdle: FiniteDuration, injectedElem: function.Creator[U]): SubSource[U, Mat] =
new SubSource(delegate.keepAlive(maxIdle, () injectedElem.create()))
/**
* Sends elements downstream with speed limited to `elements/per`. In other words, this stage set the maximum rate
* for emitting messages. This combinator works for streams where all elements have the same cost or length.
*
* Throttle implements the token bucket model. There is a bucket with a given token capacity (burst size or maximumBurst).
* Tokens drops into the bucket at a given rate and can be `spared` for later use up to bucket capacity
* to allow some burstyness. Whenever stream wants to send an element, it takes as many
* tokens from the bucket as number of elements. If there isn't any, throttle waits until the
* bucket accumulates enough tokens.
*
* Parameter `mode` manages behaviour when upstream is faster than throttle rate:
* - [[akka.stream.ThrottleMode.Shaping]] makes pauses before emitting messages to meet throttle rate
* - [[akka.stream.ThrottleMode.Enforcing]] fails with exception when upstream is faster than throttle rate
*
* '''Emits when''' upstream emits an element and configured time per each element elapsed
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def throttle(elements: Int, per: FiniteDuration, maximumBurst: Int,
mode: ThrottleMode): javadsl.SubSource[Out, Mat] =
new SubSource(delegate.throttle(elements, per, maximumBurst, mode))
/**
* Sends elements downstream with speed limited to `cost/per`. Cost is
* calculating for each element individually by calling `calculateCost` function.
* This combinator works for streams when elements have different cost(length).
* Streams of `ByteString` for example.
*
* Throttle implements the token bucket model. There is a bucket with a given token capacity (burst size or maximumBurst).
* Tokens drops into the bucket at a given rate and can be `spared` for later use up to bucket capacity
* to allow some burstyness. Whenever stream wants to send an element, it takes as many
* tokens from the bucket as element cost. If there isn't any, throttle waits until the
* bucket accumulates enough tokens. Elements that costs more than the allowed burst will be delayed proportionally
* to their cost minus available tokens, meeting the target rate.
*
* Parameter `mode` manages behaviour when upstream is faster than throttle rate:
* - [[akka.stream.ThrottleMode.Shaping]] makes pauses before emitting messages to meet throttle rate
* - [[akka.stream.ThrottleMode.Enforcing]] fails with exception when upstream is faster than throttle rate. Enforcing
* cannot emit elements that cost more than the maximumBurst
*
* '''Emits when''' upstream emits an element and configured time per each element elapsed
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def throttle(cost: Int, per: FiniteDuration, maximumBurst: Int,
costCalculation: function.Function[Out, Integer], mode: ThrottleMode): javadsl.SubSource[Out, Mat] =
new SubSource(delegate.throttle(cost, per, maximumBurst, costCalculation.apply _, mode))
/**
* Delays the initial element by the specified duration.
*
* '''Emits when''' upstream emits an element if the initial delay already elapsed
*
* '''Backpressures when''' downstream backpressures or initial delay not yet elapsed
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def initialDelay(delay: FiniteDuration): SubSource[Out, Mat] =
new SubSource(delegate.initialDelay(delay))
def withAttributes(attr: Attributes): SubSource[Out, Mat] =
new SubSource(delegate.withAttributes(attr))
def named(name: String): SubSource[Out, Mat] =
new SubSource(delegate.named(name))
/**
* Logs elements flowing through the stream as well as completion and erroring.
*
* By default element and completion signals are logged on debug level, and errors are logged on Error level.
* This can be adjusted according to your needs by providing a custom [[Attributes.LogLevels]] attribute on the given Flow:
*
* The `extract` function will be applied to each element before logging, so it is possible to log only those fields
* of a complex object flowing through this element.
*
* Uses the given [[LoggingAdapter]] for logging.
*
* '''Emits when''' the mapping function returns an element
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def log(name: String, extract: function.Function[Out, Any], log: LoggingAdapter): SubSource[Out, Mat] =
new SubSource(delegate.log(name, e extract.apply(e))(log))
/**
* Logs elements flowing through the stream as well as completion and erroring.
*
* By default element and completion signals are logged on debug level, and errors are logged on Error level.
* This can be adjusted according to your needs by providing a custom [[Attributes.LogLevels]] attribute on the given Flow:
*
* The `extract` function will be applied to each element before logging, so it is possible to log only those fields
* of a complex object flowing through this element.
*
* Uses an internally created [[LoggingAdapter]] which uses `akka.stream.Log` as it's source (use this class to configure slf4j loggers).
*
* '''Emits when''' the mapping function returns an element
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def log(name: String, extract: function.Function[Out, Any]): SubSource[Out, Mat] =
this.log(name, extract, null)
/**
* Logs elements flowing through the stream as well as completion and erroring.
*
* By default element and completion signals are logged on debug level, and errors are logged on Error level.
* This can be adjusted according to your needs by providing a custom [[Attributes.LogLevels]] attribute on the given Flow:
*
* Uses the given [[LoggingAdapter]] for logging.
*
* '''Emits when''' the mapping function returns an element
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def log(name: String, log: LoggingAdapter): SubSource[Out, Mat] =
this.log(name, ConstantFun.javaIdentityFunction[Out], log)
/**
* Logs elements flowing through the stream as well as completion and erroring.
*
* By default element and completion signals are logged on debug level, and errors are logged on Error level.
* This can be adjusted according to your needs by providing a custom [[Attributes.LogLevels]] attribute on the given Flow.
*
* Uses an internally created [[LoggingAdapter]] which uses `akka.stream.Log` as it's source (use this class to configure slf4j loggers).
*
* '''Emits when''' the mapping function returns an element
*
* '''Backpressures when''' downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def log(name: String): SubSource[Out, Mat] =
this.log(name, ConstantFun.javaIdentityFunction[Out], null)
}

View file

@ -19,20 +19,28 @@ import scala.concurrent.Future
import scala.concurrent.duration.{ Duration, FiniteDuration }
import scala.language.higherKinds
import akka.stream.impl.fusing.FlattenMerge
import akka.stream.impl.SubFlowImpl
import akka.stream.impl.fusing.GraphInterpreter
/**
* A `Flow` is a set of stream processing steps that has one open input and one open output.
*/
final class Flow[-In, +Out, +Mat](private[stream] override val module: Module)
extends FlowOps[Out, Mat] with Graph[FlowShape[In, Out], Mat] {
extends FlowOpsMat[Out, Mat] with Graph[FlowShape[In, Out], Mat] {
override val shape: FlowShape[In, Out] = module.shape.asInstanceOf[FlowShape[In, Out]]
override type Repr[+O, +M] = Flow[In @uncheckedVariance, O, M]
override type Repr[+O] = Flow[In @uncheckedVariance, O, Mat @uncheckedVariance]
override type ReprMat[+O, +M] = Flow[In @uncheckedVariance, O, M]
override type Closed = Sink[In @uncheckedVariance, Mat @uncheckedVariance]
override type ClosedMat[+M] = Sink[In @uncheckedVariance, M]
private[stream] def isIdentity: Boolean = this.module eq Stages.identityGraph.module
def viaMat[T, Mat2, Mat3](flow: Graph[FlowShape[Out, T], Mat2])(combine: (Mat, Mat2) Mat3): Flow[In, T, Mat3] = {
override def via[T, Mat2](flow: Graph[FlowShape[Out, T], Mat2]): Repr[T] = viaMat(flow)(Keep.left)
override def viaMat[T, Mat2, Mat3](flow: Graph[FlowShape[Out, T], Mat2])(combine: (Mat, Mat2) Mat3): Flow[In, T, Mat3] =
if (this.isIdentity) {
Flow.fromGraph(flow.asInstanceOf[Graph[FlowShape[In, T], Mat2]])
.mapMaterializedValue(combine(().asInstanceOf[Mat], _))
@ -43,7 +51,6 @@ final class Flow[-In, +Out, +Mat](private[stream] override val module: Module)
.fuse(flowCopy, shape.outlet, flowCopy.shape.inlets.head, combine)
.replaceShape(FlowShape(shape.inlet, flowCopy.shape.outlets.head)))
}
}
/**
* Connect this [[Flow]] to a [[Sink]], concatenating the processing steps of both.
@ -96,7 +103,7 @@ final class Flow[-In, +Out, +Mat](private[stream] override val module: Module)
/**
* Transform the materialized value of this Flow, leaving all other properties as they were.
*/
def mapMaterializedValue[Mat2](f: Mat Mat2): Repr[Out, Mat2] =
def mapMaterializedValue[Mat2](f: Mat Mat2): ReprMat[Out, Mat2] =
new Flow(module.transformMaterializedValue(f.asInstanceOf[Any Any]))
/**
@ -183,16 +190,16 @@ final class Flow[-In, +Out, +Mat](private[stream] override val module: Module)
/** INTERNAL API */
// FIXME: Only exists to keep old stuff alive
private[stream] override def deprecatedAndThen[U](op: StageModule): Repr[U, Mat] = {
private[stream] override def deprecatedAndThen[U](op: StageModule): Repr[U] = {
//No need to copy here, op is a fresh instance
if (this.isIdentity) new Flow(op).asInstanceOf[Repr[U, Mat]]
if (this.isIdentity) new Flow(op).asInstanceOf[Repr[U]]
else new Flow(module.fuse(op, shape.outlet, op.inPort).replaceShape(FlowShape(shape.inlet, op.outPort)))
}
// FIXME: Only exists to keep old stuff alive
private[akka] def deprecatedAndThenMat[U, Mat2, O >: Out](processorFactory: () (Processor[O, U], Mat2)): Repr[U, Mat2] = {
private[akka] def deprecatedAndThenMat[U, Mat2, O >: Out](processorFactory: () (Processor[O, U], Mat2)): ReprMat[U, Mat2] = {
val op = DirectProcessor(processorFactory.asInstanceOf[() (Processor[Any, Any], Any)])
if (this.isIdentity) new Flow(op).asInstanceOf[Repr[U, Mat2]]
if (this.isIdentity) new Flow(op).asInstanceOf[ReprMat[U, Mat2]]
else new Flow[In, U, Mat2](module.fuse(op, shape.outlet, op.inPort, Keep.right).replaceShape(FlowShape(shape.inlet, op.outPort)))
}
@ -201,11 +208,11 @@ final class Flow[-In, +Out, +Mat](private[stream] override val module: Module)
* operation has no effect on an empty Flow (because the attributes apply
* only to the contained processing stages).
*/
override def withAttributes(attr: Attributes): Repr[Out, Mat] =
override def withAttributes(attr: Attributes): Repr[Out] =
if (this.module eq EmptyModule) this
else new Flow(module.withAttributes(attr).nest())
override def named(name: String): Repr[Out, Mat] = withAttributes(Attributes.name(name))
override def named(name: String): Repr[Out] = withAttributes(Attributes.name(name))
/**
* Connect the `Source` to this `Flow` and then connect it to the `Sink` and run it. The returned tuple contains
@ -222,7 +229,7 @@ final class Flow[-In, +Out, +Mat](private[stream] override val module: Module)
*
* @return A [[RunnableGraph]] that materializes to a Processor when run() is called on it.
*/
def toProcessor: RunnableGraph[Processor[In @uncheckedVariance, Out @uncheckedVariance]] = {
def toProcessor: RunnableGraph[Processor[In @uncheckedVariance, Out @uncheckedVariance]] =
Source.subscriber[In].via(this).toMat(Sink.publisher[Out](false))(Keep.both[Subscriber[In], Publisher[Out]])
.mapMaterializedValue {
case (sub, pub) new Processor[In, Out] {
@ -233,7 +240,6 @@ final class Flow[-In, +Out, +Mat](private[stream] override val module: Module)
override def subscribe(s: Subscriber[_ >: Out]): Unit = pub.subscribe(s)
}
}
}
/** Converts this Scala DSL element to it's Java DSL counterpart. */
def asJava: javadsl.Flow[In, Out, Mat] = new javadsl.Flow(this)
@ -325,17 +331,25 @@ final case class RunnableGraph[+Mat](private[stream] val module: StreamLayout.Mo
/**
* Scala API: Operations offered by Sources and Flows with a free output side: the DSL flows left-to-right only.
*
* INTERNAL API: extending this trait is not supported under the binary compatibility rules for Akka.
*/
trait FlowOps[+Out, +Mat] {
import akka.stream.impl.Stages._
type Repr[+O, +M] <: FlowOps[O, M]
type Repr[+O] <: FlowOps[O, Mat]
// result of closing a Source is RunnableGraph, closing a Flow is Sink
type Closed
/*
* Repr is actually self-bounded, but that would be a cyclic type declaration that is illegal in Scala.
* Therefore we need to help the compiler by specifying that Repr
* Therefore we need to help the compiler by specifying that Repr expressions can be flattened.
*/
import language.implicitConversions
private implicit def reprFlatten[O1, M1, O2, M2](r: Repr[O1, M1]#Repr[O2, M2]): Repr[O2, M2] = r.asInstanceOf[Repr[O2, M2]]
private implicit def reprFlatten0[O1](r: Repr[O1]#Closed): Closed = r.asInstanceOf[Closed]
private implicit def reprFlatten1[O1, O2](r: Repr[O1]#Repr[O2]): Repr[O2] = r.asInstanceOf[Repr[O2]]
private implicit def reprFlatten2[O1, O2, O3](r: Repr[O1]#Repr[O2]#Repr[O3]): Repr[O3] = r.asInstanceOf[Repr[O3]]
/**
* Transform this [[Flow]] by appending the given processing steps.
@ -354,25 +368,7 @@ trait FlowOps[+Out, +Mat] {
* value of the current flow (ignoring the other Flows value), use
* [[Flow#viaMat viaMat]] if a different strategy is needed.
*/
def via[T, Mat2](flow: Graph[FlowShape[Out, T], Mat2]): Repr[T, Mat] = viaMat(flow)(Keep.left)
/**
* Transform this [[Flow]] by appending the given processing steps.
* {{{
* +----------------------------+
* | Resulting Flow |
* | |
* | +------+ +------+ |
* | | | | | |
* In ~~> | this | ~Out~> | flow | ~~> T
* | | | | | |
* | +------+ +------+ |
* +----------------------------+
* }}}
* The `combine` function is used to compose the materialized values of this flow and that
* flow into the materialized value of the resulting Flow.
*/
def viaMat[T, Mat2, Mat3](flow: Graph[FlowShape[Out, T], Mat2])(combine: (Mat, Mat2) Mat3): Repr[T, Mat3]
def via[T, Mat2](flow: Graph[FlowShape[Out, T], Mat2]): Repr[T]
/**
* Recover allows to send last element on failure and gracefully complete the stream
@ -388,7 +384,7 @@ trait FlowOps[+Out, +Mat] {
* '''Cancels when''' downstream cancels
*
*/
def recover[T >: Out](pf: PartialFunction[Throwable, T]): Repr[T, Mat] = andThen(Recover(pf))
def recover[T >: Out](pf: PartialFunction[Throwable, T]): Repr[T] = andThen(Recover(pf))
/**
* Transform this stream by applying the given function to each of the elements
@ -403,7 +399,7 @@ trait FlowOps[+Out, +Mat] {
* '''Cancels when''' downstream cancels
*
*/
def map[T](f: Out T): Repr[T, Mat] = andThen(Map(f))
def map[T](f: Out T): Repr[T] = andThen(Map(f))
/**
* Transform each input element into an `Iterable` of output elements that is
@ -423,7 +419,7 @@ trait FlowOps[+Out, +Mat] {
* '''Cancels when''' downstream cancels
*
*/
def mapConcat[T](f: Out immutable.Iterable[T]): Repr[T, Mat] = andThen(MapConcat(f))
def mapConcat[T](f: Out immutable.Iterable[T]): Repr[T] = andThen(MapConcat(f))
/**
* Transform this stream by applying the given function to each of the elements
@ -454,7 +450,7 @@ trait FlowOps[+Out, +Mat] {
*
* @see [[#mapAsyncUnordered]]
*/
def mapAsync[T](parallelism: Int)(f: Out Future[T]): Repr[T, Mat] = via(MapAsync(parallelism, f))
def mapAsync[T](parallelism: Int)(f: Out Future[T]): Repr[T] = via(MapAsync(parallelism, f))
/**
* Transform this stream by applying the given function to each of the elements
@ -485,7 +481,7 @@ trait FlowOps[+Out, +Mat] {
*
* @see [[#mapAsync]]
*/
def mapAsyncUnordered[T](parallelism: Int)(f: Out Future[T]): Repr[T, Mat] = via(MapAsyncUnordered(parallelism, f))
def mapAsyncUnordered[T](parallelism: Int)(f: Out Future[T]): Repr[T] = via(MapAsyncUnordered(parallelism, f))
/**
* Only pass on those elements that satisfy the given predicate.
@ -498,7 +494,7 @@ trait FlowOps[+Out, +Mat] {
*
* '''Cancels when''' downstream cancels
*/
def filter(p: Out Boolean): Repr[Out, Mat] = andThen(Filter(p))
def filter(p: Out Boolean): Repr[Out] = andThen(Filter(p))
/**
* Only pass on those elements that NOT satisfy the given predicate.
@ -511,7 +507,7 @@ trait FlowOps[+Out, +Mat] {
*
* '''Cancels when''' downstream cancels
*/
def filterNot(p: Out Boolean): Repr[Out, Mat] =
def filterNot(p: Out Boolean): Repr[Out] =
via(Flow[Out].filter(!p(_)).withAttributes(name("filterNot")))
/**
@ -531,7 +527,7 @@ trait FlowOps[+Out, +Mat] {
*
* '''Cancels when''' predicate returned false or downstream cancels
*/
def takeWhile(p: Out Boolean): Repr[Out, Mat] = andThen(TakeWhile(p))
def takeWhile(p: Out Boolean): Repr[Out] = andThen(TakeWhile(p))
/**
* Discard elements at the beginning of the stream while predicate is true.
@ -545,7 +541,7 @@ trait FlowOps[+Out, +Mat] {
*
* '''Cancels when''' downstream cancels
*/
def dropWhile(p: Out Boolean): Repr[Out, Mat] = andThen(DropWhile(p))
def dropWhile(p: Out Boolean): Repr[Out] = andThen(DropWhile(p))
/**
* Transform this stream by applying the given partial function to each of the elements
@ -560,7 +556,7 @@ trait FlowOps[+Out, +Mat] {
*
* '''Cancels when''' downstream cancels
*/
def collect[T](pf: PartialFunction[Out, T]): Repr[T, Mat] = andThen(Collect(pf))
def collect[T](pf: PartialFunction[Out, T]): Repr[T] = andThen(Collect(pf))
/**
* Chunk up this stream into groups of the given size, with the last group
@ -576,7 +572,7 @@ trait FlowOps[+Out, +Mat] {
*
* '''Cancels when''' downstream cancels
*/
def grouped(n: Int): Repr[immutable.Seq[Out], Mat] = andThen(Grouped(n))
def grouped(n: Int): Repr[immutable.Seq[Out]] = andThen(Grouped(n))
/**
* Apply a sliding window over the stream and return the windows as groups of elements, with the last group
@ -593,7 +589,7 @@ trait FlowOps[+Out, +Mat] {
*
* '''Cancels when''' downstream cancels
*/
def sliding(n: Int, step: Int = 1): Repr[immutable.Seq[Out], Mat] = andThen(Sliding(n, step))
def sliding(n: Int, step: Int = 1): Repr[immutable.Seq[Out]] = andThen(Sliding(n, step))
/**
* Similar to `fold` but is not a terminal operation,
@ -613,7 +609,7 @@ trait FlowOps[+Out, +Mat] {
*
* '''Cancels when''' downstream cancels
*/
def scan[T](zero: T)(f: (T, Out) T): Repr[T, Mat] = andThen(Scan(zero, f))
def scan[T](zero: T)(f: (T, Out) T): Repr[T] = andThen(Scan(zero, f))
/**
* Similar to `scan` but only emits its result when the upstream completes,
@ -632,7 +628,7 @@ trait FlowOps[+Out, +Mat] {
*
* '''Cancels when''' downstream cancels
*/
def fold[T](zero: T)(f: (T, Out) T): Repr[T, Mat] = andThen(Fold(zero, f))
def fold[T](zero: T)(f: (T, Out) T): Repr[T] = andThen(Fold(zero, f))
/**
* Intersperses stream with provided element, similar to how [[scala.collection.immutable.List.mkString]]
@ -665,7 +661,7 @@ trait FlowOps[+Out, +Mat] {
*
* '''Cancels when''' downstream cancels
*/
def intersperse[T >: Out](start: T, inject: T, end: T): Repr[T, Mat] = {
def intersperse[T >: Out](start: T, inject: T, end: T): Repr[T] = {
ReactiveStreamsCompliance.requireNonNullElement(start)
ReactiveStreamsCompliance.requireNonNullElement(inject)
ReactiveStreamsCompliance.requireNonNullElement(end)
@ -694,7 +690,7 @@ trait FlowOps[+Out, +Mat] {
*
* '''Cancels when''' downstream cancels
*/
def intersperse[T >: Out](inject: T): Repr[T, Mat] = {
def intersperse[T >: Out](inject: T): Repr[T] = {
ReactiveStreamsCompliance.requireNonNullElement(inject)
andThen(Intersperse(None, inject, None))
}
@ -717,7 +713,7 @@ trait FlowOps[+Out, +Mat] {
*
* '''Cancels when''' downstream completes
*/
def groupedWithin(n: Int, d: FiniteDuration): Repr[immutable.Seq[Out], Mat] = {
def groupedWithin(n: Int, d: FiniteDuration): Repr[immutable.Seq[Out]] = {
require(n > 0, "n must be greater than 0")
require(d > Duration.Zero)
via(new GroupedWithin[Out](n, d).withAttributes(name("groupedWithin")))
@ -735,7 +731,7 @@ trait FlowOps[+Out, +Mat] {
*
* '''Cancels when''' downstream cancels
*/
def drop(n: Long): Repr[Out, Mat] = andThen(Drop(n))
def drop(n: Long): Repr[Out] = andThen(Drop(n))
/**
* Discard the elements received within the given duration at beginning of the stream.
@ -748,7 +744,7 @@ trait FlowOps[+Out, +Mat] {
*
* '''Cancels when''' downstream cancels
*/
def dropWithin(d: FiniteDuration): Repr[Out, Mat] =
def dropWithin(d: FiniteDuration): Repr[Out] =
via(new DropWithin[Out](d).withAttributes(name("dropWithin")))
/**
@ -768,7 +764,7 @@ trait FlowOps[+Out, +Mat] {
*
* '''Cancels when''' the defined number of elements has been taken or downstream cancels
*/
def take(n: Long): Repr[Out, Mat] = andThen(Take(n))
def take(n: Long): Repr[Out] = andThen(Take(n))
/**
* Terminate processing (and cancel the upstream publisher) after the given
@ -787,7 +783,7 @@ trait FlowOps[+Out, +Mat] {
*
* '''Cancels when''' downstream cancels or timer fires
*/
def takeWithin(d: FiniteDuration): Repr[Out, Mat] = via(new TakeWithin[Out](d).withAttributes(name("takeWithin")))
def takeWithin(d: FiniteDuration): Repr[Out] = via(new TakeWithin[Out](d).withAttributes(name("takeWithin")))
/**
* Allows a faster upstream to progress independently of a slower subscriber by conflating elements into a summary
@ -808,7 +804,7 @@ trait FlowOps[+Out, +Mat] {
* @param seed Provides the first state for a conflated value using the first unconsumed element as a start
* @param aggregate Takes the currently aggregated value and the current pending element to produce a new aggregate
*/
def conflate[S](seed: Out S)(aggregate: (S, Out) S): Repr[S, Mat] = andThen(Conflate(seed, aggregate))
def conflate[S](seed: Out S)(aggregate: (S, Out) S): Repr[S] = andThen(Conflate(seed, aggregate))
/**
* Allows a faster downstream to progress independently of a slower publisher by extrapolating elements from an older
@ -834,7 +830,7 @@ trait FlowOps[+Out, +Mat] {
* @param extrapolate Takes the current extrapolation state to produce an output element and the next extrapolation
* state.
*/
def expand[S, U](seed: Out S)(extrapolate: S (U, S)): Repr[U, Mat] = andThen(Expand(seed, extrapolate))
def expand[S, U](seed: Out S)(extrapolate: S (U, S)): Repr[U] = andThen(Expand(seed, extrapolate))
/**
* Adds a fixed size buffer in the flow that allows to store elements from a faster upstream until it becomes full.
@ -855,19 +851,16 @@ trait FlowOps[+Out, +Mat] {
* @param size The size of the buffer in element count
* @param overflowStrategy Strategy that is used when incoming elements cannot fit inside the buffer
*/
def buffer(size: Int, overflowStrategy: OverflowStrategy): Repr[Out, Mat] = andThen(Buffer(size, overflowStrategy))
def buffer(size: Int, overflowStrategy: OverflowStrategy): Repr[Out] = andThen(Buffer(size, overflowStrategy))
/**
* Generic transformation of a stream with a custom processing [[akka.stream.stage.Stage]].
* This operator makes it possible to extend the `Flow` API when there is no specialized
* operator that performs the transformation.
*/
def transform[T](mkStage: () Stage[Out, T]): Repr[T, Mat] =
def transform[T](mkStage: () Stage[Out, T]): Repr[T] =
via(new PushPullGraphStage((attr) mkStage(), Attributes.none))
private[akka] def transformMaterializing[T, M](mkStageAndMaterialized: () (Stage[Out, T], M)): Repr[T, M] =
viaMat(new PushPullGraphStageWithMaterializedValue[Out, T, Unit, M]((attr) mkStageAndMaterialized(), Attributes.none))(Keep.right)
/**
* Takes up to `n` elements from the stream (less than `n` only if the upstream completes before emitting `n` elements)
* and returns a pair containing a strict sequence of the taken element
@ -890,19 +883,27 @@ trait FlowOps[+Out, +Mat] {
* '''Cancels when''' downstream cancels or substream cancels
*
*/
def prefixAndTail[U >: Out](n: Int): Repr[(immutable.Seq[Out], Source[U, Unit]), Mat] =
def prefixAndTail[U >: Out](n: Int): Repr[(immutable.Seq[Out], Source[U, Unit])] =
deprecatedAndThen(PrefixAndTail(n))
/**
* This operation demultiplexes the incoming stream into separate output
* streams, one for each element key. The key is computed for each element
* using the given function. When a new key is encountered for the first time
* it is emitted to the downstream subscriber together with a fresh
* flow that will eventually produce all the elements of the substream
* for that key. Not consuming the elements from the created streams will
* stop this processor from processing more elements, therefore you must take
* care to unblock (or cancel) all of the produced streams even if you want
* to consume only one of them.
* a new substream is opened and subsequently fed with all elements belonging to
* that key.
*
* The object returned from this method is not a normal [[Source]] or [[Flow]],
* it is a [[SubFlow]]. This means that after this combinator all transformations
* are applied to all encountered substreams in the same fashion. Substream mode
* is exited either by closing the substream (i.e. connecting it to a [[Sink]])
* or by merging the substreams back together; see the `to` and `mergeBack` methods
* on [[SubFlow]] for more information.
*
* It is important to note that the substreams also propagate back-pressure as
* any other stream, which means that blocking one substream will block the `groupBy`
* operator itselfand thereby all substreamsonce all internal or
* explicit buffers are filled.
*
* If the group by function `f` throws an exception and the supervision decision
* is [[akka.stream.Supervision.Stop]] the stream and substreams will be completed
@ -921,9 +922,25 @@ trait FlowOps[+Out, +Mat] {
*
* '''Cancels when''' downstream cancels and all substreams cancel
*
* @param maxSubstreams configures the maximum number of substreams (keys)
* that are supported; if more distinct keys are encountered then the stream fails
*/
def groupBy[K, U >: Out](f: Out K): Repr[(K, Source[U, Unit]), Mat] =
deprecatedAndThen(GroupBy(f.asInstanceOf[Any Any]))
def groupBy[K](maxSubstreams: Int, f: Out K): SubFlow[Out, Mat, Repr, Closed] = {
implicit def mat = GraphInterpreter.currentInterpreter.materializer
val merge = new SubFlowImpl.MergeBack[Out, Repr] {
override def apply[T](flow: Flow[Out, T, Unit], breadth: Int): Repr[T] =
deprecatedAndThen[Source[Out, Unit]](GroupBy(maxSubstreams, f.asInstanceOf[Any Any]))
.map(_.via(flow))
.flatMapMerge(maxSubstreams, conforms)
/*
* FIXME remove all those commented workarounds above by implementing flatMapMerge(breadth)
*/
}
val finish: (Sink[Out, Unit]) Closed = s
deprecatedAndThen[Source[Out, Unit]](GroupBy(maxSubstreams, f.asInstanceOf[Any Any]))
.to(Sink.foreach(_.runWith(s)))
new SubFlowImpl(Flow[Out], merge, finish)
}
/**
* This operation applies the given predicate to all incoming elements and
@ -946,6 +963,18 @@ trait FlowOps[+Out, +Mat] {
* true, false // subsequent substreams operate the same way
* }}}
*
* The object returned from this method is not a normal [[Source]] or [[Flow]],
* it is a [[SubFlow]]. This means that after this combinator all transformations
* are applied to all encountered substreams in the same fashion. Substream mode
* is exited either by closing the substream (i.e. connecting it to a [[Sink]])
* or by merging the substreams back together; see the `to` and `mergeBack` methods
* on [[SubFlow]] for more information.
*
* It is important to note that the substreams also propagate back-pressure as
* any other stream, which means that blocking one substream will block the `splitWhen`
* operator itselfand thereby all substreamsonce all internal or
* explicit buffers are filled.
*
* If the split predicate `p` throws an exception and the supervision decision
* is [[akka.stream.Supervision.Stop]] the stream and substreams will be completed
* with failure.
@ -966,8 +995,18 @@ trait FlowOps[+Out, +Mat] {
*
* See also [[FlowOps.splitAfter]].
*/
def splitWhen[U >: Out](p: Out Boolean): Repr[Source[U, Unit], Mat] =
deprecatedAndThen(Split.when(p.asInstanceOf[Any Boolean]))
def splitWhen(p: Out Boolean): SubFlow[Out, Mat, Repr, Closed] = {
val merge = new SubFlowImpl.MergeBack[Out, Repr] {
override def apply[T](flow: Flow[Out, T, Unit], breadth: Int): Repr[T] =
deprecatedAndThen[Source[Out, Unit]](Split.when(p.asInstanceOf[Any Boolean]))
.map(_.via(flow))
.flatMapConcat(conforms)
}
val finish: (Sink[Out, Unit]) Closed = s
deprecatedAndThen[Source[Out, Unit]](Split.when(p.asInstanceOf[Any Boolean]))
.to(Sink.foreach(_.runWith(s)(GraphInterpreter.currentInterpreter.materializer)))
new SubFlowImpl(Flow[Out], merge, finish)
}
/**
* This operation applies the given predicate to all incoming elements and
@ -981,6 +1020,18 @@ trait FlowOps[+Out, +Mat] {
* false, false, true // elements go into third substream
* }}}
*
* The object returned from this method is not a normal [[Source]] or [[Flow]],
* it is a [[SubFlow]]. This means that after this combinator all transformations
* are applied to all encountered substreams in the same fashion. Substream mode
* is exited either by closing the substream (i.e. connecting it to a [[Sink]])
* or by merging the substreams back together; see the `to` and `mergeBack` methods
* on [[SubFlow]] for more information.
*
* It is important to note that the substreams also propagate back-pressure as
* any other stream, which means that blocking one substream will block the `splitAfter`
* operator itselfand thereby all substreamsonce all internal or
* explicit buffers are filled.
*
* If the split predicate `p` throws an exception and the supervision decision
* is [[akka.stream.Supervision.Stop]] the stream and substreams will be completed
* with failure.
@ -1001,8 +1052,18 @@ trait FlowOps[+Out, +Mat] {
*
* See also [[FlowOps.splitWhen]].
*/
def splitAfter[U >: Out](p: Out Boolean): Repr[Source[U, Unit], Mat] =
deprecatedAndThen(Split.after(p.asInstanceOf[Any Boolean]))
def splitAfter(p: Out Boolean): SubFlow[Out, Mat, Repr, Closed] = {
val merge = new SubFlowImpl.MergeBack[Out, Repr] {
override def apply[T](flow: Flow[Out, T, Unit], breadth: Int): Repr[T] =
deprecatedAndThen[Source[Out, Unit]](Split.after(p.asInstanceOf[Any Boolean]))
.map(_.via(flow))
.flatMapConcat(conforms)
}
val finish: (Sink[Out, Unit]) Closed = s
deprecatedAndThen[Source[Out, Unit]](Split.after(p.asInstanceOf[Any Boolean]))
.to(Sink.foreach(_.runWith(s)(GraphInterpreter.currentInterpreter.materializer)))
new SubFlowImpl(Flow[Out], merge, finish)
}
/**
* Transform each input element into a `Source` of output elements that is
@ -1017,7 +1078,7 @@ trait FlowOps[+Out, +Mat] {
*
* '''Cancels when''' downstream cancels
*/
def flatMapConcat[T, M](f: Out Source[T, M]): Repr[T, Mat] = map(f).via(new FlattenMerge[T, M](1))
def flatMapConcat[T, M](f: Out Graph[SourceShape[T], M]): Repr[T] = map(f).via(new FlattenMerge[T, M](1))
/**
* Transform each input element into a `Source` of output elements that is
@ -1032,7 +1093,7 @@ trait FlowOps[+Out, +Mat] {
*
* '''Cancels when''' downstream cancels
*/
def flatMapMerge[T, M](breadth: Int, f: Out Source[T, M]): Repr[T, Mat] = map(f).via(new FlattenMerge[T, M](breadth))
def flatMapMerge[T, M](breadth: Int, f: Out Graph[SourceShape[T], M]): Repr[T] = map(f).via(new FlattenMerge[T, M](breadth))
/**
* If the first element has not passed through this stage before the provided timeout, the stream is failed
@ -1046,7 +1107,7 @@ trait FlowOps[+Out, +Mat] {
*
* '''Cancels when''' downstream cancels
*/
def initialTimeout(timeout: FiniteDuration): Repr[Out, Mat] = via(new Timers.Initial[Out](timeout))
def initialTimeout(timeout: FiniteDuration): Repr[Out] = via(new Timers.Initial[Out](timeout))
/**
* If the completion of the stream does not happen until the provided timeout, the stream is failed
@ -1060,7 +1121,7 @@ trait FlowOps[+Out, +Mat] {
*
* '''Cancels when''' downstream cancels
*/
def completionTimeout(timeout: FiniteDuration): Repr[Out, Mat] = via(new Timers.Completion[Out](timeout))
def completionTimeout(timeout: FiniteDuration): Repr[Out] = via(new Timers.Completion[Out](timeout))
/**
* If the time between two processed elements exceed the provided timeout, the stream is failed
@ -1074,7 +1135,7 @@ trait FlowOps[+Out, +Mat] {
*
* '''Cancels when''' downstream cancels
*/
def idleTimeout(timeout: FiniteDuration): Repr[Out, Mat] = via(new Timers.Idle[Out](timeout))
def idleTimeout(timeout: FiniteDuration): Repr[Out] = via(new Timers.Idle[Out](timeout))
/**
* Injects additional elements if the upstream does not emit for a configured amount of time. In other words, this
@ -1093,7 +1154,7 @@ trait FlowOps[+Out, +Mat] {
*
* '''Cancels when''' downstream cancels
*/
def keepAlive[U >: Out](maxIdle: FiniteDuration, injectedElem: () U): Repr[U, Mat] =
def keepAlive[U >: Out](maxIdle: FiniteDuration, injectedElem: () U): Repr[U] =
via(new Timers.IdleInject[Out, U](maxIdle, injectedElem))
/**
@ -1120,7 +1181,7 @@ trait FlowOps[+Out, +Mat] {
* '''Cancels when''' downstream cancels
*/
def throttle(elements: Int, per: FiniteDuration, maximumBurst: Int,
mode: ThrottleMode): Repr[Out, Mat] = {
mode: ThrottleMode): Repr[Out] = {
require(elements > 0, "elements must be > 0")
require(per.toMillis > 0, "per time must be > 0")
require(!(mode == ThrottleMode.Enforcing && maximumBurst < 0), "maximumBurst must be > 0 in Enforcing mode")
@ -1154,7 +1215,7 @@ trait FlowOps[+Out, +Mat] {
* '''Cancels when''' downstream cancels
*/
def throttle(cost: Int, per: FiniteDuration, maximumBurst: Int,
costCalculation: (Out) Int, mode: ThrottleMode): Repr[Out, Mat] = {
costCalculation: (Out) Int, mode: ThrottleMode): Repr[Out] = {
require(per.toMillis > 0, "per time must be > 0")
require(!(mode == ThrottleMode.Enforcing && maximumBurst < 0), "maximumBurst must be > 0 in Enforcing mode")
via(new Throttle(cost, per, maximumBurst, costCalculation, mode))
@ -1171,7 +1232,7 @@ trait FlowOps[+Out, +Mat] {
*
* '''Cancels when''' downstream cancels
*/
def initialDelay(delay: FiniteDuration): Repr[Out, Mat] = via(new Timers.DelayInitial[Out](delay))
def initialDelay(delay: FiniteDuration): Repr[Out] = via(new Timers.DelayInitial[Out](delay))
/**
* Logs elements flowing through the stream as well as completion and erroring.
@ -1190,7 +1251,7 @@ trait FlowOps[+Out, +Mat] {
*
* '''Cancels when''' downstream cancels
*/
def log(name: String, extract: Out Any = ConstantFun.scalaIdentityFunction)(implicit log: LoggingAdapter = null): Repr[Out, Mat] =
def log(name: String, extract: Out Any = ConstantFun.scalaIdentityFunction)(implicit log: LoggingAdapter = null): Repr[Out] =
andThen(Stages.Log(name, extract.asInstanceOf[Any Any], Option(log)))
/**
@ -1204,21 +1265,16 @@ trait FlowOps[+Out, +Mat] {
*
* '''Cancels when''' downstream cancels
*/
def zip[U](that: Graph[SourceShape[U], _]): Repr[(Out, U), Mat] = zipMat(that)(Keep.left)
def zip[U](that: Graph[SourceShape[U], _]): Repr[(Out, U)] = via(zipGraph(that))
/**
* Combine the elements of current flow and the given [[Source]] into a stream of tuples.
*
* @see [[#zip]].
*/
def zipMat[U, Mat2, Mat3](that: Graph[SourceShape[U], Mat2])(matF: (Mat, Mat2) Mat3): Repr[(Out, U), Mat3] =
this.viaMat(GraphDSL.create(that) { implicit b
protected def zipGraph[U, M](that: Graph[SourceShape[U], M]): Graph[FlowShape[Out @uncheckedVariance, (Out, U)], M] =
GraphDSL.create(that) { implicit b
r
import GraphDSL.Implicits._
val zip = b.add(Zip[Out, U]())
r ~> zip.in1
FlowShape(zip.in0, zip.out)
})(matF)
}
/**
* Put together the elements of current flow and the given [[Source]]
@ -1232,23 +1288,17 @@ trait FlowOps[+Out, +Mat] {
*
* '''Cancels when''' downstream cancels
*/
def zipWith[Out2, Out3](that: Graph[SourceShape[Out2], _])(combine: (Out, Out2) Out3): Repr[Out3, Mat] =
zipWithMat(that)(combine)(Keep.left)
def zipWith[Out2, Out3](that: Graph[SourceShape[Out2], _])(combine: (Out, Out2) Out3): Repr[Out3] =
via(zipWithGraph(that)(combine))
/**
* Put together the elements of current flow and the given [[Source]]
* into a stream of combined elements using a combiner function.
*
* @see [[#zipWith]].
*/
def zipWithMat[Out2, Out3, Mat2, Mat3](that: Graph[SourceShape[Out2], Mat2])(combine: (Out, Out2) Out3)(matF: (Mat, Mat2) Mat3): Repr[Out3, Mat3] =
this.viaMat(GraphDSL.create(that) { implicit b
protected def zipWithGraph[Out2, Out3, M](that: Graph[SourceShape[Out2], M])(combine: (Out, Out2) Out3): Graph[FlowShape[Out @uncheckedVariance, Out3], M] =
GraphDSL.create(that) { implicit b
r
import GraphDSL.Implicits._
val zip = b.add(ZipWith[Out, Out2, Out3](combine))
r ~> zip.in1
FlowShape(zip.in0, zip.out)
})(matF)
}
/**
* Merge the given [[Source]] to this [[Flow]], taking elements as they arrive from input streams,
@ -1262,23 +1312,17 @@ trait FlowOps[+Out, +Mat] {
*
* '''Cancels when''' downstream cancels
*/
def merge[U >: Out](that: Graph[SourceShape[U], _]): Repr[U, Mat] =
mergeMat(that)(Keep.left)
def merge[U >: Out](that: Graph[SourceShape[U], _]): Repr[U] =
via(mergeGraph(that))
/**
* Merge the given [[Source]] to this [[Flow]], taking elements as they arrive from input streams,
* picking randomly when several elements ready.
*
* @see [[#merge]].
*/
def mergeMat[U >: Out, Mat2, Mat3](that: Graph[SourceShape[U], Mat2])(matF: (Mat, Mat2) Mat3): Repr[U, Mat3] =
this.viaMat(GraphDSL.create(that) { implicit b
protected def mergeGraph[U >: Out, M](that: Graph[SourceShape[U], M]): Graph[FlowShape[Out @uncheckedVariance, U], M] =
GraphDSL.create(that) { implicit b
r
import GraphDSL.Implicits._
val merge = b.add(Merge[U](2))
r ~> merge.in(1)
FlowShape(merge.in(0), merge.out)
})(matF)
}
/**
* Concatenate the given [[Source]] to this [[Flow]], meaning that once this
@ -1298,29 +1342,17 @@ trait FlowOps[+Out, +Mat] {
*
* '''Cancels when''' downstream cancels
*/
def concat[U >: Out, Mat2](that: Graph[SourceShape[U], Mat2]): Repr[U, Mat] =
concatMat(that)(Keep.left)
def concat[U >: Out, Mat2](that: Graph[SourceShape[U], Mat2]): Repr[U] =
via(concatGraph(that))
/**
* Concatenate the given [[Source]] to this [[Flow]], meaning that once this
* Flows input is exhausted and all result elements have been generated,
* the Sources elements will be produced.
*
* Note that the [[Source]] is materialized together with this Flow and just kept
* from producing elements by asserting back-pressure until its time comes.
*
* If this [[Flow]] gets upstream error - no elements from the given [[Source]] will be pulled.
*
* @see [[#concat]].
*/
def concatMat[U >: Out, Mat2, Mat3](that: Graph[SourceShape[U], Mat2])(matF: (Mat, Mat2) Mat3): Repr[U, Mat3] =
this.viaMat(GraphDSL.create(that) { implicit b
protected def concatGraph[U >: Out, Mat2](that: Graph[SourceShape[U], Mat2]): Graph[FlowShape[Out @uncheckedVariance, U], Mat2] =
GraphDSL.create(that) { implicit b
r
import GraphDSL.Implicits._
val merge = b.add(Concat[U]())
r ~> merge.in(1)
FlowShape(merge.in(0), merge.out)
})(matF)
}
/**
* Concatenates this [[Flow]] with the given [[Source]] so the first element
@ -1329,7 +1361,26 @@ trait FlowOps[+Out, +Mat] {
*
* This is a shorthand for [[concat]]
*/
def ++[U >: Out, M](that: Graph[SourceShape[U], M]): Repr[U, Mat] = concat(that)
def ++[U >: Out, M](that: Graph[SourceShape[U], M]): Repr[U] = concat(that)
/**
* Connect this [[Flow]] to a [[Sink]], concatenating the processing steps of both.
* {{{
* +----------------------------+
* | Resulting Sink |
* | |
* | +------+ +------+ |
* | | | | | |
* In ~~> | flow | ~Out~> | sink | |
* | | | | | |
* | +------+ +------+ |
* +----------------------------+
* }}}
* The materialized value of the combined [[Sink]] will be the materialized
* value of the current flow (ignoring the given Sinks value), use
* [[Flow#toMat[Mat2* toMat]] if a different strategy is needed.
*/
def to[Mat2](sink: Graph[SinkShape[Out], Mat2]): Closed
/**
* Attaches the given [[Sink]] to this [[Flow]], meaning that elements that passes
@ -1343,7 +1394,112 @@ trait FlowOps[+Out, +Mat] {
*
* '''Cancels when''' downstream cancels
*/
def alsoTo(that: Graph[SinkShape[Out], _]): Repr[Out, Mat] = alsoToMat(that)(Keep.left)
def alsoTo(that: Graph[SinkShape[Out], _]): Repr[Out] = via(alsoToGraph(that))
protected def alsoToGraph[M](that: Graph[SinkShape[Out], M]): Graph[FlowShape[Out @uncheckedVariance, Out], M] =
GraphDSL.create(that) { implicit b
r
import GraphDSL.Implicits._
val bcast = b.add(Broadcast[Out](2))
bcast.out(1) ~> r
FlowShape(bcast.in, bcast.out(0))
}
def withAttributes(attr: Attributes): Repr[Out]
def named(name: String): Repr[Out] = withAttributes(Attributes.name(name))
/** INTERNAL API */
private[scaladsl] def andThen[T](op: SymbolicStage[Out, T]): Repr[T] =
via(SymbolicGraphStage(op))
private[scaladsl] def deprecatedAndThen[U](op: StageModule): Repr[U]
}
/**
* INTERNAL API: extending this trait is not supported under the binary compatibility rules for Akka.
*/
trait FlowOpsMat[+Out, +Mat] extends FlowOps[Out, Mat] {
type ReprMat[+O, +M] <: FlowOpsMat[O, M]
type ClosedMat[+M] <: Graph[_, M]
/**
* Transform this [[Flow]] by appending the given processing steps.
* {{{
* +----------------------------+
* | Resulting Flow |
* | |
* | +------+ +------+ |
* | | | | | |
* In ~~> | this | ~Out~> | flow | ~~> T
* | | | | | |
* | +------+ +------+ |
* +----------------------------+
* }}}
* The `combine` function is used to compose the materialized values of this flow and that
* flow into the materialized value of the resulting Flow.
*/
def viaMat[T, Mat2, Mat3](flow: Graph[FlowShape[Out, T], Mat2])(combine: (Mat, Mat2) Mat3): ReprMat[T, Mat3]
/**
* Connect this [[Flow]] to a [[Sink]], concatenating the processing steps of both.
* {{{
* +----------------------------+
* | Resulting Sink |
* | |
* | +------+ +------+ |
* | | | | | |
* In ~~> | flow | ~Out~> | sink | |
* | | | | | |
* | +------+ +------+ |
* +----------------------------+
* }}}
* The `combine` function is used to compose the materialized values of this flow and that
* Sink into the materialized value of the resulting Sink.
*/
def toMat[Mat2, Mat3](sink: Graph[SinkShape[Out], Mat2])(combine: (Mat, Mat2) Mat3): ClosedMat[Mat3]
/**
* Combine the elements of current flow and the given [[Source]] into a stream of tuples.
*
* @see [[#zip]].
*/
def zipMat[U, Mat2, Mat3](that: Graph[SourceShape[U], Mat2])(matF: (Mat, Mat2) Mat3): ReprMat[(Out, U), Mat3] =
viaMat(zipGraph(that))(matF)
/**
* Put together the elements of current flow and the given [[Source]]
* into a stream of combined elements using a combiner function.
*
* @see [[#zipWith]].
*/
def zipWithMat[Out2, Out3, Mat2, Mat3](that: Graph[SourceShape[Out2], Mat2])(combine: (Out, Out2) Out3)(matF: (Mat, Mat2) Mat3): ReprMat[Out3, Mat3] =
viaMat(zipWithGraph(that)(combine))(matF)
/**
* Merge the given [[Source]] to this [[Flow]], taking elements as they arrive from input streams,
* picking randomly when several elements ready.
*
* @see [[#merge]].
*/
def mergeMat[U >: Out, Mat2, Mat3](that: Graph[SourceShape[U], Mat2])(matF: (Mat, Mat2) Mat3): ReprMat[U, Mat3] =
viaMat(mergeGraph(that))(matF)
/**
* Concatenate the given [[Source]] to this [[Flow]], meaning that once this
* Flows input is exhausted and all result elements have been generated,
* the Sources elements will be produced.
*
* Note that the [[Source]] is materialized together with this Flow and just kept
* from producing elements by asserting back-pressure until its time comes.
*
* If this [[Flow]] gets upstream error - no elements from the given [[Source]] will be pulled.
*
* @see [[#concat]].
*/
def concatMat[U >: Out, Mat2, Mat3](that: Graph[SourceShape[U], Mat2])(matF: (Mat, Mat2) Mat3): ReprMat[U, Mat3] =
viaMat(concatGraph(that))(matF)
/**
* Attaches the given [[Sink]] to this [[Flow]], meaning that elements that passes
@ -1351,20 +1507,13 @@ trait FlowOps[+Out, +Mat] {
*
* @see [[#alsoTo]]
*/
def alsoToMat[Mat2, Mat3](that: Graph[SinkShape[Out], Mat2])(matF: (Mat, Mat2) Mat3): Repr[Out, Mat3] =
this.viaMat(GraphDSL.create(that) { implicit b
r
import GraphDSL.Implicits._
val bcast = b.add(Broadcast[Out](2))
bcast.out(1) ~> r
FlowShape(bcast.in, bcast.out(0))
})(matF)
def alsoToMat[Mat2, Mat3](that: Graph[SinkShape[Out], Mat2])(matF: (Mat, Mat2) Mat3): ReprMat[Out, Mat3] =
viaMat(alsoToGraph(that))(matF)
def withAttributes(attr: Attributes): Repr[Out, Mat]
/**
* INTERNAL API.
*/
private[akka] def transformMaterializing[T, M](mkStageAndMaterialized: () (Stage[Out, T], M)): ReprMat[T, M] =
viaMat(new PushPullGraphStageWithMaterializedValue[Out, T, Unit, M]((attr) mkStageAndMaterialized(), Attributes.none))(Keep.right)
/** INTERNAL API */
private[scaladsl] def andThen[T](op: SymbolicStage[Out, T]): Repr[T, Mat] =
via(SymbolicGraphStage(op))
private[scaladsl] def deprecatedAndThen[U](op: StageModule): Repr[U, Mat]
}

View file

@ -656,19 +656,19 @@ object GraphDSL extends GraphApply {
else junction.in(n)
}
sealed trait CombinerBase[T] extends Any {
def importAndGetPort(b: Builder[_]): Outlet[T]
sealed trait CombinerBase[+T] extends Any {
def importAndGetPort(b: Builder[_]): Outlet[T @uncheckedVariance]
def ~>[U >: T](to: Inlet[U])(implicit b: Builder[_]): Unit =
b.addEdge(importAndGetPort(b), to)
def ~>[Out](via: Graph[FlowShape[T, Out], Any])(implicit b: Builder[_]): PortOps[Out, Unit] = {
def ~>[Out](via: Graph[FlowShape[T, Out], Any])(implicit b: Builder[_]): PortOps[Out] = {
val s = b.add(via)
b.addEdge(importAndGetPort(b), s.inlet)
s.outlet
}
def ~>[Out](junction: UniformFanInShape[T, Out])(implicit b: Builder[_]): PortOps[Out, Unit] = {
def ~>[Out](junction: UniformFanInShape[T, Out])(implicit b: Builder[_]): PortOps[Out] = {
def bind(n: Int): Unit = {
if (n == junction.inSeq.length)
throw new IllegalArgumentException(s"no more inlets free on $junction")
@ -679,7 +679,7 @@ object GraphDSL extends GraphApply {
junction.out
}
def ~>[Out](junction: UniformFanOutShape[T, Out])(implicit b: Builder[_]): PortOps[Out, Unit] = {
def ~>[Out](junction: UniformFanOutShape[T, Out])(implicit b: Builder[_]): PortOps[Out] = {
b.addEdge(importAndGetPort(b), junction.in)
try findOut(b, junction, 0)
catch {
@ -687,7 +687,7 @@ object GraphDSL extends GraphApply {
}
}
def ~>[Out](flow: FlowShape[T, Out])(implicit b: Builder[_]): PortOps[Out, Unit] = {
def ~>[Out](flow: FlowShape[T, Out])(implicit b: Builder[_]): PortOps[Out] = {
b.addEdge(importAndGetPort(b), flow.inlet)
flow.outlet
}
@ -744,31 +744,37 @@ object GraphDSL extends GraphApply {
// Although Mat is always Unit, it cannot be removed as a type parameter, otherwise the "override type"
// won't work below
class PortOps[Out, Mat](val outlet: Outlet[Out], b: Builder[_]) extends FlowOps[Out, Mat] with CombinerBase[Out] {
override type Repr[+O, +M] = PortOps[O, M] @uncheckedVariance
trait PortOps[+Out] extends FlowOps[Out, Unit] with CombinerBase[Out] {
override type Repr[+O] = PortOps[O]
override type Closed = Unit
def outlet: Outlet[Out @uncheckedVariance]
}
override def withAttributes(attr: Attributes): Repr[Out, Mat] =
private class PortOpsImpl[+Out](override val outlet: Outlet[Out @uncheckedVariance], b: Builder[_])
extends PortOps[Out] {
override def withAttributes(attr: Attributes): Repr[Out] =
throw new UnsupportedOperationException("Cannot set attributes on chained ops from a junction output port")
override def importAndGetPort(b: Builder[_]): Outlet[Out] = outlet
override def importAndGetPort(b: Builder[_]): Outlet[Out @uncheckedVariance] = outlet
override def via[T, Mat2](flow: Graph[FlowShape[Out, T], Mat2]): Repr[T, Mat] =
super.~>(flow)(b).asInstanceOf[Repr[T, Mat]]
override def via[T, Mat2](flow: Graph[FlowShape[Out, T], Mat2]): Repr[T] =
super.~>(flow)(b)
override def viaMat[T, Mat2, Mat3](flow: Graph[FlowShape[Out, T], Mat2])(combine: (Mat, Mat2) Mat3) =
throw new UnsupportedOperationException("Cannot use viaMat on a port")
override private[scaladsl] def deprecatedAndThen[U](op: StageModule): PortOps[U, Mat] = {
override private[scaladsl] def deprecatedAndThen[U](op: StageModule): Repr[U] = {
b.deprecatedAndThen(outlet, op)
new PortOps(op.shape.outlet.asInstanceOf[Outlet[U]], b)
new PortOpsImpl(op.shape.outlet.asInstanceOf[Outlet[U]], b)
}
def to[Mat2](sink: Graph[SinkShape[Out], Mat2]): Closed =
super.~>(sink)(b)
}
final class DisabledPortOps[Out, Mat](msg: String) extends PortOps[Out, Mat](null, null) {
private class DisabledPortOps[Out, Mat](msg: String) extends PortOpsImpl[Out](null, null) {
override def importAndGetPort(b: Builder[_]): Outlet[Out] = throw new IllegalArgumentException(msg)
override def viaMat[T, Mat2, Mat3](flow: Graph[FlowShape[Out, T], Mat2])(combine: (Mat, Mat2) Mat3) =
override def via[T, Mat2](flow: Graph[FlowShape[Out, T], Mat2]): Repr[T] =
throw new IllegalArgumentException(msg)
}
@ -872,14 +878,14 @@ object GraphDSL extends GraphApply {
import scala.language.implicitConversions
implicit def port2flow[T](from: Outlet[T])(implicit b: Builder[_]): PortOps[T, Unit] =
new PortOps(from, b)
implicit def port2flow[T](from: Outlet[T])(implicit b: Builder[_]): PortOps[T] =
new PortOpsImpl(from, b)
implicit def fanOut2flow[I, O](j: UniformFanOutShape[I, O])(implicit b: Builder[_]): PortOps[O, Unit] =
new PortOps(findOut(b, j, 0), b)
implicit def fanOut2flow[I, O](j: UniformFanOutShape[I, O])(implicit b: Builder[_]): PortOps[O] =
new PortOpsImpl(findOut(b, j, 0), b)
implicit def flow2flow[I, O](f: FlowShape[I, O])(implicit b: Builder[_]): PortOps[O, Unit] =
new PortOps(f.outlet, b)
implicit def flow2flow[I, O](f: FlowShape[I, O])(implicit b: Builder[_]): PortOps[O] =
new PortOpsImpl(f.outlet, b)
implicit final class SourceArrow[T](val s: Graph[SourceShape[T], _]) extends AnyVal with CombinerBase[T] {
override def importAndGetPort(b: Builder[_]): Outlet[T] = b.add(s).outlet

View file

@ -4,7 +4,6 @@
package akka.stream.scaladsl
import java.io.{ OutputStream, InputStream, File }
import akka.actor.{ ActorRef, Cancellable, Props }
import akka.stream.actor.ActorPublisher
import akka.stream.impl.Stages.{ DefaultAttributes, StageModule }
@ -15,12 +14,12 @@ import akka.stream.impl.{ EmptyPublisher, ErrorPublisher, _ }
import akka.stream.{ Outlet, SourceShape, _ }
import akka.util.ByteString
import org.reactivestreams.{ Publisher, Subscriber }
import scala.annotation.tailrec
import scala.collection.immutable
import scala.concurrent.duration.{ FiniteDuration, _ }
import scala.concurrent.{ Future, Promise }
import scala.language.higherKinds
import scala.annotation.unchecked.uncheckedVariance
/**
* A `Source` is a set of stream processing steps that has one open output. It can comprise
@ -29,13 +28,19 @@ import scala.language.higherKinds
* a Reactive Streams `Publisher` (at least conceptually).
*/
final class Source[+Out, +Mat](private[stream] override val module: Module)
extends FlowOps[Out, Mat] with Graph[SourceShape[Out], Mat] {
extends FlowOpsMat[Out, Mat] with Graph[SourceShape[Out], Mat] {
override type Repr[+O, +M] = Source[O, M]
override type Repr[+O] = Source[O, Mat @uncheckedVariance]
override type ReprMat[+O, +M] = Source[O, M]
override type Closed = RunnableGraph[Mat @uncheckedVariance]
override type ClosedMat[+M] = RunnableGraph[M]
override val shape: SourceShape[Out] = module.shape.asInstanceOf[SourceShape[Out]]
def viaMat[T, Mat2, Mat3](flow: Graph[FlowShape[Out, T], Mat2])(combine: (Mat, Mat2) Mat3): Source[T, Mat3] = {
override def via[T, Mat2](flow: Graph[FlowShape[Out, T], Mat2]): Repr[T] = viaMat(flow)(Keep.left)
override def viaMat[T, Mat2, Mat3](flow: Graph[FlowShape[Out, T], Mat2])(combine: (Mat, Mat2) Mat3): Source[T, Mat3] = {
if (flow.module eq Stages.identityGraph.module) this.asInstanceOf[Source[T, Mat3]]
else {
val flowCopy = flow.module.carbonCopy
@ -64,11 +69,11 @@ final class Source[+Out, +Mat](private[stream] override val module: Module)
/**
* Transform only the materialized value of this Source, leaving all other properties as they were.
*/
def mapMaterializedValue[Mat2](f: Mat Mat2): Repr[Out, Mat2] =
new Source(module.transformMaterializedValue(f.asInstanceOf[Any Any]))
def mapMaterializedValue[Mat2](f: Mat Mat2): ReprMat[Out, Mat2] =
new Source[Out, Mat2](module.transformMaterializedValue(f.asInstanceOf[Any Any]))
/** INTERNAL API */
override private[scaladsl] def deprecatedAndThen[U](op: StageModule): Repr[U, Mat] = {
override private[scaladsl] def deprecatedAndThen[U](op: StageModule): Repr[U] = {
// No need to copy here, op is a fresh instance
new Source(
module
@ -107,10 +112,10 @@ final class Source[+Out, +Mat](private[stream] override val module: Module)
* @param attr the attributes to add
* @return a new Source with the added attributes
*/
override def withAttributes(attr: Attributes): Repr[Out, Mat] =
override def withAttributes(attr: Attributes): Repr[Out] =
new Source(module.withAttributes(attr).nest()) // User API
override def named(name: String): Repr[Out, Mat] = withAttributes(Attributes.name(name))
override def named(name: String): Repr[Out] = withAttributes(Attributes.name(name))
/** Converts this Scala DSL element to it's Java DSL counterpart. */
def asJava: javadsl.Source[Out, Mat] = new javadsl.Source(this)

View file

@ -0,0 +1,40 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.stream.scaladsl
import akka.stream._
import akka.stream.impl.Stages.StageModule
import language.higherKinds
import scala.annotation.unchecked.uncheckedVariance
/**
* A stream of streams sub-flow of data elements, e.g. produced by `groupBy`.
* SubFlows cannot contribute to the super-flows materialized value since they
* are materialized later, during the runtime of the flow graph processing.
*/
trait SubFlow[+Out, +Mat, +F[+_], C] extends FlowOps[Out, Mat] {
override type Repr[+T] = SubFlow[T, Mat @uncheckedVariance, F @uncheckedVariance, C @uncheckedVariance]
override type Closed = C
/**
* Attach a [[Sink]] to each sub-flow, closing the overall Graph that is being
* constructed.
*/
def to[M](sink: Graph[SinkShape[Out], M]): C
/**
* Flatten the sub-flows back into the super-flow by performing a merge.
*/
def mergeSubstreams: F[Out]
/**
* Flatten the sub-flows back into the super-flow by concatenating them.
* This is usually a bad idea when combined with `groupBy` since it can
* easily lead to deadlockthe concatenation does not consume from the second
* substream until the first has finished and the `groupBy` stage will get
* back-pressure from the second stream.
*/
def concatSubstreams: F[Out]
}

View file

@ -305,13 +305,13 @@ abstract class GraphStageLogic private[stream] (val inCount: Int, val outCount:
* INTERNAL API
*/
// Using common array to reduce overhead for small port counts
private[stream] var handlers = Array.ofDim[Any](inCount + outCount)
private[stream] val handlers = Array.ofDim[Any](inCount + outCount)
/**
* INTERNAL API
*/
// Using common array to reduce overhead for small port counts
private[stream] var portToConn = Array.ofDim[Int](handlers.length)
private[stream] val portToConn = Array.ofDim[Int](handlers.length)
/**
* INTERNAL API