[
https://issues.apache.org/jira/browse/KAFKA-18762?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17925742#comment-17925742
]
Colin McCabe commented on KAFKA-18762:
--------------------------------------
As [~dajac] said, this isn't really about KRaft, but about KIP-966. The
behavior of trunk is also the same as the behavior of 3.9.0. So it cannot be a
regression in 4.0.
The issue boils down to: what do we do when someone sets an "unreasonable"
value for min.insync.replicas? In other words a value that's greater than the
existing number of replicas. Our options are basically:
1. We could just not allow acks=all to work at all with the unreasonable value
2. We could just take max(min.insync.replicas, actual_number_of_replicas)
3. We could try to police configuration better so as to not end up in this
situation
Note that #3 isn't a full solution since people who upgrade from earlier
versions can always have the "unreasonable" configurations in place. Also, in
general, it's just safer to handle all possible configuration values.
The reason why we chose #2 rather than #1 was because #1 would have blocked the
high water mark from advancing at all in these misconfigured topics. This would
be a significant regression for people who had been successfully using them
with acks=0 or acks=1. We also didn't see any actual advantages to #1. Nobody
actually wants a topic to be non-functional with acks=all; it provides no
benefit.
I agree that we could improve validation when creating topics, to ensure that
min.insync.replicas was less than or equal to the actual number of replicas.
Keep in mind, though, if we did that, someone would file a JIRA just like this
one, saying "I used to be able to create a topic with min.insync.replicas = 2
and 1 replica, and now I can't, please fix" :)
There is also the fact that we have that cluster-level min.insync.replicas
configuration, which in many cases is set to 2. Does that mean creating
single-partition topics is completely forbidden in such a cluster, unless they
have a topic-level override? Or what if the old cluster-level
min.insync.replicas was 1, but now it's 2? What happens to all the
single-partition topics? We could also talk about topics where some partitions
have different numbers of replicas.
In my opinion the current behavior seems like the best tradeoff we could make.
> 'Acks=all' not working correctly in Kafka Kraft mode.
> -----------------------------------------------------
>
> Key: KAFKA-18762
> URL: https://issues.apache.org/jira/browse/KAFKA-18762
> Project: Kafka
> Issue Type: Bug
> Components: kraft
> Affects Versions: 3.9.0
> Reporter: Shubham Raj
> Priority: Critical
>
> In Kraft mode, producers are able to produce messages to a topic with a
> replication factor of 1, even when the {{min.insync.replicas}} setting is 2
> and {{{}acks=all{}}}. This behavior is inconsistent with Zookeeper mode,
> where the production fails with a timeout exception as expected. This was
> observed in Kafka 3.9.0. We tried both Java and Python API.
>
> *Steps to Reproduce:*
> # Set up a Kafka cluster in Kraft mode.
> # Create a topic with the following properties:
> ** Replication Factor: 1
> ** Partitions: 1 (for simplicity)
> # Set the {{min.insync.replicas}} configuration for the cluster to 2.
> # Configure a Kafka producer with {{{}acks=all{}}}.
> # Attempt to produce messages to the topic.
>
>
--
This message was sent by Atlassian Jira
(v8.20.10#820010)