From 83e37a93174bf1ca513f00efcdf49b77d0b26b14 Mon Sep 17 00:00:00 2001
From: souvikrakshit Early Access of the Streams Rebalance Protocol
Set unstable.feature.versions.enable=true for controllers and brokers, and
set unstable.api.versions.enable=true on the brokers as well. In your Kafka Streams application
configuration, set group.protocol=streams.
+ Upgrade the `streams.version` feature to 1. To do this, run
+ kafka-features.sh --bootstrap-server localhost:9092 upgrade --feature streams.version=1
+ to upgrade, or use Admin#updateFeatures API to update `streams.version`. After the feature is updated, check
+ kafka-features.sh --bootstrap-server localhost:9092 describe
+ and `streams.version` should now have FinalizedVersionLevel 1.
From 9c83c6d1f30445267fadb51e861db8c00224518f Mon Sep 17 00:00:00 2001
From: lucliu1108 Early Access of the Streams Rebalance Protocol
Set unstable.feature.versions.enable=true for controllers and brokers, and
set unstable.api.versions.enable=true on the brokers as well. In your Kafka Streams application
configuration, set group.protocol=streams.
- Upgrade the `streams.version` feature to 1. To do this, run
- kafka-features.sh --bootstrap-server localhost:9092 upgrade --feature streams.version=1
- to upgrade, or use Admin#updateFeatures API to update `streams.version`. After the feature is updated, check
+ After the new feature is configured, check
kafka-features.sh --bootstrap-server localhost:9092 describe
and `streams.version` should now have FinalizedVersionLevel 1.
min.insync.replicas and acks allow you to enforce greater durability guarantees. " +
"A typical scenario would be to create a topic with a replication factor of 3, " +
"set min.insync.replicas to 2, and produce with acks of \"all\". " +
- "This will ensure that a majority of replicas must persist a write before it's considered successful by the producer and it's visible to consumers.";
+ "This ensures that a majority of replicas must persist a write before it's considered successful by the producer and it's visible to consumers." +
+ "Note that when the Eligible Leader Replicas feature is enabled, the semantics of this config changes. Please refer to the ELR section for more info.
"; public static final String COMPRESSION_TYPE_CONFIG = "compression.type"; public static final String COMPRESSION_TYPE_DOC = "Specify the final compression type for a given topic. " + diff --git a/docs/ops.html b/docs/ops.html index 529bce159bb1e..be0bfe89e8bf9 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -4499,9 +4499,16 @@The ELR fields can be checked through the API DescribeTopicPartitions. The admin client can fetch the ELR info by describing the topics.
- Also note that, if min.insync.replicas is updated for a topic, the ELR field will be cleaned. If cluster default min ISR is updated,
- all the ELR fields will be cleaned.
The ELR fields can be checked through the API DescribeTopicPartitions. The admin client can fetch the ELR info by describing the topics.
+Note that when the ELR feature is enabled:
+min.insync.replicas config will be added if there is not any. The value is the same as the static config in the active controller.min.insync.replicas config at the cluster-level is not allowed.min.insync.replicas is updated, even if the value is unchanged, all the ELR state will be cleaned.min.insync.replicas value at the broker-level config will be removed. Please set at the cluster-level if necessary.min.insync.replicas config at the broker-level is not allowed.min.insync.replicas is updated for a topic, the ELR state will be cleaned.min.insync.replicas value at the broker-level config will be removed.
+ Please set at the cluster-level if necessary.
For further details, please refer to here.
kafka.log.LogCleaner to org.apache.kafka.storage.internals.log.LogCleaner in the log4j2.yaml configuration file.
Added loggers for org.apache.kafka.storage.internals.log.LogCleaner$CleanerThread and org.apache.kafka.storage.internals.log.Cleaner classes to CleanerAppender.
+ state-change.log files has been updated from stage-change.log.[date] to state-change.log.[date] in the log4j2.yaml configuration file.
+ For a rolling upgrade:
@@ -97,7 +100,7 @@Note: Apache Kafka 4.0 only supports KRaft mode - ZooKeeper mode has been removed. As such, broker upgrades to 4.0.0 (and higher) require KRaft mode and the software and metadata versions must be at least 3.3.x (the first version when KRaft mode was deemed production ready). For clusters in KRaft mode @@ -121,7 +124,13 @@
IBP_4_0_IV1(23, "4.0", "IV1", true) means this version has metadata changes).
Given your current and target versions, a downgrade is only possible if there are no metadata changes in the versions between.
-
+ state-change.log files has been updated from stage-change.log.[date] to state-change.log.[date] in the log4j2.yaml configuration file.
+ See KAFKA-19576 for details.
+ state-change.log files incorrectly rotates to stage-change.log.[date] (changing state to stage). This issue is corrected in 4.0.1.
+ See KAFKA-19576 for details.
+ 30000 (30 seconds)30000 (30 seconds) (at-least-once) / 100 (exactly-once)DeserializationExceptionHandler interface.LogAndContinueExceptionHandlerLogAndFailExceptionHandlerdsl.store.suppliers.class
+ The default state store type used by DSL operators.
"ROCKS_DB"30000 (30 seconds)100null600000/${java.io.tmpdir}/kafka-streamsTaskAssignor interface.0 ms, a task would raise an error for the first internal error. For any timeout larger than 0 ms, a task will retry at least once before an error is raised.300000StreamsConfig.NO_OPTIMIZATION (none), StreamsConfig.OPTIMIZE (all) or a comma separated list of specific optimizations: StreamsConfig.REUSE_KTABLE_SOURCE_TOPICS (reuse.ktable.source.topics), StreamsConfig.MERGE_REPARTITION_TOPICS (merge.repartition.topics),
StreamsConfig.SINGLE_STORE_SELF_JOIN (single.store.self.join). "NO_OPTIMIZATION"null86400000null