diff --git a/ci.py b/ci.py
index 8349b8cafd19f..1dd550310dcc5 100644
--- a/ci.py
+++ b/ci.py
@@ -23,7 +23,7 @@
import subprocess
import sys
-from confluent.ci.scripts.ci_utils import run_cmd, regex_replace, replace
+from ci_tools.utils import run_cmd, regex_replace, replace
logging.basicConfig(level=logging.INFO, format='%(message)s')
log = logging.getLogger(__name__)
diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java
index 08fad4535685e..77c476b49462d 100755
--- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java
@@ -181,7 +181,8 @@ public class TopicConfig {
"When used together, 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/config/log4j2.yaml b/config/log4j2.yaml index 49bcf78d136c6..de263c57c928e 100644 --- a/config/log4j2.yaml +++ b/config/log4j2.yaml @@ -44,7 +44,7 @@ Configuration: # State Change appender - name: StateChangeAppender fileName: "${sys:kafka.logs.dir}/state-change.log" - filePattern: "${sys:kafka.logs.dir}/stage-change.log.%d{yyyy-MM-dd-HH}" + filePattern: "${sys:kafka.logs.dir}/state-change.log.%d{yyyy-MM-dd-HH}" PatternLayout: pattern: "${logPattern}" TimeBasedTriggeringPolicy: 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.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"null86400000nullunstable.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.
+ After the new feature is configured, check
+ kafka-features.sh --bootstrap-server localhost:9092 describe
+ and `streams.version` should now have FinalizedVersionLevel 1.
diff --git a/docs/toc.html b/docs/toc.html index 032228de3d219..304bd1c8a53c1 100644 --- a/docs/toc.html +++ b/docs/toc.html @@ -153,6 +153,7 @@
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.
+ 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.
For a rolling upgrade:
@@ -95,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 @@ -119,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.
+