Add PartitonHub, #21880
* FixedSizePartitionHub in Artery * expose consumer queue size
This commit is contained in:
parent
3ba093d27e
commit
945ade245e
14 changed files with 1449 additions and 29 deletions
|
|
@ -74,7 +74,7 @@ before any materialization takes place.
|
|||
|
||||
@@@
|
||||
|
||||
## Dynamic fan-in and fan-out with MergeHub and BroadcastHub
|
||||
## Dynamic fan-in and fan-out with MergeHub, BroadcastHub and PartitionHub
|
||||
|
||||
There are many cases when consumers or producers of a certain service (represented as a Sink, Source, or possibly Flow)
|
||||
are dynamic and not known in advance. The Graph DSL does not allow to represent this, all connections of the graph
|
||||
|
|
@ -169,4 +169,71 @@ Scala
|
|||
: @@snip [HubsDocSpec.scala]($code$/scala/docs/stream/HubsDocSpec.scala) { #pub-sub-4 }
|
||||
|
||||
Java
|
||||
: @@snip [HubDocTest.java]($code$/java/jdocs/stream/HubDocTest.java) { #pub-sub-4 }
|
||||
: @@snip [HubDocTest.java]($code$/java/jdocs/stream/HubDocTest.java) { #pub-sub-4 }
|
||||
|
||||
### Using the PartitionHub
|
||||
|
||||
**This is a @ref:[may change](../common/may-change.md) feature***
|
||||
|
||||
A `PartitionHub` can be used to route elements from a common producer to a dynamic set of consumers.
|
||||
The selection of consumer is done with a function. Each element can be routed to only one consumer.
|
||||
|
||||
The rate of the producer will be automatically adapted to the slowest consumer. In this case, the hub is a `Sink`
|
||||
to which the single producer must be attached first. Consumers can only be attached once the `Sink` has
|
||||
been materialized (i.e. the producer has been started). One example of using the `PartitionHub`:
|
||||
|
||||
Scala
|
||||
: @@snip [HubsDocSpec.scala]($code$/scala/docs/stream/HubsDocSpec.scala) { #partition-hub }
|
||||
|
||||
Java
|
||||
: @@snip [HubDocTest.java]($code$/java/jdocs/stream/HubDocTest.java) { #partition-hub }
|
||||
|
||||
The `partitioner` function takes two parameters; the first is the number of active consumers and the second
|
||||
is the stream element. The function should return the index of the selected consumer for the given element,
|
||||
i.e. `int` greater than or equal to 0 and less than number of consumers.
|
||||
|
||||
The resulting `Source` can be materialized any number of times, each materialization effectively attaching
|
||||
a new consumer. If there are no consumers attached to this hub then it will not drop any elements but instead
|
||||
backpressure the upstream producer until consumers arrive. This behavior can be tweaked by using the combinators
|
||||
`.buffer` for example with a drop strategy, or just attaching a consumer that drops all messages. If there
|
||||
are no other consumers, this will ensure that the producer is kept drained (dropping all elements) and once a new
|
||||
consumer arrives and messages are routed to the new consumer it will adaptively slow down, ensuring no more messages
|
||||
are dropped.
|
||||
|
||||
It is possible to define how many initial consumers that are required before it starts emitting any messages
|
||||
to the attached consumers. While not enough consumers have been attached messages are buffered and when the
|
||||
buffer is full the upstream producer is backpressured. No messages are dropped.
|
||||
|
||||
The above example illustrate a stateless partition function. For more advanced stateful routing the @java[`ofStateful`]
|
||||
@scala[`statefulSink`] can be used. Here is an example of a stateful round-robin function:
|
||||
|
||||
Scala
|
||||
: @@snip [HubsDocSpec.scala]($code$/scala/docs/stream/HubsDocSpec.scala) { #partition-hub-stateful }
|
||||
|
||||
Java
|
||||
: @@snip [HubDocTest.java]($code$/java/jdocs/stream/HubDocTest.java) { #partition-hub-stateful }
|
||||
|
||||
Note that it is a factory of a function to to be able to hold stateful variables that are
|
||||
unique for each materialization. @java[In this example the `partitioner` function is implemented as a class to
|
||||
be able to hold the mutable variable. A new instance of `RoundRobin` is created for each materialization of the hub.]
|
||||
|
||||
@@@ div { .group-java }
|
||||
@@snip [HubDocTest.java]($code$/java/jdocs/stream/HubDocTest.java) { #partition-hub-stateful-function }
|
||||
@@@
|
||||
|
||||
The function takes two parameters; the first is information about active consumers, including an array of
|
||||
consumer identifiers and the second is the stream element. The function should return the selected consumer
|
||||
identifier for the given element. The function will never be called when there are no active consumers, i.e.
|
||||
there is always at least one element in the array of identifiers.
|
||||
|
||||
Another interesting type of routing is to prefer routing to the fastest consumers. The `ConsumerInfo`
|
||||
has an accessor `queueSize` that is approximate number of buffered elements for a consumer.
|
||||
Larger value than other consumers could be an indication of that the consumer is slow.
|
||||
Note that this is a moving target since the elements are consumed concurrently. Here is an example of
|
||||
a hub that routes to the consumer with least buffered elements:
|
||||
|
||||
Scala
|
||||
: @@snip [HubsDocSpec.scala]($code$/scala/docs/stream/HubsDocSpec.scala) { #partition-hub-fastest }
|
||||
|
||||
Java
|
||||
: @@snip [HubDocTest.java]($code$/java/jdocs/stream/HubDocTest.java) { #partition-hub-fastest }
|
||||
|
|
|
|||
|
|
@ -11,17 +11,25 @@ import akka.japi.Pair;
|
|||
import akka.stream.ActorMaterializer;
|
||||
import akka.stream.KillSwitches;
|
||||
import akka.stream.Materializer;
|
||||
import akka.stream.ThrottleMode;
|
||||
import akka.stream.UniqueKillSwitch;
|
||||
import akka.stream.javadsl.*;
|
||||
import akka.stream.javadsl.PartitionHub.ConsumerInfo;
|
||||
|
||||
import jdocs.AbstractJavaTest;
|
||||
import akka.testkit.javadsl.TestKit;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import scala.concurrent.duration.Duration;
|
||||
import scala.concurrent.duration.FiniteDuration;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.BiFunction;
|
||||
import java.util.function.Supplier;
|
||||
import java.util.function.ToLongBiFunction;
|
||||
|
||||
public class HubDocTest extends AbstractJavaTest {
|
||||
|
||||
|
|
@ -137,4 +145,136 @@ public class HubDocTest extends AbstractJavaTest {
|
|||
killSwitch.shutdown();
|
||||
//#pub-sub-4
|
||||
}
|
||||
|
||||
@Test
|
||||
public void dynamicPartition() {
|
||||
// Used to be able to clean up the running stream
|
||||
ActorMaterializer materializer = ActorMaterializer.create(system);
|
||||
|
||||
//#partition-hub
|
||||
// A simple producer that publishes a new "message-n" every second
|
||||
Source<String, Cancellable> producer = Source.tick(
|
||||
FiniteDuration.create(1, TimeUnit.SECONDS),
|
||||
FiniteDuration.create(1, TimeUnit.SECONDS),
|
||||
"message"
|
||||
).zipWith(Source.range(0, 100), (a, b) -> a + "-" + b);
|
||||
|
||||
// Attach a PartitionHub Sink to the producer. This will materialize to a
|
||||
// corresponding Source.
|
||||
// (We need to use toMat and Keep.right since by default the materialized
|
||||
// value to the left is used)
|
||||
RunnableGraph<Source<String, NotUsed>> runnableGraph =
|
||||
producer.toMat(PartitionHub.of(
|
||||
String.class,
|
||||
(size, elem) -> Math.abs(elem.hashCode()) % size,
|
||||
2, 256), Keep.right());
|
||||
|
||||
// By running/materializing the producer, we get back a Source, which
|
||||
// gives us access to the elements published by the producer.
|
||||
Source<String, NotUsed> fromProducer = runnableGraph.run(materializer);
|
||||
|
||||
// Print out messages from the producer in two independent consumers
|
||||
fromProducer.runForeach(msg -> System.out.println("consumer1: " + msg), materializer);
|
||||
fromProducer.runForeach(msg -> System.out.println("consumer2: " + msg), materializer);
|
||||
//#partition-hub
|
||||
|
||||
// Cleanup
|
||||
materializer.shutdown();
|
||||
}
|
||||
|
||||
//#partition-hub-stateful-function
|
||||
// Using a class since variable must otherwise be final.
|
||||
// New instance is created for each materialization of the PartitionHub.
|
||||
static class RoundRobin<T> implements ToLongBiFunction<ConsumerInfo, T> {
|
||||
|
||||
private long i = -1;
|
||||
|
||||
@Override
|
||||
public long applyAsLong(ConsumerInfo info, T elem) {
|
||||
i++;
|
||||
return info.consumerIdByIdx((int) (i % info.size()));
|
||||
}
|
||||
}
|
||||
//#partition-hub-stateful-function
|
||||
|
||||
@Test
|
||||
public void dynamicStatefulPartition() {
|
||||
// Used to be able to clean up the running stream
|
||||
ActorMaterializer materializer = ActorMaterializer.create(system);
|
||||
|
||||
//#partition-hub-stateful
|
||||
// A simple producer that publishes a new "message-n" every second
|
||||
Source<String, Cancellable> producer = Source.tick(
|
||||
FiniteDuration.create(1, TimeUnit.SECONDS),
|
||||
FiniteDuration.create(1, TimeUnit.SECONDS),
|
||||
"message"
|
||||
).zipWith(Source.range(0, 100), (a, b) -> a + "-" + b);
|
||||
|
||||
// Attach a PartitionHub Sink to the producer. This will materialize to a
|
||||
// corresponding Source.
|
||||
// (We need to use toMat and Keep.right since by default the materialized
|
||||
// value to the left is used)
|
||||
RunnableGraph<Source<String, NotUsed>> runnableGraph =
|
||||
producer.toMat(
|
||||
PartitionHub.ofStateful(
|
||||
String.class,
|
||||
() -> new RoundRobin<String>(),
|
||||
2,
|
||||
256),
|
||||
Keep.right());
|
||||
|
||||
// By running/materializing the producer, we get back a Source, which
|
||||
// gives us access to the elements published by the producer.
|
||||
Source<String, NotUsed> fromProducer = runnableGraph.run(materializer);
|
||||
|
||||
// Print out messages from the producer in two independent consumers
|
||||
fromProducer.runForeach(msg -> System.out.println("consumer1: " + msg), materializer);
|
||||
fromProducer.runForeach(msg -> System.out.println("consumer2: " + msg), materializer);
|
||||
//#partition-hub-stateful
|
||||
|
||||
// Cleanup
|
||||
materializer.shutdown();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void dynamicFastestPartition() {
|
||||
// Used to be able to clean up the running stream
|
||||
ActorMaterializer materializer = ActorMaterializer.create(system);
|
||||
|
||||
//#partition-hub-fastest
|
||||
Source<Integer, NotUsed> producer = Source.range(0, 100);
|
||||
|
||||
// ConsumerInfo.queueSize is the approximate number of buffered elements for a consumer.
|
||||
// Note that this is a moving target since the elements are consumed concurrently.
|
||||
RunnableGraph<Source<Integer, NotUsed>> runnableGraph =
|
||||
producer.toMat(
|
||||
PartitionHub.ofStateful(
|
||||
Integer.class,
|
||||
() -> (info, elem) -> {
|
||||
final List<Object> ids = info.getConsumerIds();
|
||||
int minValue = info.queueSize(0);
|
||||
long fastest = info.consumerIdByIdx(0);
|
||||
for (int i = 1; i < ids.size(); i++) {
|
||||
int value = info.queueSize(i);
|
||||
if (value < minValue) {
|
||||
minValue = value;
|
||||
fastest = info.consumerIdByIdx(i);
|
||||
}
|
||||
}
|
||||
return fastest;
|
||||
},
|
||||
2,
|
||||
8),
|
||||
Keep.right());
|
||||
|
||||
Source<Integer, NotUsed> fromProducer = runnableGraph.run(materializer);
|
||||
|
||||
fromProducer.runForeach(msg -> System.out.println("consumer1: " + msg), materializer);
|
||||
fromProducer.throttle(10, Duration.create(100, TimeUnit.MILLISECONDS), 10, ThrottleMode.shaping())
|
||||
.runForeach(msg -> System.out.println("consumer2: " + msg), materializer);
|
||||
//#partition-hub-fastest
|
||||
|
||||
// Cleanup
|
||||
materializer.shutdown();
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -10,6 +10,7 @@ import akka.testkit.AkkaSpec
|
|||
import docs.CompileOnlySpec
|
||||
|
||||
import scala.concurrent.duration._
|
||||
import akka.stream.ThrottleMode
|
||||
|
||||
class HubsDocSpec extends AkkaSpec with CompileOnlySpec {
|
||||
implicit val materializer = ActorMaterializer()
|
||||
|
|
@ -104,6 +105,86 @@ class HubsDocSpec extends AkkaSpec with CompileOnlySpec {
|
|||
//#pub-sub-4
|
||||
}
|
||||
|
||||
"demonstrate creating a dynamic partition hub" in compileOnlySpec {
|
||||
//#partition-hub
|
||||
// A simple producer that publishes a new "message-" every second
|
||||
val producer = Source.tick(1.second, 1.second, "message")
|
||||
.zipWith(Source(1 to 100))((a, b) => s"$a-$b")
|
||||
|
||||
// Attach a PartitionHub Sink to the producer. This will materialize to a
|
||||
// corresponding Source.
|
||||
// (We need to use toMat and Keep.right since by default the materialized
|
||||
// value to the left is used)
|
||||
val runnableGraph: RunnableGraph[Source[String, NotUsed]] =
|
||||
producer.toMat(PartitionHub.sink(
|
||||
(size, elem) => math.abs(elem.hashCode) % size,
|
||||
startAfterNrOfConsumers = 2, bufferSize = 256))(Keep.right)
|
||||
|
||||
// By running/materializing the producer, we get back a Source, which
|
||||
// gives us access to the elements published by the producer.
|
||||
val fromProducer: Source[String, NotUsed] = runnableGraph.run()
|
||||
|
||||
// Print out messages from the producer in two independent consumers
|
||||
fromProducer.runForeach(msg => println("consumer1: " + msg))
|
||||
fromProducer.runForeach(msg => println("consumer2: " + msg))
|
||||
//#partition-hub
|
||||
}
|
||||
|
||||
"demonstrate creating a dynamic stateful partition hub" in compileOnlySpec {
|
||||
//#partition-hub-stateful
|
||||
// A simple producer that publishes a new "message-" every second
|
||||
val producer = Source.tick(1.second, 1.second, "message")
|
||||
.zipWith(Source(1 to 100))((a, b) => s"$a-$b")
|
||||
|
||||
// New instance of the partitioner function and its state is created
|
||||
// for each materialization of the PartitionHub.
|
||||
def roundRobin(): (PartitionHub.ConsumerInfo, String) ⇒ Long = {
|
||||
var i = -1L
|
||||
|
||||
(info, elem) => {
|
||||
i += 1
|
||||
info.consumerIdByIdx((i % info.size).toInt)
|
||||
}
|
||||
}
|
||||
|
||||
// Attach a PartitionHub Sink to the producer. This will materialize to a
|
||||
// corresponding Source.
|
||||
// (We need to use toMat and Keep.right since by default the materialized
|
||||
// value to the left is used)
|
||||
val runnableGraph: RunnableGraph[Source[String, NotUsed]] =
|
||||
producer.toMat(PartitionHub.statefulSink(
|
||||
() => roundRobin(),
|
||||
startAfterNrOfConsumers = 2, bufferSize = 256))(Keep.right)
|
||||
|
||||
// By running/materializing the producer, we get back a Source, which
|
||||
// gives us access to the elements published by the producer.
|
||||
val fromProducer: Source[String, NotUsed] = runnableGraph.run()
|
||||
|
||||
// Print out messages from the producer in two independent consumers
|
||||
fromProducer.runForeach(msg => println("consumer1: " + msg))
|
||||
fromProducer.runForeach(msg => println("consumer2: " + msg))
|
||||
//#partition-hub-stateful
|
||||
}
|
||||
|
||||
"demonstrate creating a dynamic partition hub routing to fastest consumer" in compileOnlySpec {
|
||||
//#partition-hub-fastest
|
||||
val producer = Source(0 until 100)
|
||||
|
||||
// ConsumerInfo.queueSize is the approximate number of buffered elements for a consumer.
|
||||
// Note that this is a moving target since the elements are consumed concurrently.
|
||||
val runnableGraph: RunnableGraph[Source[Int, NotUsed]] =
|
||||
producer.toMat(PartitionHub.statefulSink(
|
||||
() => (info, elem) ⇒ info.consumerIds.minBy(id ⇒ info.queueSize(id)),
|
||||
startAfterNrOfConsumers = 2, bufferSize = 16))(Keep.right)
|
||||
|
||||
val fromProducer: Source[Int, NotUsed] = runnableGraph.run()
|
||||
|
||||
fromProducer.runForeach(msg => println("consumer1: " + msg))
|
||||
fromProducer.throttle(10, 100.millis, 10, ThrottleMode.Shaping)
|
||||
.runForeach(msg => println("consumer2: " + msg))
|
||||
//#partition-hub-fastest
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue