Hi Mohan,
Please find my inline comments below.

>One small clarification regarding the proposal. I understand how Solution
(1) enables the other replicas to be chosen as the leader. But it is
possible that the other replicas may not be in sync yet and if unclean
leader election is not enabled, the other replicas may not become the
leader right ?

If other replicas are already out of sync for other reasons then
unclean leader election needs to be enabled to choose a leader.
Solution (1) allows the existing insync replicas to be insync even
though the leader takes longer to process their existing fetch
requests.


>It is not clear to me whether Solution 2 can happen independently. For
example, if the leader exceeds *leader.fetch.process.time.max.ms
<http://leader.fetch.process.time.max.ms>* due to a transient condition,
should it relinquish leadership immediately ? That might be aggressive in
some cases. Detecting that a leader is slow cannot be determined by just
one occurrence, right ?

Solution(2) is an extension to Solution(1) as mentioned earlier in the
KIP. This config is applicable only if
`follower.fetch.pending.reads.insync.enable` is set as true. I have
also updated the config description in the KIP to make that clear.
In our observations, we do not always see this behavior continuously.
It occurs intermittently and makes all the other requests pile up in
the request queue. Sometimes, the broker goes down and makes the
partitions offline.  Users need to set the config based on their
host's configuration and behavior. We can also think about extending
this config based on others observations.

Thanks,
Satish.

On Mon, 28 Jun 2021 at 04:36, Mohan Parthasarathy <mposde...@gmail.com> wrote:
>
> Hi Satish,
>
> One small clarification regarding the proposal. I understand how Solution
> (1) enables the other replicas to be chosen as the leader. But it is
> possible that the other replicas may not be in sync yet and if unclean
> leader election is not enabled, the other replicas may not become the
> leader right ?
>
>  It is not clear to me whether Solution 2 can happen independently. For
> example, if the leader exceeds *leader.fetch.process.time.max.ms
> <http://leader.fetch.process.time.max.ms>* due to a transient condition,
> should it relinquish leadership immediately ? That might be aggressive in
> some cases. Detecting that a leader is slow cannot be determined by just
> one occurrence, right ?
>
> Thanks
> Mohan
>
>
> On Sun, Jun 27, 2021 at 4:01 AM Satish Duggana <satish.dugg...@gmail.com>
> wrote:
>
> > Hi Dhruvil,
> > Thanks for looking into the KIP and providing your comments.
> >
> > There are two problems about the scenario raised in this KIP:
> >
> > a) Leader is slow and it is not available for reads or writes.
> > b) Leader is causing the followers to be out of sync and cause the
> > partitions unavailability.
> >
> > (a) should be detected and mitigated so that the broker can become a
> > leader or replace with a different node if this node continues having
> > issues.
> >
> > (b) will cause the partition to go under minimum ISR and eventually
> > make that partition offline if the leader goes down. In this case,
> > users have to enable unclean leader election for making the partition
> > available. This may cause data loss based on the replica chosen as a
> > leader. This is what several folks(including us) observed in their
> > production environments.
> >
> > Solution(1) in the KIP addresses (b) to avoid offline partitions by
> > not removing the replicas from ISR. This allows the partition to be
> > available if the leader is moved to one of the other replicas in ISR.
> >
> > Solution (2) in the KIP extends solution (1) by relinquishing the
> > leadership and allowing one of the other insync replicas to become a
> > leader.
> >
> > ~Satish.
> >

Reply via email to