You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Satish Duggana <sa...@gmail.com> on 2021/06/23 00:29:00 UTC

[DISCUSS] KIP-501 Avoid out-of-sync or offline partitions when follower fetch requests are not processed in time

Hi,
Bumping up the discussion thread on KIP-501 about avoiding out-of-sync or
offline partitions when follower fetch requests are not processed in time
by the leader replica. This issue occurred several times in multiple
production environments (at Uber, Yelp, Twitter, etc).

KIP-501 is located here
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-501+Avoid+out-of-sync+or+offline+partitions+when+follower+fetch+requests+are+not+processed+in+time>.
You may want to look at the earlier mail discussion thread here
<https://mail-archives.apache.org/mod_mbox/kafka-dev/202002.mbox/%3Cpony-9f4e96e457398374499ab892281453dcaa7dc679-11722f366b06d9f46bcb5905ff94fd6ab167598e%40dev.kafka.apache.org%3E>,
and here
<https://mail-archives.apache.org/mod_mbox/kafka-dev/202002.mbox/%3CCAM-aUZnJ4z%2B_ztjF6sXSL61M1me0ogWZ1BV6%2BoV45rJMG8EoZA%40mail.gmail.com%3E>
.

Please take a look, I would like to hear your feedback and suggestions.

Thanks,
Satish.

Re: [DISCUSS] KIP-501 Avoid out-of-sync or offline partitions when follower fetch requests are not processed in time

Posted by Satish Duggana <sa...@gmail.com>.
Hi Jun,
Thanks for looking into the KIP and providing your comments.

>1. For Solution 2, we probably want to be a bit careful with letting each broker automatically relinquish leadership. The danger of doing that is if all brokers start doing the same (say due to increased data volume), the whole cluster could get into a state with no leaders.

Even without this feature, when a broker is flooded with data and it
is not able to serve the follower fetch requests within time, then it
makes other replicas out of sync and it may also cause offline
partitions if that broker goes down.
It looks like the trade off here is about consistency or availability
when the whole cluster is flooded with data. The proposal here is to
relinquish the leadership for that specific partition only but not for
all the partitions hosted by this broker. This will not make a
partition without a leader as we are not making any replica out of
sync including the current leader but we are moving the current leader
to the last element in the assigned replica list. So, the existing
leader will continue to be the leader until the controller processes
the assigned replica.


>2. For Solution 1, I am wondering to what extent it solves the problem. As Lucas mentioned earlier, if the disk is slow, eventually it will slow down the produce requests and delay the processing of the follower fetch requests. Do you know how well this approach works in practice?

As I mentioned in the KIP, this solution has limitations. This
includes subsequent fetch requests getting stuck in the request queue
and the respective replicas may be considered as out of sync. It may
not work well in practice incase of a frequent problem with
disk/memory.

>3. I am thinking that yet another approach is to introduce some kind of pluggable failure detection module to detect individual broker failure. Admins can then build a plugin for their environment, configure replica.lag.max.time.ms that matches how quickly failure can be detected and build tools to determine what to do with detected failures.

It may be difficult for users to implement a plugin and good to
address it internally with a config.

Thanks,
Satish.

On Wed, 7 Jul 2021 at 23:41, Jun Rao <ju...@confluent.io.invalid> wrote:
>
> Hi, Satish,
>
> Thanks for the KIP.
>
> 1. For Solution 2, we probably want to be a bit careful with letting each
> broker automatically relinquish leadership. The danger of doing that is if
> all brokers start doing the same (say due to increased data volume), the
> whole cluster could get into a state with no leaders.
>
> 2. For Solution 1, I am wondering to what extent it solves the problem. As
> Lucas mentioned earlier, if the disk is slow, eventually it will slow down
> the produce requests and delay the processing of the follower fetch
> requests. Do you know how well this approach works in practice?
>
> 3. I am thinking that yet another approach is to introduce some kind of
> pluggable failure detection module to detect individual broker failure.
> Admins can then build a plugin for their environment, configure
> replica.lag.max.time.ms that matches how quickly failure can be detected
> and build tools to determine what to do with detected failures.
>
> Jun
>
> On Tue, Jun 29, 2021 at 11:49 PM Satish Duggana <sa...@gmail.com>
> wrote:
>
> > > That clarification in the document helps. But then setting the first
> > option
> > > to true does not necessarily mean that the condition is happening. Did
> > you
> > > mean to say that relinquish the leadership if it is taking longer than
> > > leader.fetch.process.time.max.ms AND there are fetch requests pending
> > which
> > > are >= log-end-offset of the earlier fetch request ?
> >
> > Right. This config triggers relinquishing the leadership only for the
> > mentioned cases in the KIP.
> >
> > Thanks,
> > Satish.
> >
> > On Mon, 28 Jun 2021 at 23:11, Mohan Parthasarathy <mp...@gmail.com>
> > wrote:
> > >
> > > Hi Satish,
> > >
> > >
> > > >
> > > >
> > > > >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.
> > > >
> > > >
> > > That clarification in the document helps. But then setting the first
> > option
> > > to true does not necessarily mean that the condition is happening. Did
> > you
> > > mean to say that relinquish the leadership if it is taking longer than
> > > leader.fetch.process.time.max.ms AND there are fetch requests pending
> > which
> > > are >= log-end-offset of the earlier fetch request ?
> > >
> > > -Thanks
> > > Mohan
> > >
> > > > Thanks,
> > > > Satish.
> > > >
> > > > On Mon, 28 Jun 2021 at 04:36, Mohan Parthasarathy <mposdev21@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.duggana@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.
> > > > > >
> > > >
> >

Re: [DISCUSS] KIP-501 Avoid out-of-sync or offline partitions when follower fetch requests are not processed in time

Posted by Jun Rao <ju...@confluent.io.INVALID>.
Hi, Satish,

Thanks for the KIP.

1. For Solution 2, we probably want to be a bit careful with letting each
broker automatically relinquish leadership. The danger of doing that is if
all brokers start doing the same (say due to increased data volume), the
whole cluster could get into a state with no leaders.

2. For Solution 1, I am wondering to what extent it solves the problem. As
Lucas mentioned earlier, if the disk is slow, eventually it will slow down
the produce requests and delay the processing of the follower fetch
requests. Do you know how well this approach works in practice?

3. I am thinking that yet another approach is to introduce some kind of
pluggable failure detection module to detect individual broker failure.
Admins can then build a plugin for their environment, configure
replica.lag.max.time.ms that matches how quickly failure can be detected
and build tools to determine what to do with detected failures.

Jun

On Tue, Jun 29, 2021 at 11:49 PM Satish Duggana <sa...@gmail.com>
wrote:

> > That clarification in the document helps. But then setting the first
> option
> > to true does not necessarily mean that the condition is happening. Did
> you
> > mean to say that relinquish the leadership if it is taking longer than
> > leader.fetch.process.time.max.ms AND there are fetch requests pending
> which
> > are >= log-end-offset of the earlier fetch request ?
>
> Right. This config triggers relinquishing the leadership only for the
> mentioned cases in the KIP.
>
> Thanks,
> Satish.
>
> On Mon, 28 Jun 2021 at 23:11, Mohan Parthasarathy <mp...@gmail.com>
> wrote:
> >
> > Hi Satish,
> >
> >
> > >
> > >
> > > >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.
> > >
> > >
> > That clarification in the document helps. But then setting the first
> option
> > to true does not necessarily mean that the condition is happening. Did
> you
> > mean to say that relinquish the leadership if it is taking longer than
> > leader.fetch.process.time.max.ms AND there are fetch requests pending
> which
> > are >= log-end-offset of the earlier fetch request ?
> >
> > -Thanks
> > Mohan
> >
> > > Thanks,
> > > Satish.
> > >
> > > On Mon, 28 Jun 2021 at 04:36, Mohan Parthasarathy <mposdev21@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.duggana@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.
> > > > >
> > >
>

Re: [DISCUSS] KIP-501 Avoid out-of-sync or offline partitions when follower fetch requests are not processed in time

Posted by Satish Duggana <sa...@gmail.com>.
> That clarification in the document helps. But then setting the first option
> to true does not necessarily mean that the condition is happening. Did you
> mean to say that relinquish the leadership if it is taking longer than
> leader.fetch.process.time.max.ms AND there are fetch requests pending which
> are >= log-end-offset of the earlier fetch request ?

Right. This config triggers relinquishing the leadership only for the
mentioned cases in the KIP.

Thanks,
Satish.

On Mon, 28 Jun 2021 at 23:11, Mohan Parthasarathy <mp...@gmail.com> wrote:
>
> Hi Satish,
>
>
> >
> >
> > >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.
> >
> >
> That clarification in the document helps. But then setting the first option
> to true does not necessarily mean that the condition is happening. Did you
> mean to say that relinquish the leadership if it is taking longer than
> leader.fetch.process.time.max.ms AND there are fetch requests pending which
> are >= log-end-offset of the earlier fetch request ?
>
> -Thanks
> Mohan
>
> > Thanks,
> > Satish.
> >
> > On Mon, 28 Jun 2021 at 04:36, Mohan Parthasarathy <mp...@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.duggana@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.
> > > >
> >

Re: [DISCUSS] KIP-501 Avoid out-of-sync or offline partitions when follower fetch requests are not processed in time

Posted by Mohan Parthasarathy <mp...@gmail.com>.
Hi Satish,


>
>
> >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.
>
>
That clarification in the document helps. But then setting the first option
to true does not necessarily mean that the condition is happening. Did you
mean to say that relinquish the leadership if it is taking longer than
leader.fetch.process.time.max.ms AND there are fetch requests pending which
are >= log-end-offset of the earlier fetch request ?

-Thanks
Mohan

> Thanks,
> Satish.
>
> On Mon, 28 Jun 2021 at 04:36, Mohan Parthasarathy <mp...@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.duggana@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.
> > >
>

Re: [DISCUSS] KIP-501 Avoid out-of-sync or offline partitions when follower fetch requests are not processed in time

Posted by Satish Duggana <sa...@gmail.com>.
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 <mp...@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 <sa...@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.
> >

Re: [DISCUSS] KIP-501 Avoid out-of-sync or offline partitions when follower fetch requests are not processed in time

Posted by Mohan Parthasarathy <mp...@gmail.com>.
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 <sa...@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.
>

Re: [DISCUSS] KIP-501 Avoid out-of-sync or offline partitions when follower fetch requests are not processed in time

Posted by Satish Duggana <sa...@gmail.com>.
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.

Re: [DISCUSS] KIP-501 Avoid out-of-sync or offline partitions when follower fetch requests are not processed in time

Posted by Dhruvil Shah <dh...@confluent.io.INVALID>.
Thanks for the KIP, Satish.

I am trying to understand the problem we are looking to solve with this
KIP. When the leader is slow in processing fetch requests from the follower
(due to disk, GC, or other reasons), the primary problem is that it could
impact read and write latency and at times cause unavailability depending
on how long the leader continues to be in this state.

How does solution 1 solve the problem? It seems like it prevents followers
from being removed from the ISR but that by itself would not address the
availability problem, is that right?

- Dhruvil

On Wed, Jun 23, 2021 at 6:12 AM Ryanne Dolan <ry...@gmail.com> wrote:

> Satish, we encounter this frequently and consider it a major bug. Your
> solution makes sense to me.
>
> Ryanne
>
> On Tue, Jun 22, 2021, 7:29 PM Satish Duggana <sa...@gmail.com>
> wrote:
>
> > Hi,
> > Bumping up the discussion thread on KIP-501 about avoiding out-of-sync or
> > offline partitions when follower fetch requests are not processed in time
> > by the leader replica. This issue occurred several times in multiple
> > production environments (at Uber, Yelp, Twitter, etc).
> >
> > KIP-501 is located here
> > <
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-501+Avoid+out-of-sync+or+offline+partitions+when+follower+fetch+requests+are+not+processed+in+time
> > >.
> > You may want to look at the earlier mail discussion thread here
> > <
> >
> https://mail-archives.apache.org/mod_mbox/kafka-dev/202002.mbox/%3Cpony-9f4e96e457398374499ab892281453dcaa7dc679-11722f366b06d9f46bcb5905ff94fd6ab167598e%40dev.kafka.apache.org%3E
> > >,
> > and here
> > <
> >
> https://mail-archives.apache.org/mod_mbox/kafka-dev/202002.mbox/%3CCAM-aUZnJ4z%2B_ztjF6sXSL61M1me0ogWZ1BV6%2BoV45rJMG8EoZA%40mail.gmail.com%3E
> > >
> > .
> >
> > Please take a look, I would like to hear your feedback and suggestions.
> >
> > Thanks,
> > Satish.
> >
>

Re: [DISCUSS] KIP-501 Avoid out-of-sync or offline partitions when follower fetch requests are not processed in time

Posted by Ryanne Dolan <ry...@gmail.com>.
Satish, we encounter this frequently and consider it a major bug. Your
solution makes sense to me.

Ryanne

On Tue, Jun 22, 2021, 7:29 PM Satish Duggana <sa...@gmail.com>
wrote:

> Hi,
> Bumping up the discussion thread on KIP-501 about avoiding out-of-sync or
> offline partitions when follower fetch requests are not processed in time
> by the leader replica. This issue occurred several times in multiple
> production environments (at Uber, Yelp, Twitter, etc).
>
> KIP-501 is located here
> <
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-501+Avoid+out-of-sync+or+offline+partitions+when+follower+fetch+requests+are+not+processed+in+time
> >.
> You may want to look at the earlier mail discussion thread here
> <
> https://mail-archives.apache.org/mod_mbox/kafka-dev/202002.mbox/%3Cpony-9f4e96e457398374499ab892281453dcaa7dc679-11722f366b06d9f46bcb5905ff94fd6ab167598e%40dev.kafka.apache.org%3E
> >,
> and here
> <
> https://mail-archives.apache.org/mod_mbox/kafka-dev/202002.mbox/%3CCAM-aUZnJ4z%2B_ztjF6sXSL61M1me0ogWZ1BV6%2BoV45rJMG8EoZA%40mail.gmail.com%3E
> >
> .
>
> Please take a look, I would like to hear your feedback and suggestions.
>
> Thanks,
> Satish.
>