You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@kafka.apache.org by Puneet Mehta <me...@gmail.com> on 2015/02/28 01:13:09 UTC

Question on ISR inclusion & leader election for failed replica on catchup

Hi Gang,

I am testing some of the durability guarantees given by Kafka 8.2.1 which
involve min in-sync replicas and disabling unclean leader election.

My question is: *When will the failed replica after successfully coming up
will be included back in ISR? Is this governed by replica.lag.max.messages
property or will it have to completely catch up with the leader to be back
in ISR?*

Alternately, In more detail, Will we loose a committed write in the
following theoretical setup:

   - Single topic
   - 3 Kafka Brokers K1, K2, K3
   - Replication : 3
   - Minimum In-Sync Replica : 2
   - Acks : -1
   - Compression : Gzip
   - Producer type : Async
   - Batch size : 16000
   - replica.lag.max.messages : 4000

There are 3 batches of data to be sent. Producer will retry if the batch of
data fails on error callback.

Batch 1 : Leader : K1 ; ISR : K1, K2, K3   Result: Data committed
Batch 2 : Leader : K1 ; ISR : K1, K2 ( K3 crashed)  Result: Data committed
Batch 3 : Leader : K1 ; ISR : K1 (K2 crashed)  Result: Data uncommitted due
to min in-sync replica violation.

K3 wakes up, Starts catching up with current leader. It doesn't have batch
2 data. At this point, broker K1 crashes and K3 has about 2K messages less
than K1.

Will K3 be elected the leader at this point as it's within 4K messages to
be in ISR? If true, this probably will lead to committed data loss despite
disabling the unclean leader election, if I am not wrong here?


Thanks,
Puneet Mehta

Re: Question on ISR inclusion & leader election for failed replica on catchup

Posted by Jun Rao <ju...@confluent.io>.
When K1 crashes before K3 fully catches up, by default, Kafka allows K3 to
become the new leader. In this case, data in batch 2 will be lost. Our
default behavior favors availability over consistency. If you prefer
consistency, you can set unclean.leader.election.enable to false on the
broker. With this setting, Kafka will not select K3 as the new leader and
instead, will wait for K1 to come back.

Thanks,

Jun

On Fri, Feb 27, 2015 at 4:13 PM, Puneet Mehta <me...@gmail.com> wrote:

> Hi Gang,
>
> I am testing some of the durability guarantees given by Kafka 8.2.1 which
> involve min in-sync replicas and disabling unclean leader election.
>
> My question is: *When will the failed replica after successfully coming up
> will be included back in ISR? Is this governed by replica.lag.max.messages
> property or will it have to completely catch up with the leader to be back
> in ISR?*
>
> Alternately, In more detail, Will we loose a committed write in the
> following theoretical setup:
>
>    - Single topic
>    - 3 Kafka Brokers K1, K2, K3
>    - Replication : 3
>    - Minimum In-Sync Replica : 2
>    - Acks : -1
>    - Compression : Gzip
>    - Producer type : Async
>    - Batch size : 16000
>    - replica.lag.max.messages : 4000
>
> There are 3 batches of data to be sent. Producer will retry if the batch of
> data fails on error callback.
>
> Batch 1 : Leader : K1 ; ISR : K1, K2, K3   Result: Data committed
> Batch 2 : Leader : K1 ; ISR : K1, K2 ( K3 crashed)  Result: Data committed
> Batch 3 : Leader : K1 ; ISR : K1 (K2 crashed)  Result: Data uncommitted due
> to min in-sync replica violation.
>
> K3 wakes up, Starts catching up with current leader. It doesn't have batch
> 2 data. At this point, broker K1 crashes and K3 has about 2K messages less
> than K1.
>
> Will K3 be elected the leader at this point as it's within 4K messages to
> be in ISR? If true, this probably will lead to committed data loss despite
> disabling the unclean leader election, if I am not wrong here?
>
>
> Thanks,
> Puneet Mehta
>

Re: Question on ISR inclusion & leader election for failed replica on catchup

Posted by Jiangjie Qin <jq...@linkedin.com.INVALID>.
Hi Puneet,

One of the conditions for K3 back to ISR is K3¹s log end offset to be
higher than the K1(leaderReplica)¹s high watermark.
If batch 2 is committed, then the leader high watermark will be above the
offsets of messages in batch 2.
In order to be added into ISR again, K3 has to at least have all the batch
2 in its log.

So in this case, we will not lose committed messages.

Jiangjie (Becket) Qin


On 2/27/15, 4:13 PM, "Puneet Mehta" <me...@gmail.com> wrote:

>Hi Gang,
>
>I am testing some of the durability guarantees given by Kafka 8.2.1 which
>involve min in-sync replicas and disabling unclean leader election.
>
>My question is: *When will the failed replica after successfully coming up
>will be included back in ISR? Is this governed by replica.lag.max.messages
>property or will it have to completely catch up with the leader to be back
>in ISR?*
>
>Alternately, In more detail, Will we loose a committed write in the
>following theoretical setup:
>
>   - Single topic
>   - 3 Kafka Brokers K1, K2, K3
>   - Replication : 3
>   - Minimum In-Sync Replica : 2
>   - Acks : -1
>   - Compression : Gzip
>   - Producer type : Async
>   - Batch size : 16000
>   - replica.lag.max.messages : 4000
>
>There are 3 batches of data to be sent. Producer will retry if the batch
>of
>data fails on error callback.
>
>Batch 1 : Leader : K1 ; ISR : K1, K2, K3   Result: Data committed
>Batch 2 : Leader : K1 ; ISR : K1, K2 ( K3 crashed)  Result: Data committed
>Batch 3 : Leader : K1 ; ISR : K1 (K2 crashed)  Result: Data uncommitted
>due
>to min in-sync replica violation.
>
>K3 wakes up, Starts catching up with current leader. It doesn't have batch
>2 data. At this point, broker K1 crashes and K3 has about 2K messages less
>than K1.
>
>Will K3 be elected the leader at this point as it's within 4K messages to
>be in ISR? If true, this probably will lead to committed data loss despite
>disabling the unclean leader election, if I am not wrong here?
>
>
>Thanks,
>Puneet Mehta