From 30d1989db139ece5e5c7ecec0c4177e008e14003 Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Tue, 16 Nov 2021 17:34:49 -0800 Subject: [PATCH] MINOR: update Kafka Streams standby task config (#11404) Reviewers: A. Sophie Blee-Goldman , Antony Stubbs , James Galasyn --- .../developer-guide/config-streams.html | 223 ++++++++++-------- .../apache/kafka/streams/StreamsConfig.java | 10 +- 2 files changed, 127 insertions(+), 106 deletions(-) diff --git a/docs/streams/developer-guide/config-streams.html b/docs/streams/developer-guide/config-streams.html index 9dd096a427c..dd9298d053e 100644 --- a/docs/streams/developer-guide/config-streams.html +++ b/docs/streams/developer-guide/config-streams.html @@ -55,43 +55,47 @@ settings.put(... , ...);

This section contains the most common Streams configuration parameters. For a full reference, see the Streams Javadocs.

@@ -146,6 +150,74 @@ settings.put(... , ...); +

Optional configuration parameters

Here are the optional Streams javadocs, sorted by level of importance:

@@ -280,7 +352,7 @@ settings.put(... , ...); 30000 milliseconds (30 seconds) num.standby.replicas - Medium + High The number of standby replicas for each task. 0 @@ -680,14 +752,19 @@ streamsConfiguration.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, preferred to restart on an instance that has standby replicas so that the local state store restoration process from its changelog can be minimized. Details about how Kafka Streams makes use of the standby replicas to minimize the cost of resuming tasks on failover can be found in the State section. +
+
Recommendation:
+
Increase the number of standbys to 1 to get instant fail-over, i.e., high-availability. + Increasing the number of standbys requires more client-side storage space. + For example, with 1 standby, 2x space is required.
+
+
+
Note:
+
If you enable n standby tasks, you need to provision n+1 KafkaStreams instances.
+
-
-

Note

-

If you enable n standby tasks, you need to provision n+1 KafkaStreams - instances.

-

num.stream.threads

@@ -755,7 +832,7 @@ streamsConfiguration.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG,
Recommendation:
Increase the replication factor to 3 to ensure that the internal Kafka Streams topic can tolerate up to 2 broker failures. - Note that you will require more storage space as well (3 times more with the replication factor of 3).
+ Note that you will require more storage space as well (3x with the replication factor of 3).
@@ -953,19 +1030,19 @@ streamsSettings.put(StreamsConfig.topicPrefix("PARAMETER_NAME"), "topic-value"); allow.auto.create.topics Consumer - false + false auto.offset.reset Consumer - earliest + earliest linger.ms Producer - 100 + 100 max.poll.records Consumer - 1000 + 1000 @@ -976,62 +1053,6 @@ streamsSettings.put(StreamsConfig.topicPrefix("PARAMETER_NAME"), "topic-value"); value to false. Consumers will only commit explicitly via commitSync calls when the Kafka Streams library or a user decides to commit the current processing state. - diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 2ebd5543ad0..6fdfc8f4053 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -620,6 +620,11 @@ public class StreamsConfig extends AbstractConfig { Type.LIST, Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) + .define(NUM_STANDBY_REPLICAS_CONFIG, + Type.INT, + 0, + Importance.HIGH, + NUM_STANDBY_REPLICAS_DOC) .define(STATE_DIR_CONFIG, Type.STRING, System.getProperty("java.io.tmpdir") + File.separator + "kafka-streams", @@ -701,11 +706,6 @@ public class StreamsConfig extends AbstractConfig { atLeast(1), Importance.MEDIUM, MAX_WARMUP_REPLICAS_DOC) - .define(NUM_STANDBY_REPLICAS_CONFIG, - Type.INT, - 0, - Importance.MEDIUM, - NUM_STANDBY_REPLICAS_DOC) .define(NUM_STREAM_THREADS_CONFIG, Type.INT, 1,