Added MergeSequence graph stage (#29247)

Fixes #28769

Use case for this is if you have a sequence of elements that has been
partitioned across multiple streams, and you want to merge them back
together in order. It will typically be used in combination with
`zipWithIndex` to define the index for the sequence, followed by a
`Partition`, followed by the processing of different substreams with
different flows (each flow emitting exactly one output for each input),
and then merging with this stage, using the index from `zipWithIndex`.

A more concrete use case is if you're consuming messages from a message
broker, and you have a flow that you wish to apply to some messages, but
not others, you can partition the message stream according to which
should be processed by the flow and which should bypass it, and then
bring the elements back together acknowledgement. If an ordinary merge
was used rather than this, the messages that bypass the processing flow
would likely overtake the messages going through the processing flow,
and the result would be out of order offset acknowledgement which would
lead to dropping messages on failure.

I've included a minimal version of the above example in the documentation.
This commit is contained in:
James Roper 2020-07-10 01:52:46 +10:00 committed by GitHub
parent 1898216b0d
commit 558160702b
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
9 changed files with 471 additions and 0 deletions

View file

@ -0,0 +1,46 @@
# MergeSequence
Merge a linear sequence partitioned across multiple sources.
@ref[Fan-in operators](index.md#fan-in-operators)
## Signature
@apidoc[MergeSequence]
## Description
Merge a linear sequence partitioned across multiple sources. Each element from upstream must have a defined index,
starting from 0. There must be no gaps in the sequence, nor may there be any duplicates. Each upstream source must be
ordered by the sequence.
## Example
`MergeSequence` is most useful when used in combination with `Partition`, to merge the partitioned stream back into
a single stream, while maintaining the order of the original elements. `zipWithIndex` can be used before partitioning
the stream to generate the index.
The example below shows partitioning a stream of messages into one stream for elements that must be processed by a
given processing flow, and another stream for elements for which no processing will be done, and then merges them
back together so that the messages can be acknowledged in order.
Scala
: @@snip [MergeSequenceDocExample.scala](/akka-docs/src/test/scala/docs/stream/operators/MergeSequenceDocExample.scala) { #merge-sequence }
Java
: @@snip [MergeSequenceDocExample.java](/akka-docs/src/test/java/jdocs/stream/operators/MergeSequenceDocExample.java) { #import #merge-sequence }
## Reactive Streams semantics
@@@div { .callout }
**emits** when one of the upstreams has the next expected element in the sequence available.
**backpressures** when downstream backpressures
**completes** when all upstreams complete
**cancels** downstream cancels
@@@

View file

@ -263,6 +263,7 @@ the inputs in different ways.
| |Operator|Description|
|--|--|--|
| |<a name="mergesequence"></a>@ref[MergeSequence](MergeSequence.md)|Merge a linear sequence partitioned across multiple sources.|
|Source/Flow|<a name="concat"></a>@ref[concat](Source-or-Flow/concat.md)|After completion of the original upstream the elements of the given source will be emitted.|
|Source/Flow|<a name="interleave"></a>@ref[interleave](Source-or-Flow/interleave.md)|Emits a specifiable number of elements from the original source, then from the provided source and repeats.|
|Source/Flow|<a name="merge"></a>@ref[merge](Source-or-Flow/merge.md)|Merge multiple sources.|
@ -487,6 +488,7 @@ For more background see the @ref[Error Handling in Streams](../stream-error.md)
* [mergeLatest](Source-or-Flow/mergeLatest.md)
* [mergePreferred](Source-or-Flow/mergePreferred.md)
* [mergePrioritized](Source-or-Flow/mergePrioritized.md)
* [MergeSequence](MergeSequence.md)
* [mergeSorted](Source-or-Flow/mergeSorted.md)
* [monitor](Source-or-Flow/monitor.md)
* [never](Source/never.md)

View file

@ -47,6 +47,7 @@ Akka Streams currently provide these junctions (for a detailed list see the @ref
* @scala[`MergePreferred[In]`]@java[`MergePreferred<In>`] like `Merge` but if elements are available on `preferred` port, it picks from it, otherwise randomly from `others`
* @scala[`MergePrioritized[In]`]@java[`MergePrioritized<In>`] like `Merge` but if elements are available on all input ports, it picks from them randomly based on their `priority`
* @scala[`MergeLatest[In]`]@java[`MergeLatest<In>`] *(N inputs, 1 output)* emits `List[In]`, when i-th input stream emits element, then i-th element in emitted list is updated
* @scala[`MergeSequence[In]`]@java[`MergeSequence<In>`] *(N inputs, 1 output)* emits `List[In]`, where the input streams must represent a partitioned sequence that must be merged back together in order
* @scala[`ZipWith[A,B,...,Out]`]@java[`ZipWith<A,B,...,Out>`] *(N inputs, 1 output)* which takes a function of N inputs that given a value for each input emits 1 output element
* @scala[`Zip[A,B]`]@java[`Zip<A,B>`] *(2 inputs, 1 output)* is a `ZipWith` specialised to zipping input streams of `A` and `B` into a @scala[`(A,B)`]@java[`Pair(A,B)`] tuple stream
* @scala[`Concat[A]`]@java[`Concat<A>`] *(2 inputs, 1 output)* concatenates two streams (first consume one, then the second one)

View file

@ -0,0 +1,87 @@
/*
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
*/
package jdocs.stream.operators;
import akka.actor.ActorSystem;
// #import
import akka.NotUsed;
import akka.japi.Pair;
import akka.stream.ClosedShape;
import akka.stream.UniformFanInShape;
import akka.stream.UniformFanOutShape;
import akka.stream.javadsl.Flow;
import akka.stream.javadsl.GraphDSL;
import akka.stream.javadsl.MergeSequence;
import akka.stream.javadsl.Partition;
import akka.stream.javadsl.RunnableGraph;
import akka.stream.javadsl.Sink;
import akka.stream.javadsl.Source;
// #import
public class MergeSequenceDocExample {
private final ActorSystem system = ActorSystem.create("MergeSequenceDocExample");
interface Message {}
boolean shouldProcess(Message message) {
return true;
}
Source<Message, NotUsed> createSubscription() {
return null;
}
Flow<Pair<Message, Long>, Pair<Message, Long>, NotUsed> createMessageProcessor() {
return null;
}
Sink<Message, NotUsed> createMessageAcknowledger() {
return null;
}
void mergeSequenceExample() {
// #merge-sequence
Source<Message, NotUsed> subscription = createSubscription();
Flow<Pair<Message, Long>, Pair<Message, Long>, NotUsed> messageProcessor =
createMessageProcessor();
Sink<Message, NotUsed> messageAcknowledger = createMessageAcknowledger();
RunnableGraph.fromGraph(
GraphDSL.create(
builder -> {
// Partitions stream into messages that should or should not be processed
UniformFanOutShape<Pair<Message, Long>, Pair<Message, Long>> partition =
builder.add(
Partition.create(2, element -> shouldProcess(element.first()) ? 0 : 1));
// Merges stream by the index produced by zipWithIndex
UniformFanInShape<Pair<Message, Long>, Pair<Message, Long>> merge =
builder.add(MergeSequence.create(2, Pair::second));
builder.from(builder.add(subscription.zipWithIndex())).viaFanOut(partition);
// First goes through message processor
builder.from(partition.out(0)).via(builder.add(messageProcessor)).viaFanIn(merge);
// Second partition bypasses message processor
builder.from(partition.out(1)).viaFanIn(merge);
// Unwrap message index pairs and send to acknowledger
builder
.from(merge.out())
.to(
builder.add(
Flow.<Pair<Message, Long>>create()
.map(Pair::first)
.to(messageAcknowledger)));
return ClosedShape.getInstance();
}))
.run(system);
// #merge-sequence
}
}

View file

@ -0,0 +1,52 @@
/*
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
*/
package docs.stream.operators
import akka.actor.ActorSystem
object MergeSequenceDocExample {
implicit val system: ActorSystem = ???
// #merge-sequence
import akka.NotUsed
import akka.stream.ClosedShape
import akka.stream.scaladsl.{ Flow, GraphDSL, MergeSequence, Partition, RunnableGraph, Sink, Source }
val subscription: Source[Message, NotUsed] = createSubscription()
val messageProcessor: Flow[(Message, Long), (Message, Long), NotUsed] =
createMessageProcessor()
val messageAcknowledger: Sink[Message, NotUsed] = createMessageAcknowledger()
RunnableGraph
.fromGraph(GraphDSL.create() { implicit builder =>
import GraphDSL.Implicits._
// Partitions stream into messages that should or should not be processed
val partition = builder.add(Partition[(Message, Long)](2, {
case (message, _) if shouldProcess(message) => 0
case _ => 1
}))
// Merges stream by the index produced by zipWithIndex
val merge = builder.add(MergeSequence[(Message, Long)](2)(_._2))
subscription.zipWithIndex ~> partition.in
// First goes through message processor
partition.out(0) ~> messageProcessor ~> merge
// Second partition bypasses message processor
partition.out(1) ~> merge
merge.out.map(_._1) ~> messageAcknowledger
ClosedShape
})
.run()
// #merge-sequence
def shouldProcess(message: Message): Boolean = true
trait Message
def createSubscription(): Source[Message, NotUsed] = ???
def createMessageProcessor(): Flow[(Message, Long), (Message, Long), NotUsed] = ???
def createMessageAcknowledger(): Sink[Message, NotUsed] = ???
}