Hi Artem
Thanks so much for the comments!

1. Yes, you are right, when the leader gets fenced, it will be put into
ELR. The unclean recovery can only be triggered if the mode is Proactive.
Let me clarify the trigger requirement in the KIP.

2. Good point, the controller should wait for all the LastKnownELR to be
unfenced then trigger the recovery.

3. Let me rewrite this part. The URM should have access to the
ReplicationControllManager which stores the partition registration. Then it
can check the replicas and LastKnownELR. But I guess those are
implementation details.

Thanks!


On Thu, Sep 7, 2023 at 9:07 PM Artem Livshits
<alivsh...@confluent.io.invalid> wrote:

> Hi Calvin,
>
> Thanks for the KIP.  The new ELR protocol looks good to me.  I have some
> questions about unclean recovery, specifically in "balanced" mode:
>
> 1. The KIP mentions that the controller would trigger unclear recovery when
> the leader is fenced, but my understanding is that when a leader is fenced,
> it would get into ELR.  Would it be more precise to say that an unclear
> leader election is triggered when the last member of ELR gets unfenced and
> registers with unclean shutdown?
> 2. For balanced mode, we need replies from at least LastKnownELR, in which
> case, does it make sense to start unclean recovery if some of the
> LastKnownELR are fenced?
> 3. "The URM takes the partition info to initiate an unclear recovery task
> ..." the parameters are topic-partition and replica ids -- what are those?
> Would those be just the whole replica assignment or just LastKnownELR?
>
> -Artem
>
> On Thu, Aug 10, 2023 at 3:47 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