You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Paolo Moriello <pa...@gmail.com> on 2020/02/27 19:06:23 UTC

[DISCUSS] KAFKA-4680: min.insync.replica can be set > replication factor

Hello,

I'd like to take up this Jira ticket
<https://issues.apache.org/jira/browse/KAFKA-4680>. This is an old ticket,
marked as a Kafka bug.

Before moving forward, I'd like to open a discussion on what would be the
best approach to take on when doing the validation, as well as discuss
about possible edge cases we should consider.

Basically, at the moment, it is possible to specify min.insync.replicas >
replication factor. When this happens, it is not possible to produce on a
topic when acks=all as client callback returns NOT_ENOUGH_REPLICAS, and the
broker logs error messages on each request. As suggested in the Jira, the
validation should happen much earlier in the process, eg. at topic
creation/configuration setup.

Regarding the approach to use on validating the configuration; do we want,
for instance, to:
1. print a WARN about the mismatch in the configuration
2. make the request FAIL
3. or print a more specific message on produce

Options 1 and 2 anticipate the validation on topic creation / configuration
change. These require to validate the configuration in more than one place:
at topic creation, at configuration setup/update (both for
min.insync.replicas and the default.replication.factor), at partition
reassignment (when reducing replication factor). Don't know about
consequences
Option 3 is simpler; it does not anticipate the validation, but at least
improves the visibility over the issue on the client side.

I'd be in favor of a softer approach, which might include both printing a
warning on topic creation/configuration-update and eventually a more
specific message when producing on the topic. On the other end, this does
not solve the problem, as we would allow anyway the mismatch in the
configuration. Option 2 would solve the problem with an harder validation
(eg blocking topic creation or configuration setup), but this requires to
validate any possible edge case (eg. how do we prevent a change in
min.insync.replicas if we have already created topic with lower replication
factor?).

Let me know what's your opinion on this, and if there is any other scenario
we should consider for the validation (for instance, what's the impact on
internal topics?).

Thanks,
Paolo

Re: [DISCUSS] KAFKA-4680: min.insync.replica can be set > replication factor

Posted by Alexandre Dupriez <al...@gmail.com>.
Thanks Paolo for taking care of this.

I think option (3) would be the closest to iso-functionality. I wonder
how useful it could be to always add the replication factor of the
topic in the error message [1] versus returning a modified one fitted
for this specific case?

[1] https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/cluster/Partition.scala#L971-L975

Le jeu. 27 févr. 2020 à 19:06, Paolo Moriello
<pa...@gmail.com> a écrit :
>
> Hello,
>
> I'd like to take up this Jira ticket
> <https://issues.apache.org/jira/browse/KAFKA-4680>. This is an old ticket,
> marked as a Kafka bug.
>
> Before moving forward, I'd like to open a discussion on what would be the
> best approach to take on when doing the validation, as well as discuss
> about possible edge cases we should consider.
>
> Basically, at the moment, it is possible to specify min.insync.replicas >
> replication factor. When this happens, it is not possible to produce on a
> topic when acks=all as client callback returns NOT_ENOUGH_REPLICAS, and the
> broker logs error messages on each request. As suggested in the Jira, the
> validation should happen much earlier in the process, eg. at topic
> creation/configuration setup.
>
> Regarding the approach to use on validating the configuration; do we want,
> for instance, to:
> 1. print a WARN about the mismatch in the configuration
> 2. make the request FAIL
> 3. or print a more specific message on produce
>
> Options 1 and 2 anticipate the validation on topic creation / configuration
> change. These require to validate the configuration in more than one place:
> at topic creation, at configuration setup/update (both for
> min.insync.replicas and the default.replication.factor), at partition
> reassignment (when reducing replication factor). Don't know about
> consequences
> Option 3 is simpler; it does not anticipate the validation, but at least
> improves the visibility over the issue on the client side.
>
> I'd be in favor of a softer approach, which might include both printing a
> warning on topic creation/configuration-update and eventually a more
> specific message when producing on the topic. On the other end, this does
> not solve the problem, as we would allow anyway the mismatch in the
> configuration. Option 2 would solve the problem with an harder validation
> (eg blocking topic creation or configuration setup), but this requires to
> validate any possible edge case (eg. how do we prevent a change in
> min.insync.replicas if we have already created topic with lower replication
> factor?).
>
> Let me know what's your opinion on this, and if there is any other scenario
> we should consider for the validation (for instance, what's the impact on
> internal topics?).
>
> Thanks,
> Paolo