This is an automated email from the ASF dual-hosted git repository.

mjsax pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 6f946d5026e MINOR: in min.insync.replicas config doc, explicitly state 
that all ISR must ack when acks=all (#19973)
6f946d5026e is described below

commit 6f946d5026eafeca10cd271ff60ddbd0c856d143
Author: Dave Troiano <[email protected]>
AuthorDate: Mon Jun 16 21:40:54 2025 -0400

    MINOR: in min.insync.replicas config doc, explicitly state that all ISR 
must ack when acks=all (#19973)
    
    Clarify the interaction of `min.insync.replicas` and `ack=all`
    configuration.  Prior to this change, the doc for `min.insync.replicas`
    could have been  interpreted as being used to short-circuit in the
    `acks=all` case as if  it would be enough if `min.inscyn.replicas`
    number of brokers replicated  a message before it can be acknowledged
    back to the producer.
    
    Reviewers: Matthias J. Sax <[email protected]>
    
    ---------
    
    Co-authored-by: Matthias J. Sax <[email protected]>
---
 .../main/java/org/apache/kafka/common/config/TopicConfig.java | 11 +++++++----
 1 file changed, 7 insertions(+), 4 deletions(-)

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 3fcea1968b8..08fad453568 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
@@ -169,10 +169,13 @@ public class TopicConfig {
          "to trigger the unclean leader election immediately if needed.</p>";
 
     public static final String MIN_IN_SYNC_REPLICAS_CONFIG = 
"min.insync.replicas";
-    public static final String MIN_IN_SYNC_REPLICAS_DOC = "When a producer 
sets acks to \"all\" (or \"-1\"), " +
-        "this configuration specifies the minimum number of replicas that must 
acknowledge " +
-        "a write for the write to be considered successful. If this minimum 
cannot be met, " +
-        "then the producer will raise an exception (either 
<code>NotEnoughReplicas</code> or 
<code>NotEnoughReplicasAfterAppend</code>).<br> " +
+    public static final String MIN_IN_SYNC_REPLICAS_DOC = "Specifies the 
<i>minimum</i> number of in-sync replicas (including the leader) " +
+        "required for a write to succeed when a producer sets 
<code>acks</code> to \"all\" (or \"-1\"). In the <code>acks=all</code> " +
+        "case, every in-sync replica must acknowledge a write for it to be 
considered successful. E.g., if a topic has " +
+        "<code>replication.factor</code> of 3 and the ISR set includes all 
three replicas, then all three replicas must acknowledge an " +
+        "<code>acks=all</code> write for it to succeed, even if 
<code>min.insync.replicas</code> happens to be less than 3. " +
+        "If <code>acks=all</code> and the current ISR set contains fewer than 
<code>min.insync.replicas</code> members, then the producer " +
+        "will raise an exception (either <code>NotEnoughReplicas</code> or 
<code>NotEnoughReplicasAfterAppend</code>).<br> " +
         "Regardless of the <code>acks</code> setting, the messages will not be 
visible to the consumers until " +
         "they are replicated to all in-sync replicas and the 
<code>min.insync.replicas</code> condition is met.<br> " +
         "When used together, <code>min.insync.replicas</code> and 
<code>acks</code> allow you to enforce greater durability guarantees. " +

Reply via email to