1. Yes, the new protocol requires 2 things to advance the HWM. a) The
   messages have been replicated to the controller-committed ISR members. b)
   The number of ISR members should be at least the min ISR.
   2. With the current protocol, we are not able to select broker 1 as the
   leader. If we first imply we have the new HWM requirement in place, then
   broker 1 is a good candidate to choose. The following part of the KIP (ELR)
   part will explain a new mechanism to enable us to choose broker 1. Note, if
   both HWM and ELR are in place, broker 1 will be actually elected in T3.


On Fri, Aug 11, 2023 at 10:05 AM Jeff Kim <jeff....@confluent.io.invalid>
wrote:

> Hi Calvin,
>
> Thanks for the KIP! I'm still digesting it but I have two questions:
>
> > In the scenario raised in the motivation section, the server may receive
> ack=1 messages during T1 and advance High Watermark when the leader
> is the only one in ISR.
>
> To confirm, the current protocol allows advancing the HWM if all brokers in
> the ISR append to their logs (in this case only the leader). And we're
> proposing
> to advance the HWM only when <at least min.insync.replicas> brokers
> replicate. Is this correct?
>
> > Then, if we elect broker 1 as the leader at T4, though we can guarantee
> the safety of ack=all messages, the High Watermark may move backward
> which causes further impacts on the consumers.
>
> How can broker 1 become the leader if it was ineligible in T3? Or are
> you referring to broker 2?
>
> Thanks,
> Jeff
>
> On Thu, Aug 10, 2023 at 6:48 PM Calvin Liu <ca...@confluent.io.invalid>
> wrote:
>
> > Hi everyone,
> > I'd like to discuss a series of enhancement to the replication protocol.
> >
> > A partition replica can experience local data loss in unclean shutdown
> > scenarios where unflushed data in the OS page cache is lost - such as an
> > availability zone power outage or a server error. The Kafka replication
> > protocol is designed to handle these situations by removing such replicas
> > from the ISR and only re-adding them once they have caught up and
> therefore
> > recovered any lost data. This prevents replicas that lost an arbitrary
> log
> > suffix, which included committed data, from being elected leader.
> > However, there is a "last replica standing" state which when combined
> with
> > a data loss unclean shutdown event can turn a local data loss scenario
> into
> > a global data loss scenario, i.e., committed data can be removed from all
> > replicas. When the last replica in the ISR experiences an unclean
> shutdown
> > and loses committed data, it will be reelected leader after starting up
> > again, causing rejoining followers to truncate their logs and thereby
> > removing the last copies of the committed records which the leader lost
> > initially.
> >
> > The new KIP will maximize the protection and provides MinISR-1 tolerance
> to
> > data loss unclean shutdown events.
> >
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-966%3A+Eligible+Leader+Replicas
> >
>

Reply via email to