Add PartitonHub, #21880

* FixedSizePartitionHub in Artery
* expose consumer queue size
This commit is contained in:
Patrik Nordwall 2017-06-05 18:36:01 +02:00
parent 3ba093d27e
commit 945ade245e
14 changed files with 1449 additions and 29 deletions

View file

@ -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 }

View file

@ -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();
}
}

View file

@ -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
}
}
}