!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:
parent
654fa41443
commit
1500d1f36d
56 changed files with 3484 additions and 720 deletions
|
|
@ -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]] {
|
||||
|
|
|
|||
|
|
@ -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]",
|
||||
|
|
|
|||
|
|
@ -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(
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
------------------------------------------------
|
||||
|
|
|
|||
|
|
@ -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
|
||||
===============
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue