+str #19020 reduce combinator
This commit is contained in:
parent
55425e5ef3
commit
a2ab7f29e1
15 changed files with 247 additions and 37 deletions
|
|
@ -55,8 +55,10 @@ public class RecipeReduceByKeyTest extends RecipeTest {
|
|||
final Source<Pair<String, Integer>, NotUsed> counts = words
|
||||
// split the words into separate streams first
|
||||
.groupBy(MAXIMUM_DISTINCT_WORDS, i -> i)
|
||||
//transform each element to pair with number of words in it
|
||||
.map(i -> new Pair<>(i, 1))
|
||||
// add counting logic to the streams
|
||||
.fold(new Pair<>("", 0), (pair, elem) -> new Pair<>(elem, pair.second() + 1))
|
||||
.reduce((left, right) -> new Pair<>(left.first(), left.second() + right.second()))
|
||||
// get a stream of word counts
|
||||
.mergeSubstreams();
|
||||
//#word-count
|
||||
|
|
@ -77,17 +79,13 @@ public class RecipeReduceByKeyTest extends RecipeTest {
|
|||
static public <In, K, Out> Flow<In, Pair<K, Out>, NotUsed> reduceByKey(
|
||||
int maximumGroupSize,
|
||||
Function<In, K> groupKey,
|
||||
Function<K, Out> foldZero,
|
||||
Function2<Out, In, Out> fold,
|
||||
Materializer mat) {
|
||||
Function<In, Out> map,
|
||||
Function2<Out, Out, Out> reduce) {
|
||||
|
||||
return Flow.<In> create()
|
||||
.groupBy(maximumGroupSize, i -> i)
|
||||
.fold((Pair<K, Out>) null, (pair, elem) -> {
|
||||
final K key = groupKey.apply(elem);
|
||||
if (pair == null) return new Pair<>(key, fold.apply(foldZero.apply(key), elem));
|
||||
else return new Pair<>(key, fold.apply(pair.second(), elem));
|
||||
})
|
||||
.groupBy(maximumGroupSize, groupKey)
|
||||
.map(i -> new Pair<>(groupKey.apply(i), map.apply(i)))
|
||||
.reduce((left, right) -> new Pair<>(left.first(), reduce.apply(left.second(), right.second())))
|
||||
.mergeSubstreams();
|
||||
}
|
||||
//#reduce-by-key-general
|
||||
|
|
@ -104,9 +102,8 @@ public class RecipeReduceByKeyTest extends RecipeTest {
|
|||
Source<Pair<String, Integer>, NotUsed> counts = words.via(reduceByKey(
|
||||
MAXIMUM_DISTINCT_WORDS,
|
||||
word -> word,
|
||||
key -> 0,
|
||||
(count, elem) -> count + 1,
|
||||
mat));
|
||||
word -> 1,
|
||||
(left, right) -> left + right));
|
||||
|
||||
//#reduce-by-key-general2
|
||||
final Future<List<Pair<String, Integer>>> f = counts.grouped(10).runWith(Sink.head(), mat);
|
||||
|
|
|
|||
|
|
@ -113,7 +113,7 @@ 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). ``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
|
||||
the ``reduce`` 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
|
||||
|
|
@ -133,8 +133,8 @@ stream cannot continue without violating its resource bound, in this case
|
|||
By extracting the parts specific to *wordcount* into
|
||||
|
||||
* a ``groupKey`` function that defines the groups
|
||||
* a ``foldZero`` that defines the zero element used by the fold on the substream given the group key
|
||||
* a ``fold`` function that does the actual reduction
|
||||
* a ``map`` map each element to value that is used by the reduce on the substream
|
||||
* a ``reduce`` function that does the actual reduction
|
||||
|
||||
we get a generalized version below:
|
||||
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue