diff --git a/docs/streams/developer-guide/dsl-api.html b/docs/streams/developer-guide/dsl-api.html index db4940fb273..79c1ec6a662 100644 --- a/docs/streams/developer-guide/dsl-api.html +++ b/docs/streams/developer-guide/dsl-api.html @@ -863,6 +863,25 @@ +
Repartition
+Manually trigger repartitioning of the stream with desired number of partitions. (details)
+ +repartition() is similar to through() however Kafka Streams will manage the topic for you.
+ Generated topic is treated as internal topic, as a result data will be purged automatically as any other internal repartition topic.
+ In addition, you can specify the desired number of partitions, which allows to easily scale in/out downstream sub-topologies.
+ repartition() operation always triggers repartitioning of the stream, as a result it can be used with embedded Processor API methods (like transform() et al.) that do not trigger auto repartitioning when key changing operation is performed beforehand.
+
+ KStream<byte[], String> stream = ... ;
+KStream<byte[], String> repartitionedStream = stream.repartition(Repartitioned.numberOfPartitions(10));"exactly_once_beta".
Note that you need brokers with version 2.5 or newer to use this feature.
+
+ As of 2.6.0 Kafka Streams offers a new KStream.repartition() operator (as per KIP-221).
+ KStream.repartition() is similar to KStream.through(), however Kafka Streams will manage the topic for you.
+ Refer to the developer guide for more details.
+