You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Calvin Liu <ca...@confluent.io.INVALID> on 2023/08/10 22:46:52 UTC

[DISCUSS] KIP-966: Eligible Leader Replicas

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

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Calvin Liu <ca...@confluent.io.INVALID>.
Hi Jun,
I have a second thought on the kafka.replication.electable_replicas_count.
I don't think it is a useful metric and it seems wasteful to have a per
partition metric to track the sum(ISR, ELR). I will remove this metric.

On Tue, Sep 12, 2023 at 3:35 PM Calvin Liu <ca...@confluent.io> wrote:

> Hi Jun,
> Thanks for the follow-up. Adjusted the KIP as following.
>
> 16. The controller does not know whether a broker is live or not when it
> is fenced. Updated the behavior. In Balanced mode, the URM will start
> recovery when all the members in the LastKnownELR are all unfenced. In
> the Proactive mode, the URM will query all the unfenced replica.
>
> 21. Updated the detail in the Delivery plan. if unclean.leader.election.enable
> is false and the ELR is empty, the controller will elect the first replica
> in the LastKnownELR to be the leader when it is unfenced. If this replica
> can't be unfenced, then the controller will keep waiting.
>
> 24. Actually, I think this minimalReplicas is not needed as the KIP
> changes. The Unclean Recovery with Proactive/Balance modes have their own
> requirements independent to the number of replicas. Will remove this field.
>
> 25. Will use manual_leader_election_required_partition_count.
>
> 26. Updated. It is unset/set to 0 if there is no live unclean recovery.
>
> 31. Good advice! Updated.
>
> 32. Updated.
>
> 33. Updated.
>
> 34. If only Unclean recovery is implemented, the ISR model does not
> change(no empty ISR). If only the ELR is implemented, we will strictly
> waiting for the first replica in the LastKnownELR to be unfenced.
>
>
> On Tue, Sep 12, 2023 at 12:11 PM Jun Rao <ju...@confluent.io.invalid> wrote:
>
>> Hi, Calvin,
>>
>> Thanks for the reply and the updated KIP. A few more comments.
>>
>> 16. "The URM will query all the replicas including the fenced
>> replicas." Currently, could we tell whether a fenced broker is alive or
>> not? Ideally, we only want to query the live brokers. Otherwise URM will
>> be
>> forced to wait for the timeout.
>>
>> 21. DesiredLeaders:  Could we document that the leader will be selected in
>> the order specified?
>>
>> 24. The minimalReplicas is a field in the json file. But it needs to be in
>> ElectLeadersRequest too, right? We also need to explain how the controller
>> acts on that.
>>
>> 25. Would manual_operation_required_partition_count be better named as
>> manual_leader_election_required_partition_count?
>>
>> 26. Could we document when
>> kafka.replication.unclean_recovery_partitions_count is unset?
>>
>> 31. DescribeTopicResponse: LastKnownLeader is a singleton. In the
>> controller, LastKnownELR is a set. Should we expose LastKnownELR in
>> DescribeTopicResponse instead?
>>
>> 32. GetReplicaLogInfo is only used between the controller and the broker.
>> Should the ACL be CLUSTER_ACTION?
>>
>> 33. Could we document that kafka.replication.electable_replicas_count is
>> partition level?
>>
>> 34. "If only the ELR or Unclean recovery is implemented, the
>> LastKnownLeader is preferred." Do we wait strictly for the lastKnownLeader
>> until it's unfenced or do we select the first unfenced replica from
>> LastKnownELR?
>>
>> Thanks,
>>
>> Jun
>>
>> On Fri, Sep 8, 2023 at 2:56 PM Calvin Liu <ca...@confluent.io.invalid>
>> wrote:
>>
>> > 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
>> > <al...@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 <caliu@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
>> > > >
>> > >
>> >
>>
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Calvin Liu <ca...@confluent.io.INVALID>.
Hi Jun,
Thanks for the follow-up. Adjusted the KIP as following.

16. The controller does not know whether a broker is live or not when it is
fenced. Updated the behavior. In Balanced mode, the URM will start recovery
when all the members in the LastKnownELR are all unfenced. In the Proactive
mode, the URM will query all the unfenced replica.

21. Updated the detail in the Delivery plan. if unclean.leader.election.enable
is false and the ELR is empty, the controller will elect the first replica
in the LastKnownELR to be the leader when it is unfenced. If this replica
can't be unfenced, then the controller will keep waiting.

24. Actually, I think this minimalReplicas is not needed as the KIP
changes. The Unclean Recovery with Proactive/Balance modes have their own
requirements independent to the number of replicas. Will remove this field.

25. Will use manual_leader_election_required_partition_count.

26. Updated. It is unset/set to 0 if there is no live unclean recovery.

31. Good advice! Updated.

32. Updated.

33. Updated.

34. If only Unclean recovery is implemented, the ISR model does not
change(no empty ISR). If only the ELR is implemented, we will strictly
waiting for the first replica in the LastKnownELR to be unfenced.


On Tue, Sep 12, 2023 at 12:11 PM Jun Rao <ju...@confluent.io.invalid> wrote:

> Hi, Calvin,
>
> Thanks for the reply and the updated KIP. A few more comments.
>
> 16. "The URM will query all the replicas including the fenced
> replicas." Currently, could we tell whether a fenced broker is alive or
> not? Ideally, we only want to query the live brokers. Otherwise URM will be
> forced to wait for the timeout.
>
> 21. DesiredLeaders:  Could we document that the leader will be selected in
> the order specified?
>
> 24. The minimalReplicas is a field in the json file. But it needs to be in
> ElectLeadersRequest too, right? We also need to explain how the controller
> acts on that.
>
> 25. Would manual_operation_required_partition_count be better named as
> manual_leader_election_required_partition_count?
>
> 26. Could we document when
> kafka.replication.unclean_recovery_partitions_count is unset?
>
> 31. DescribeTopicResponse: LastKnownLeader is a singleton. In the
> controller, LastKnownELR is a set. Should we expose LastKnownELR in
> DescribeTopicResponse instead?
>
> 32. GetReplicaLogInfo is only used between the controller and the broker.
> Should the ACL be CLUSTER_ACTION?
>
> 33. Could we document that kafka.replication.electable_replicas_count is
> partition level?
>
> 34. "If only the ELR or Unclean recovery is implemented, the
> LastKnownLeader is preferred." Do we wait strictly for the lastKnownLeader
> until it's unfenced or do we select the first unfenced replica from
> LastKnownELR?
>
> Thanks,
>
> Jun
>
> On Fri, Sep 8, 2023 at 2:56 PM Calvin Liu <ca...@confluent.io.invalid>
> wrote:
>
> > 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
> > <al...@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 <caliu@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
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

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

Thanks for the reply and the updated KIP. A few more comments.

16. "The URM will query all the replicas including the fenced
replicas." Currently, could we tell whether a fenced broker is alive or
not? Ideally, we only want to query the live brokers. Otherwise URM will be
forced to wait for the timeout.

21. DesiredLeaders:  Could we document that the leader will be selected in
the order specified?

24. The minimalReplicas is a field in the json file. But it needs to be in
ElectLeadersRequest too, right? We also need to explain how the controller
acts on that.

25. Would manual_operation_required_partition_count be better named as
manual_leader_election_required_partition_count?

26. Could we document when
kafka.replication.unclean_recovery_partitions_count is unset?

31. DescribeTopicResponse: LastKnownLeader is a singleton. In the
controller, LastKnownELR is a set. Should we expose LastKnownELR in
DescribeTopicResponse instead?

32. GetReplicaLogInfo is only used between the controller and the broker.
Should the ACL be CLUSTER_ACTION?

33. Could we document that kafka.replication.electable_replicas_count is
partition level?

34. "If only the ELR or Unclean recovery is implemented, the
LastKnownLeader is preferred." Do we wait strictly for the lastKnownLeader
until it's unfenced or do we select the first unfenced replica from
LastKnownELR?

Thanks,

Jun

On Fri, Sep 8, 2023 at 2:56 PM Calvin Liu <ca...@confluent.io.invalid>
wrote:

> 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
> <al...@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
> > >
> >
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Calvin Liu <ca...@confluent.io.INVALID>.
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
<al...@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
> >
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Artem Livshits <al...@confluent.io.INVALID>.
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
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Calvin Liu <ca...@confluent.io.INVALID>.
Hi Jun
Thanks for the comments!

10. Updated

11. It is mentioned in the ELR invariants section. It is mainly to save
metadata space.

12. Good catch, let me update the graph.

13. The thing we did not change is the original HWM advance requirement
“replicate to maximum ISR”. The proposal adds another requirement of
“committed ISR size should be larger than min ISR.” Let me update it.

14. In the Proactive mode, it will have the unclean recovery.

15. Good advice, I will update to “In Balance mode, all the LastKnownELR
members have replied, plus the replicas replied within the timeout.”

16. We can have a race between broker unfence and the log query. So just
try the best we can.

17. Will update. It is not downgradable once the Unclean Recovery is
enabled.

18. We don’t name ISR as ISRs so I think we don’t have to add s to the
LastKnownELR.

19. Good catch. I will rename it to PreviousBrokerEpoch. If the broker does
not find any previous epoch, it can set it to -1.

20. The issuer should be admin clients or brokers. The controller will
serve the request because brokers do not know the ELR info.

21. Yes, it uses the index to map the partitions and the desired leaders.

22. The admin clients may not know the topic id. So give it an option to
use topic name.

23. I will add the unclean. DESIGNATION refers to elect the provided
replica to be the leader.

24. The minimalReplicas is a field in the json file. See the example of
--path-to-json-file

25. Maybe call it manual_operation_required_partition_count? A partition
can be leaderless but it is waiting for a ELR member to be online, not
necessarily requiring operators to check.

26. It should be added 1 when a partition is triggered for an unclean
recovery and -1 when the partition has a new leader.

27. The words are not accurate. Min ISR only works on HWM advance and
whether to reject an ack=all request.

28. Good advice. Will update.

29. Yes, it does maintain the last ISR member. Just the last ISR member is
usually the last known leader. And the reason behind is for election.

30. Will update.
Thanks again!

On Thu, Sep 7, 2023 at 11:45 AM Jun Rao <ju...@confluent.io.invalid> wrote:

> Hi, Calvin,
>
> Thanks for the KIP. A few comments below.
>
> 10. "The High Watermark forwarding still requires a quorum within the ISR."
> Should it say that it requires the full ISR?
>
> 11. "remove the duplicate member in both ISR and ELR from ELR." Hmm, not
> sure that I follow since the KIP says that ELR doesn't overlap with ISR.
>
> 12. T1: In this case, min ISR is 3. Why would HWM advance to 2 with only 2
> members in ISR?
>
> 13. The KIP says "Note that, if maximal ISR > ISR, the message should be
> replicated to the maximal ISR before covering the message under HWM. The
> proposal does not change this behavior." and "Currently, we would advance
> HWM because it replicated to 2 brokers (the ones in Maximal ISR), but in
> the new protocol we wait until the controller updates ISR=[0,2] to avoid
> advancing HWM beyond what ELR=[1] has." They seem a bit inconsistent since
> one says no change and the other describes a change.
>
> 14. "If there are no ELR members. If the
> unclean.recovery.strategy=balanced, the controller will do the unclean
> recovery. Otherwise, unclean.recovery.strategy=Manual, the controller will
> not attempt to elect a leader. Waiting for the user operations." What
> happens with unclean.recovery.strategy=Proactive?
>
> 15. "In Balance mode, all the LastKnownELR members have replied." In
> Proactive, we wait for all replicas within a fixed amount of time. Balance
> should do the same since it's designed to preserve more data, right?
>
> 16. "The URM will query all the replicas including the fenced replicas."
> Why include the fenced replicas? Could a fenced replica be elected as the
> leader?
>
> 17. Once unclean.recovery.strategy is enabled, new metadata records could
> be written to the metadata log. At that point, is the broker downgradable?
> It would be useful to document that.
>
> 18. Since LastKnownELR can have more than 1 member, should it be
> LastKnownELRs?
>
> 19. BrokerRegistration.BrokerEpoch: "The broker's assigned epoch or the
> epoch before a clean shutdown." How do we tell whether the value is for the
> current or the previous epoch? Does it matter?
>
> 20. DescribeTopicRequest: Who issues that request? Who can serve that
> request? Is it only the controller or any broker?
>
> 21. DesiredLeaders: Does the ordering matter?
>
> 22. GetReplicaLogInfo only uses topicId while DescribeTopicRequest uses
> both topicId and name. Should they be consistent?
>
> 23. --election-type: The description mentions unclean, but that option
> doesn't exist. Also, could we describe what DESIGNATION means?
>
> 24. kafka-leader-election.sh has minimalReplicas, but ElectLeadersRequest
> doesn't seem to have a corresponding field?
>
> 25. kafka.replication.paused_partitions_count: paused doesn't seem to match
> the meaning of the metric. Should this be leaderless_partitions_count?
>
> 26. kafka.replication.unclean_recovery_partitions_count: When is it set?
> Does it ever get unset?
>
> 27. "min.insync.replicas now applies to the replication of all kinds of
> messages." Not sure that I follow. Could you explain a bit more?
>
> 28. "But later, the current leader will put the follower into the pending
> ISR" : It would be useful to clarify this is after the network partitioning
> is gone.
>
> 29. "last known leader" behavior. Our current behavior is to preserve the
> last known ISR, right?
>
> 30. For all new requests, it would be useful to document the corresponding
> ACL.
>
> Jun
>
> On Wed, Sep 6, 2023 at 11:21 AM Calvin Liu <ca...@confluent.io.invalid>
> wrote:
>
> > Hi Jack
> > Thanks for the comment.
> > I have updated the reassignment part. Now the reassignment can only be
> > completed/canceled if the final ISR size is larger than min ISR.
> > Thanks to your efforts of the TLA+! It has been a great help to the KIP!
> >
> > On Wed, Sep 6, 2023 at 6:32 AM Jack Vanlightly <va...@apache.org>
> > wrote:
> >
> > > Hi Calvin,
> > >
> > > Regarding partition reassignment, I have two comments.
> > >
> > > I notice the KIP says "The AlterPartitionReassignments will not change
> > the
> > > ELR" however, when a reassignment completes (or reverts) any replicas
> > > removed from the replica set would be removed from the ELR. Sounds
> > obvious
> > > but I figured we should be explicit about that.
> > >
> > > Reassignment should also respect min.insync.replicas because currently
> a
> > > reassignment can complete as long as the ISR is not empty and all added
> > > replicas are members. However, my TLA+ specification, which now
> includes
> > > reassignment, finds single broker failures that can cause committed
> data
> > > loss - despite the added protection of the ELR and
> min.insync.replicas=2.
> > > These scenarios are limited to shrinking the size of the replica set.
> If
> > we
> > > modify the PartitionChangeBuilder to add the completion condition that
> > the
> > > target ISR >= min.insync.replicas, then that closes this last
> > > single-broker-failure data loss case.
> > >
> > > With the above modification, the TLA+ specification of the ELR part of
> > the
> > > design is standing up to all safety and liveness checks. The only thing
> > > that is not modeled is the unclean recovery though I may leave that as
> > the
> > > specification is already very large.
> > >
> > > Jack
> > >
> > > On 2023/09/01 22:27:10 Calvin Liu wrote:
> > > > Hi Justine
> > > > 1. With the new proposal, in order to let the consumer consume a
> > message
> > > > when only 1 replica commits it to its log, the min ISR has to be set
> to
> > > 1.
> > > > 2. Yes, we will need an unclean recovery if the leader has an unclean
> > > > shutdown.
> > > > 3. If the min ISR config is changed to a larger value, the ISR and
> ELR
> > > will
> > > > not be updated. ISR members are always valid no matter how min ISR
> > > changes.
> > > > If ELR is not empty, then the HWM can't advance as well after the min
> > ISR
> > > > increase, so the ELR members are safe to stay.
> > > > 4. I will highlight the explanation. Thanks.
> > > >
> > > > On Thu, Aug 31, 2023 at 4:35 PM Justine Olshan
> > > <jo...@confluent.io.invalid>
> > > > wrote:
> > > >
> > > > > Hey Calvin,
> > > > >
> > > > > Thanks for the responses. I think I understood most of it, but had
> a
> > > few
> > > > > follow up questions
> > > > >
> > > > > 1. For the acks=1 case, I was wondering if there is any way to
> > continue
> > > > > with the current behavior (ie -- we only need one ack to produce to
> > > the log
> > > > > and consider the request complete.) My understanding is that we can
> > > also
> > > > > consume from such topics at that point.
> > > > > If users wanted this lower durability could they set
> > > min.insync.replicas to
> > > > > 1?
> > > > >
> > > > > 2. For the case where we elect a leader that was unknowingly
> offline.
> > > Say
> > > > > this replica was the only one in ELR. My understanding is that we
> > would
> > > > > promote it to ISR and remove it from ELR when it is the leader, but
> > > then we
> > > > > would remove it from ISR and have no brokers in ISR or ELR. From
> > there
> > > we
> > > > > would need to do unclean recovery right?
> > > > >
> > > > > 3. Did we address the case where dynamically min isr is increased?
> > > > >
> > > > > 4. I think my comment was more about confusion on the KIP. It was
> not
> > > clear
> > > > > to me that the section was describing points if one was done before
> > the
> > > > > other. But I now see the sentence explaining that. I think I
> skipped
> > > from
> > > > > "delivery plan" to the bullet points.
> > > > >
> > > > > Justine
> > > > >
> > > > > On Thu, Aug 31, 2023 at 4:04 PM Calvin Liu
> > <caliu@confluent.io.invalid
> > > >
> > > > > wrote:
> > > > >
> > > > > > Hi Justine
> > > > > > Thanks for the questions!
> > > > > >   *a. For my understanding, will we block replication? Or just
> the
> > > high
> > > > > > watermark advancement?*
> > > > > >   - The replication will not be blocked. The followers are free
> to
> > > > > > replicate messages above the HWM. Only HWM advancement is
> blocked.
> > > > > >
> > > > > >   b. *Also in the acks=1 case, if folks want to continue the
> > previous
> > > > > > behavior, they also need to set min.insync.replicas to 1,
> correct?*
> > > > > >   - If the clients only send ack=1 messages and minISR=2. The HWM
> > > > > behavior
> > > > > > will only be different when there is 1 replica in the ISR. In
> this
> > > case,
> > > > > > the min ISR does not do much in the current system. It is kind
> of a
> > > > > > trade-off but we think it is ok.
> > > > > >
> > > > > >   c. *The KIP seems to suggest that we remove from ELR when we
> > start
> > > up
> > > > > > again and notice we do not have the clean shutdown file. Is
> there a
> > > > > chance
> > > > > > we have an offline broker in ELR that had an unclean shutdown
> that
> > we
> > > > > elect
> > > > > > as a leader before we get the change to realize the shutdown was
> > > > > unclean?*
> > > > > > *  - *The controller will only elect an unfenced(online) replica
> as
> > > the
> > > > > > leader. If a broker has an unclean shutdown, it should register
> to
> > > the
> > > > > > controller first(where it has to declair whether it is a
> > > clean/unclean
> > > > > > shutdown) and then start to serve broker requests. So
> > > > > >      1. If the broker has an unclean shutdown before the
> controller
> > > is
> > > > > > aware that the replica is offline, then the broker can become the
> > > leader
> > > > > > temporarily. But it can't serve any Fetch requests before it
> > > registers
> > > > > > again, and that's when the controller will re-elect a leader.
> > > > > >      2. If the controller knows the replica is offline(missing
> > > heartbeats
> > > > > > from the broker for a while) before the broker re-registers, the
> > > broker
> > > > > > can't be elected as a leader.
> > > > > >
> > > > > > d. *Would this be the case for strictly a smaller min ISR?*
> > > > > > - Yes, only when we have a smaller min ISR. Once the leader is
> > aware
> > > of
> > > > > the
> > > > > > minISR change, the HWM can advance and make the current ELR
> > > obsolete. So
> > > > > > the controller should clear the ELR if the ISR >= the new min
> ISR.
> > > > > >
> > > > > > e. *I thought we said the above "Last Leader” behavior can’t be
> > > > > maintained
> > > > > > with an empty ISR and it should be removed."*
> > > > > > -  As the Kip is a big one, we have to consider delivering it in
> > > phases.
> > > > > If
> > > > > > only the Unclean Recovery is delivered, we do not touch the ISR
> > then
> > > the
> > > > > > ISR behavior will be the same as the current. I am open to the
> > > proposal
> > > > > > that directly starting unclean recovery if the last leader fails.
> > > Let's
> > > > > see
> > > > > > if other folks hope to have more if Unclean Recover delivers
> first.
> > > > > >
> > > > > > On Tue, Aug 29, 2023 at 4:53 PM Justine Olshan
> > > > > > <jo...@confluent.io.invalid>
> > > > > > wrote:
> > > > > >
> > > > > > > Hey Calvin,
> > > > > > >
> > > > > > > Thanks for the KIP. This will close some of the gaps in leader
> > > > > election!
> > > > > > I
> > > > > > > has a few questions:
> > > > > > >
> > > > > > > *>* *High Watermark can only advance if the ISR size is larger
> or
> > > equal
> > > > > > > to min.insync.replicas*.
> > > > > > >
> > > > > > > For my understanding, will we block replication? Or just the
> high
> > > > > > watermark
> > > > > > > advancement?
> > > > > > > Also in the acks=1 case, if folks want to continue the previous
> > > > > behavior,
> > > > > > > they also need to set min.insync.replicas to 1, correct? It
> seems
> > > like
> > > > > > this
> > > > > > > change takes some control away from clients when it comes to
> > > durability
> > > > > > vs
> > > > > > > availability.
> > > > > > >
> > > > > > > *> *
> > > > > > > *ELR + ISR size will not be dropped below the min ISR unless
> the
> > > > > > controller
> > > > > > > discovers an ELR member has an unclean shutdown. *
> > > > > > > The KIP seems to suggest that we remove from ELR when we start
> up
> > > again
> > > > > > and
> > > > > > > notice we do not have the clean shutdown file. Is there a
> chance
> > we
> > > > > have
> > > > > > an
> > > > > > > offline broker in ELR that had an unclean shutdown that we
> elect
> > > as a
> > > > > > > leader before we get the change to realize the shutdown was
> > > unclean?
> > > > > > > This seems like it could cause some problems. I may have missed
> > > how we
> > > > > > > avoid this scenario though.
> > > > > > >
> > > > > > > *> When updating the config **min.insync.replicas, *
> > > > > > > *if the new min ISR <= current ISR, the ELR will be
> > removed.*Would
> > > this
> > > > > > be
> > > > > > > the case for strictly a smaller min ISR? I suppose if we
> increase
> > > the
> > > > > > ISR,
> > > > > > > we can't reason about ELR. Can we reason about high water mark
> in
> > > this
> > > > > > > case--seems like we will have the broker out of ISR not in ISR
> or
> > > ELR?
> > > > > > > (Forgive me if we can't increase min ISR if the increase will
> put
> > > us
> > > > > > under
> > > > > > > it)
> > > > > > >
> > > > > > > *> Unclean recovery. *
> > > > > > >
> > > > > > >    - *The unclean leader election will be replaced by the
> unclean
> > > > > > > recovery.*
> > > > > > >    - *unclean.leader.election.enable will only be replaced by
> > > > > > >    the unclean.recovery.strategy after ELR is delivered.*
> > > > > > >    - *As there is no change to the ISR, the "last known leader"
> > > > > behavior
> > > > > > is
> > > > > > >    maintained.*
> > > > > > >
> > > > > > > What does "last known leader behavior maintained" mean here? I
> > > thought
> > > > > we
> > > > > > > said *"*The above “*Last Leader” behavior can’t be maintained
> > with
> > > an
> > > > > > empty
> > > > > > > ISR and it should be removed." *My understanding is once
> metadata
> > > > > version
> > > > > > > is updated we will always take the more thoughtful unclean
> > election
> > > > > > process
> > > > > > > (ie, inspect the logs)
> > > > > > >
> > > > > > > Overall though, the general KIP is pretty solid. Looking at the
> > > > > rejected
> > > > > > > alternatives, it looks like a lot was considered, so it's nice
> to
> > > see
> > > > > the
> > > > > > > final proposal.
> > > > > > >
> > > > > > > Justine
> > > > > > >
> > > > > > > On Mon, Aug 14, 2023 at 8:50 AM Calvin Liu
> > > <caliu@confluent.io.invalid
> > > > > >
> > > > > > > wrote:
> > > > > > >
> > > > > > > >    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.kim@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
> > > > > > <caliu@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
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

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

Thanks for the KIP. A few comments below.

10. "The High Watermark forwarding still requires a quorum within the ISR."
Should it say that it requires the full ISR?

11. "remove the duplicate member in both ISR and ELR from ELR." Hmm, not
sure that I follow since the KIP says that ELR doesn't overlap with ISR.

12. T1: In this case, min ISR is 3. Why would HWM advance to 2 with only 2
members in ISR?

13. The KIP says "Note that, if maximal ISR > ISR, the message should be
replicated to the maximal ISR before covering the message under HWM. The
proposal does not change this behavior." and "Currently, we would advance
HWM because it replicated to 2 brokers (the ones in Maximal ISR), but in
the new protocol we wait until the controller updates ISR=[0,2] to avoid
advancing HWM beyond what ELR=[1] has." They seem a bit inconsistent since
one says no change and the other describes a change.

14. "If there are no ELR members. If the
unclean.recovery.strategy=balanced, the controller will do the unclean
recovery. Otherwise, unclean.recovery.strategy=Manual, the controller will
not attempt to elect a leader. Waiting for the user operations." What
happens with unclean.recovery.strategy=Proactive?

15. "In Balance mode, all the LastKnownELR members have replied." In
Proactive, we wait for all replicas within a fixed amount of time. Balance
should do the same since it's designed to preserve more data, right?

16. "The URM will query all the replicas including the fenced replicas."
Why include the fenced replicas? Could a fenced replica be elected as the
leader?

17. Once unclean.recovery.strategy is enabled, new metadata records could
be written to the metadata log. At that point, is the broker downgradable?
It would be useful to document that.

18. Since LastKnownELR can have more than 1 member, should it be
LastKnownELRs?

19. BrokerRegistration.BrokerEpoch: "The broker's assigned epoch or the
epoch before a clean shutdown." How do we tell whether the value is for the
current or the previous epoch? Does it matter?

20. DescribeTopicRequest: Who issues that request? Who can serve that
request? Is it only the controller or any broker?

21. DesiredLeaders: Does the ordering matter?

22. GetReplicaLogInfo only uses topicId while DescribeTopicRequest uses
both topicId and name. Should they be consistent?

23. --election-type: The description mentions unclean, but that option
doesn't exist. Also, could we describe what DESIGNATION means?

24. kafka-leader-election.sh has minimalReplicas, but ElectLeadersRequest
doesn't seem to have a corresponding field?

25. kafka.replication.paused_partitions_count: paused doesn't seem to match
the meaning of the metric. Should this be leaderless_partitions_count?

26. kafka.replication.unclean_recovery_partitions_count: When is it set?
Does it ever get unset?

27. "min.insync.replicas now applies to the replication of all kinds of
messages." Not sure that I follow. Could you explain a bit more?

28. "But later, the current leader will put the follower into the pending
ISR" : It would be useful to clarify this is after the network partitioning
is gone.

29. "last known leader" behavior. Our current behavior is to preserve the
last known ISR, right?

30. For all new requests, it would be useful to document the corresponding
ACL.

Jun

On Wed, Sep 6, 2023 at 11:21 AM Calvin Liu <ca...@confluent.io.invalid>
wrote:

> Hi Jack
> Thanks for the comment.
> I have updated the reassignment part. Now the reassignment can only be
> completed/canceled if the final ISR size is larger than min ISR.
> Thanks to your efforts of the TLA+! It has been a great help to the KIP!
>
> On Wed, Sep 6, 2023 at 6:32 AM Jack Vanlightly <va...@apache.org>
> wrote:
>
> > Hi Calvin,
> >
> > Regarding partition reassignment, I have two comments.
> >
> > I notice the KIP says "The AlterPartitionReassignments will not change
> the
> > ELR" however, when a reassignment completes (or reverts) any replicas
> > removed from the replica set would be removed from the ELR. Sounds
> obvious
> > but I figured we should be explicit about that.
> >
> > Reassignment should also respect min.insync.replicas because currently a
> > reassignment can complete as long as the ISR is not empty and all added
> > replicas are members. However, my TLA+ specification, which now includes
> > reassignment, finds single broker failures that can cause committed data
> > loss - despite the added protection of the ELR and min.insync.replicas=2.
> > These scenarios are limited to shrinking the size of the replica set. If
> we
> > modify the PartitionChangeBuilder to add the completion condition that
> the
> > target ISR >= min.insync.replicas, then that closes this last
> > single-broker-failure data loss case.
> >
> > With the above modification, the TLA+ specification of the ELR part of
> the
> > design is standing up to all safety and liveness checks. The only thing
> > that is not modeled is the unclean recovery though I may leave that as
> the
> > specification is already very large.
> >
> > Jack
> >
> > On 2023/09/01 22:27:10 Calvin Liu wrote:
> > > Hi Justine
> > > 1. With the new proposal, in order to let the consumer consume a
> message
> > > when only 1 replica commits it to its log, the min ISR has to be set to
> > 1.
> > > 2. Yes, we will need an unclean recovery if the leader has an unclean
> > > shutdown.
> > > 3. If the min ISR config is changed to a larger value, the ISR and ELR
> > will
> > > not be updated. ISR members are always valid no matter how min ISR
> > changes.
> > > If ELR is not empty, then the HWM can't advance as well after the min
> ISR
> > > increase, so the ELR members are safe to stay.
> > > 4. I will highlight the explanation. Thanks.
> > >
> > > On Thu, Aug 31, 2023 at 4:35 PM Justine Olshan
> > <jo...@confluent.io.invalid>
> > > wrote:
> > >
> > > > Hey Calvin,
> > > >
> > > > Thanks for the responses. I think I understood most of it, but had a
> > few
> > > > follow up questions
> > > >
> > > > 1. For the acks=1 case, I was wondering if there is any way to
> continue
> > > > with the current behavior (ie -- we only need one ack to produce to
> > the log
> > > > and consider the request complete.) My understanding is that we can
> > also
> > > > consume from such topics at that point.
> > > > If users wanted this lower durability could they set
> > min.insync.replicas to
> > > > 1?
> > > >
> > > > 2. For the case where we elect a leader that was unknowingly offline.
> > Say
> > > > this replica was the only one in ELR. My understanding is that we
> would
> > > > promote it to ISR and remove it from ELR when it is the leader, but
> > then we
> > > > would remove it from ISR and have no brokers in ISR or ELR. From
> there
> > we
> > > > would need to do unclean recovery right?
> > > >
> > > > 3. Did we address the case where dynamically min isr is increased?
> > > >
> > > > 4. I think my comment was more about confusion on the KIP. It was not
> > clear
> > > > to me that the section was describing points if one was done before
> the
> > > > other. But I now see the sentence explaining that. I think I skipped
> > from
> > > > "delivery plan" to the bullet points.
> > > >
> > > > Justine
> > > >
> > > > On Thu, Aug 31, 2023 at 4:04 PM Calvin Liu
> <caliu@confluent.io.invalid
> > >
> > > > wrote:
> > > >
> > > > > Hi Justine
> > > > > Thanks for the questions!
> > > > >   *a. For my understanding, will we block replication? Or just the
> > high
> > > > > watermark advancement?*
> > > > >   - The replication will not be blocked. The followers are free to
> > > > > replicate messages above the HWM. Only HWM advancement is blocked.
> > > > >
> > > > >   b. *Also in the acks=1 case, if folks want to continue the
> previous
> > > > > behavior, they also need to set min.insync.replicas to 1, correct?*
> > > > >   - If the clients only send ack=1 messages and minISR=2. The HWM
> > > > behavior
> > > > > will only be different when there is 1 replica in the ISR. In this
> > case,
> > > > > the min ISR does not do much in the current system. It is kind of a
> > > > > trade-off but we think it is ok.
> > > > >
> > > > >   c. *The KIP seems to suggest that we remove from ELR when we
> start
> > up
> > > > > again and notice we do not have the clean shutdown file. Is there a
> > > > chance
> > > > > we have an offline broker in ELR that had an unclean shutdown that
> we
> > > > elect
> > > > > as a leader before we get the change to realize the shutdown was
> > > > unclean?*
> > > > > *  - *The controller will only elect an unfenced(online) replica as
> > the
> > > > > leader. If a broker has an unclean shutdown, it should register to
> > the
> > > > > controller first(where it has to declair whether it is a
> > clean/unclean
> > > > > shutdown) and then start to serve broker requests. So
> > > > >      1. If the broker has an unclean shutdown before the controller
> > is
> > > > > aware that the replica is offline, then the broker can become the
> > leader
> > > > > temporarily. But it can't serve any Fetch requests before it
> > registers
> > > > > again, and that's when the controller will re-elect a leader.
> > > > >      2. If the controller knows the replica is offline(missing
> > heartbeats
> > > > > from the broker for a while) before the broker re-registers, the
> > broker
> > > > > can't be elected as a leader.
> > > > >
> > > > > d. *Would this be the case for strictly a smaller min ISR?*
> > > > > - Yes, only when we have a smaller min ISR. Once the leader is
> aware
> > of
> > > > the
> > > > > minISR change, the HWM can advance and make the current ELR
> > obsolete. So
> > > > > the controller should clear the ELR if the ISR >= the new min ISR.
> > > > >
> > > > > e. *I thought we said the above "Last Leader” behavior can’t be
> > > > maintained
> > > > > with an empty ISR and it should be removed."*
> > > > > -  As the Kip is a big one, we have to consider delivering it in
> > phases.
> > > > If
> > > > > only the Unclean Recovery is delivered, we do not touch the ISR
> then
> > the
> > > > > ISR behavior will be the same as the current. I am open to the
> > proposal
> > > > > that directly starting unclean recovery if the last leader fails.
> > Let's
> > > > see
> > > > > if other folks hope to have more if Unclean Recover delivers first.
> > > > >
> > > > > On Tue, Aug 29, 2023 at 4:53 PM Justine Olshan
> > > > > <jo...@confluent.io.invalid>
> > > > > wrote:
> > > > >
> > > > > > Hey Calvin,
> > > > > >
> > > > > > Thanks for the KIP. This will close some of the gaps in leader
> > > > election!
> > > > > I
> > > > > > has a few questions:
> > > > > >
> > > > > > *>* *High Watermark can only advance if the ISR size is larger or
> > equal
> > > > > > to min.insync.replicas*.
> > > > > >
> > > > > > For my understanding, will we block replication? Or just the high
> > > > > watermark
> > > > > > advancement?
> > > > > > Also in the acks=1 case, if folks want to continue the previous
> > > > behavior,
> > > > > > they also need to set min.insync.replicas to 1, correct? It seems
> > like
> > > > > this
> > > > > > change takes some control away from clients when it comes to
> > durability
> > > > > vs
> > > > > > availability.
> > > > > >
> > > > > > *> *
> > > > > > *ELR + ISR size will not be dropped below the min ISR unless the
> > > > > controller
> > > > > > discovers an ELR member has an unclean shutdown. *
> > > > > > The KIP seems to suggest that we remove from ELR when we start up
> > again
> > > > > and
> > > > > > notice we do not have the clean shutdown file. Is there a chance
> we
> > > > have
> > > > > an
> > > > > > offline broker in ELR that had an unclean shutdown that we elect
> > as a
> > > > > > leader before we get the change to realize the shutdown was
> > unclean?
> > > > > > This seems like it could cause some problems. I may have missed
> > how we
> > > > > > avoid this scenario though.
> > > > > >
> > > > > > *> When updating the config **min.insync.replicas, *
> > > > > > *if the new min ISR <= current ISR, the ELR will be
> removed.*Would
> > this
> > > > > be
> > > > > > the case for strictly a smaller min ISR? I suppose if we increase
> > the
> > > > > ISR,
> > > > > > we can't reason about ELR. Can we reason about high water mark in
> > this
> > > > > > case--seems like we will have the broker out of ISR not in ISR or
> > ELR?
> > > > > > (Forgive me if we can't increase min ISR if the increase will put
> > us
> > > > > under
> > > > > > it)
> > > > > >
> > > > > > *> Unclean recovery. *
> > > > > >
> > > > > >    - *The unclean leader election will be replaced by the unclean
> > > > > > recovery.*
> > > > > >    - *unclean.leader.election.enable will only be replaced by
> > > > > >    the unclean.recovery.strategy after ELR is delivered.*
> > > > > >    - *As there is no change to the ISR, the "last known leader"
> > > > behavior
> > > > > is
> > > > > >    maintained.*
> > > > > >
> > > > > > What does "last known leader behavior maintained" mean here? I
> > thought
> > > > we
> > > > > > said *"*The above “*Last Leader” behavior can’t be maintained
> with
> > an
> > > > > empty
> > > > > > ISR and it should be removed." *My understanding is once metadata
> > > > version
> > > > > > is updated we will always take the more thoughtful unclean
> election
> > > > > process
> > > > > > (ie, inspect the logs)
> > > > > >
> > > > > > Overall though, the general KIP is pretty solid. Looking at the
> > > > rejected
> > > > > > alternatives, it looks like a lot was considered, so it's nice to
> > see
> > > > the
> > > > > > final proposal.
> > > > > >
> > > > > > Justine
> > > > > >
> > > > > > On Mon, Aug 14, 2023 at 8:50 AM Calvin Liu
> > <caliu@confluent.io.invalid
> > > > >
> > > > > > wrote:
> > > > > >
> > > > > > >    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.kim@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
> > > > > <caliu@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
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Calvin Liu <ca...@confluent.io.INVALID>.
Hi Jack
Thanks for the comment.
I have updated the reassignment part. Now the reassignment can only be
completed/canceled if the final ISR size is larger than min ISR.
Thanks to your efforts of the TLA+! It has been a great help to the KIP!

On Wed, Sep 6, 2023 at 6:32 AM Jack Vanlightly <va...@apache.org>
wrote:

> Hi Calvin,
>
> Regarding partition reassignment, I have two comments.
>
> I notice the KIP says "The AlterPartitionReassignments will not change the
> ELR" however, when a reassignment completes (or reverts) any replicas
> removed from the replica set would be removed from the ELR. Sounds obvious
> but I figured we should be explicit about that.
>
> Reassignment should also respect min.insync.replicas because currently a
> reassignment can complete as long as the ISR is not empty and all added
> replicas are members. However, my TLA+ specification, which now includes
> reassignment, finds single broker failures that can cause committed data
> loss - despite the added protection of the ELR and min.insync.replicas=2.
> These scenarios are limited to shrinking the size of the replica set. If we
> modify the PartitionChangeBuilder to add the completion condition that the
> target ISR >= min.insync.replicas, then that closes this last
> single-broker-failure data loss case.
>
> With the above modification, the TLA+ specification of the ELR part of the
> design is standing up to all safety and liveness checks. The only thing
> that is not modeled is the unclean recovery though I may leave that as the
> specification is already very large.
>
> Jack
>
> On 2023/09/01 22:27:10 Calvin Liu wrote:
> > Hi Justine
> > 1. With the new proposal, in order to let the consumer consume a message
> > when only 1 replica commits it to its log, the min ISR has to be set to
> 1.
> > 2. Yes, we will need an unclean recovery if the leader has an unclean
> > shutdown.
> > 3. If the min ISR config is changed to a larger value, the ISR and ELR
> will
> > not be updated. ISR members are always valid no matter how min ISR
> changes.
> > If ELR is not empty, then the HWM can't advance as well after the min ISR
> > increase, so the ELR members are safe to stay.
> > 4. I will highlight the explanation. Thanks.
> >
> > On Thu, Aug 31, 2023 at 4:35 PM Justine Olshan
> <jo...@confluent.io.invalid>
> > wrote:
> >
> > > Hey Calvin,
> > >
> > > Thanks for the responses. I think I understood most of it, but had a
> few
> > > follow up questions
> > >
> > > 1. For the acks=1 case, I was wondering if there is any way to continue
> > > with the current behavior (ie -- we only need one ack to produce to
> the log
> > > and consider the request complete.) My understanding is that we can
> also
> > > consume from such topics at that point.
> > > If users wanted this lower durability could they set
> min.insync.replicas to
> > > 1?
> > >
> > > 2. For the case where we elect a leader that was unknowingly offline.
> Say
> > > this replica was the only one in ELR. My understanding is that we would
> > > promote it to ISR and remove it from ELR when it is the leader, but
> then we
> > > would remove it from ISR and have no brokers in ISR or ELR. From there
> we
> > > would need to do unclean recovery right?
> > >
> > > 3. Did we address the case where dynamically min isr is increased?
> > >
> > > 4. I think my comment was more about confusion on the KIP. It was not
> clear
> > > to me that the section was describing points if one was done before the
> > > other. But I now see the sentence explaining that. I think I skipped
> from
> > > "delivery plan" to the bullet points.
> > >
> > > Justine
> > >
> > > On Thu, Aug 31, 2023 at 4:04 PM Calvin Liu <caliu@confluent.io.invalid
> >
> > > wrote:
> > >
> > > > Hi Justine
> > > > Thanks for the questions!
> > > >   *a. For my understanding, will we block replication? Or just the
> high
> > > > watermark advancement?*
> > > >   - The replication will not be blocked. The followers are free to
> > > > replicate messages above the HWM. Only HWM advancement is blocked.
> > > >
> > > >   b. *Also in the acks=1 case, if folks want to continue the previous
> > > > behavior, they also need to set min.insync.replicas to 1, correct?*
> > > >   - If the clients only send ack=1 messages and minISR=2. The HWM
> > > behavior
> > > > will only be different when there is 1 replica in the ISR. In this
> case,
> > > > the min ISR does not do much in the current system. It is kind of a
> > > > trade-off but we think it is ok.
> > > >
> > > >   c. *The KIP seems to suggest that we remove from ELR when we start
> up
> > > > again and notice we do not have the clean shutdown file. Is there a
> > > chance
> > > > we have an offline broker in ELR that had an unclean shutdown that we
> > > elect
> > > > as a leader before we get the change to realize the shutdown was
> > > unclean?*
> > > > *  - *The controller will only elect an unfenced(online) replica as
> the
> > > > leader. If a broker has an unclean shutdown, it should register to
> the
> > > > controller first(where it has to declair whether it is a
> clean/unclean
> > > > shutdown) and then start to serve broker requests. So
> > > >      1. If the broker has an unclean shutdown before the controller
> is
> > > > aware that the replica is offline, then the broker can become the
> leader
> > > > temporarily. But it can't serve any Fetch requests before it
> registers
> > > > again, and that's when the controller will re-elect a leader.
> > > >      2. If the controller knows the replica is offline(missing
> heartbeats
> > > > from the broker for a while) before the broker re-registers, the
> broker
> > > > can't be elected as a leader.
> > > >
> > > > d. *Would this be the case for strictly a smaller min ISR?*
> > > > - Yes, only when we have a smaller min ISR. Once the leader is aware
> of
> > > the
> > > > minISR change, the HWM can advance and make the current ELR
> obsolete. So
> > > > the controller should clear the ELR if the ISR >= the new min ISR.
> > > >
> > > > e. *I thought we said the above "Last Leader” behavior can’t be
> > > maintained
> > > > with an empty ISR and it should be removed."*
> > > > -  As the Kip is a big one, we have to consider delivering it in
> phases.
> > > If
> > > > only the Unclean Recovery is delivered, we do not touch the ISR then
> the
> > > > ISR behavior will be the same as the current. I am open to the
> proposal
> > > > that directly starting unclean recovery if the last leader fails.
> Let's
> > > see
> > > > if other folks hope to have more if Unclean Recover delivers first.
> > > >
> > > > On Tue, Aug 29, 2023 at 4:53 PM Justine Olshan
> > > > <jo...@confluent.io.invalid>
> > > > wrote:
> > > >
> > > > > Hey Calvin,
> > > > >
> > > > > Thanks for the KIP. This will close some of the gaps in leader
> > > election!
> > > > I
> > > > > has a few questions:
> > > > >
> > > > > *>* *High Watermark can only advance if the ISR size is larger or
> equal
> > > > > to min.insync.replicas*.
> > > > >
> > > > > For my understanding, will we block replication? Or just the high
> > > > watermark
> > > > > advancement?
> > > > > Also in the acks=1 case, if folks want to continue the previous
> > > behavior,
> > > > > they also need to set min.insync.replicas to 1, correct? It seems
> like
> > > > this
> > > > > change takes some control away from clients when it comes to
> durability
> > > > vs
> > > > > availability.
> > > > >
> > > > > *> *
> > > > > *ELR + ISR size will not be dropped below the min ISR unless the
> > > > controller
> > > > > discovers an ELR member has an unclean shutdown. *
> > > > > The KIP seems to suggest that we remove from ELR when we start up
> again
> > > > and
> > > > > notice we do not have the clean shutdown file. Is there a chance we
> > > have
> > > > an
> > > > > offline broker in ELR that had an unclean shutdown that we elect
> as a
> > > > > leader before we get the change to realize the shutdown was
> unclean?
> > > > > This seems like it could cause some problems. I may have missed
> how we
> > > > > avoid this scenario though.
> > > > >
> > > > > *> When updating the config **min.insync.replicas, *
> > > > > *if the new min ISR <= current ISR, the ELR will be removed.*Would
> this
> > > > be
> > > > > the case for strictly a smaller min ISR? I suppose if we increase
> the
> > > > ISR,
> > > > > we can't reason about ELR. Can we reason about high water mark in
> this
> > > > > case--seems like we will have the broker out of ISR not in ISR or
> ELR?
> > > > > (Forgive me if we can't increase min ISR if the increase will put
> us
> > > > under
> > > > > it)
> > > > >
> > > > > *> Unclean recovery. *
> > > > >
> > > > >    - *The unclean leader election will be replaced by the unclean
> > > > > recovery.*
> > > > >    - *unclean.leader.election.enable will only be replaced by
> > > > >    the unclean.recovery.strategy after ELR is delivered.*
> > > > >    - *As there is no change to the ISR, the "last known leader"
> > > behavior
> > > > is
> > > > >    maintained.*
> > > > >
> > > > > What does "last known leader behavior maintained" mean here? I
> thought
> > > we
> > > > > said *"*The above “*Last Leader” behavior can’t be maintained with
> an
> > > > empty
> > > > > ISR and it should be removed." *My understanding is once metadata
> > > version
> > > > > is updated we will always take the more thoughtful unclean election
> > > > process
> > > > > (ie, inspect the logs)
> > > > >
> > > > > Overall though, the general KIP is pretty solid. Looking at the
> > > rejected
> > > > > alternatives, it looks like a lot was considered, so it's nice to
> see
> > > the
> > > > > final proposal.
> > > > >
> > > > > Justine
> > > > >
> > > > > On Mon, Aug 14, 2023 at 8:50 AM Calvin Liu
> <caliu@confluent.io.invalid
> > > >
> > > > > wrote:
> > > > >
> > > > > >    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.kim@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
> > > > <caliu@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
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Jack Vanlightly <va...@apache.org>.
Hi Calvin,

Regarding partition reassignment, I have two comments.

I notice the KIP says "The AlterPartitionReassignments will not change the ELR" however, when a reassignment completes (or reverts) any replicas removed from the replica set would be removed from the ELR. Sounds obvious but I figured we should be explicit about that.

Reassignment should also respect min.insync.replicas because currently a reassignment can complete as long as the ISR is not empty and all added replicas are members. However, my TLA+ specification, which now includes reassignment, finds single broker failures that can cause committed data loss - despite the added protection of the ELR and min.insync.replicas=2. These scenarios are limited to shrinking the size of the replica set. If we modify the PartitionChangeBuilder to add the completion condition that the target ISR >= min.insync.replicas, then that closes this last single-broker-failure data loss case.

With the above modification, the TLA+ specification of the ELR part of the design is standing up to all safety and liveness checks. The only thing that is not modeled is the unclean recovery though I may leave that as the specification is already very large.

Jack

On 2023/09/01 22:27:10 Calvin Liu wrote:
> Hi Justine
> 1. With the new proposal, in order to let the consumer consume a message
> when only 1 replica commits it to its log, the min ISR has to be set to 1.
> 2. Yes, we will need an unclean recovery if the leader has an unclean
> shutdown.
> 3. If the min ISR config is changed to a larger value, the ISR and ELR will
> not be updated. ISR members are always valid no matter how min ISR changes.
> If ELR is not empty, then the HWM can't advance as well after the min ISR
> increase, so the ELR members are safe to stay.
> 4. I will highlight the explanation. Thanks.
> 
> On Thu, Aug 31, 2023 at 4:35 PM Justine Olshan <jo...@confluent.io.invalid>
> wrote:
> 
> > Hey Calvin,
> >
> > Thanks for the responses. I think I understood most of it, but had a few
> > follow up questions
> >
> > 1. For the acks=1 case, I was wondering if there is any way to continue
> > with the current behavior (ie -- we only need one ack to produce to the log
> > and consider the request complete.) My understanding is that we can also
> > consume from such topics at that point.
> > If users wanted this lower durability could they set min.insync.replicas to
> > 1?
> >
> > 2. For the case where we elect a leader that was unknowingly offline. Say
> > this replica was the only one in ELR. My understanding is that we would
> > promote it to ISR and remove it from ELR when it is the leader, but then we
> > would remove it from ISR and have no brokers in ISR or ELR. From there we
> > would need to do unclean recovery right?
> >
> > 3. Did we address the case where dynamically min isr is increased?
> >
> > 4. I think my comment was more about confusion on the KIP. It was not clear
> > to me that the section was describing points if one was done before the
> > other. But I now see the sentence explaining that. I think I skipped from
> > "delivery plan" to the bullet points.
> >
> > Justine
> >
> > On Thu, Aug 31, 2023 at 4:04 PM Calvin Liu <ca...@confluent.io.invalid>
> > wrote:
> >
> > > Hi Justine
> > > Thanks for the questions!
> > >   *a. For my understanding, will we block replication? Or just the high
> > > watermark advancement?*
> > >   - The replication will not be blocked. The followers are free to
> > > replicate messages above the HWM. Only HWM advancement is blocked.
> > >
> > >   b. *Also in the acks=1 case, if folks want to continue the previous
> > > behavior, they also need to set min.insync.replicas to 1, correct?*
> > >   - If the clients only send ack=1 messages and minISR=2. The HWM
> > behavior
> > > will only be different when there is 1 replica in the ISR. In this case,
> > > the min ISR does not do much in the current system. It is kind of a
> > > trade-off but we think it is ok.
> > >
> > >   c. *The KIP seems to suggest that we remove from ELR when we start up
> > > again and notice we do not have the clean shutdown file. Is there a
> > chance
> > > we have an offline broker in ELR that had an unclean shutdown that we
> > elect
> > > as a leader before we get the change to realize the shutdown was
> > unclean?*
> > > *  - *The controller will only elect an unfenced(online) replica as the
> > > leader. If a broker has an unclean shutdown, it should register to the
> > > controller first(where it has to declair whether it is a clean/unclean
> > > shutdown) and then start to serve broker requests. So
> > >      1. If the broker has an unclean shutdown before the controller is
> > > aware that the replica is offline, then the broker can become the leader
> > > temporarily. But it can't serve any Fetch requests before it registers
> > > again, and that's when the controller will re-elect a leader.
> > >      2. If the controller knows the replica is offline(missing heartbeats
> > > from the broker for a while) before the broker re-registers, the broker
> > > can't be elected as a leader.
> > >
> > > d. *Would this be the case for strictly a smaller min ISR?*
> > > - Yes, only when we have a smaller min ISR. Once the leader is aware of
> > the
> > > minISR change, the HWM can advance and make the current ELR obsolete. So
> > > the controller should clear the ELR if the ISR >= the new min ISR.
> > >
> > > e. *I thought we said the above "Last Leader” behavior can’t be
> > maintained
> > > with an empty ISR and it should be removed."*
> > > -  As the Kip is a big one, we have to consider delivering it in phases.
> > If
> > > only the Unclean Recovery is delivered, we do not touch the ISR then the
> > > ISR behavior will be the same as the current. I am open to the proposal
> > > that directly starting unclean recovery if the last leader fails. Let's
> > see
> > > if other folks hope to have more if Unclean Recover delivers first.
> > >
> > > On Tue, Aug 29, 2023 at 4:53 PM Justine Olshan
> > > <jo...@confluent.io.invalid>
> > > wrote:
> > >
> > > > Hey Calvin,
> > > >
> > > > Thanks for the KIP. This will close some of the gaps in leader
> > election!
> > > I
> > > > has a few questions:
> > > >
> > > > *>* *High Watermark can only advance if the ISR size is larger or equal
> > > > to min.insync.replicas*.
> > > >
> > > > For my understanding, will we block replication? Or just the high
> > > watermark
> > > > advancement?
> > > > Also in the acks=1 case, if folks want to continue the previous
> > behavior,
> > > > they also need to set min.insync.replicas to 1, correct? It seems like
> > > this
> > > > change takes some control away from clients when it comes to durability
> > > vs
> > > > availability.
> > > >
> > > > *> *
> > > > *ELR + ISR size will not be dropped below the min ISR unless the
> > > controller
> > > > discovers an ELR member has an unclean shutdown. *
> > > > The KIP seems to suggest that we remove from ELR when we start up again
> > > and
> > > > notice we do not have the clean shutdown file. Is there a chance we
> > have
> > > an
> > > > offline broker in ELR that had an unclean shutdown that we elect as a
> > > > leader before we get the change to realize the shutdown was unclean?
> > > > This seems like it could cause some problems. I may have missed how we
> > > > avoid this scenario though.
> > > >
> > > > *> When updating the config **min.insync.replicas, *
> > > > *if the new min ISR <= current ISR, the ELR will be removed.*Would this
> > > be
> > > > the case for strictly a smaller min ISR? I suppose if we increase the
> > > ISR,
> > > > we can't reason about ELR. Can we reason about high water mark in this
> > > > case--seems like we will have the broker out of ISR not in ISR or ELR?
> > > > (Forgive me if we can't increase min ISR if the increase will put us
> > > under
> > > > it)
> > > >
> > > > *> Unclean recovery. *
> > > >
> > > >    - *The unclean leader election will be replaced by the unclean
> > > > recovery.*
> > > >    - *unclean.leader.election.enable will only be replaced by
> > > >    the unclean.recovery.strategy after ELR is delivered.*
> > > >    - *As there is no change to the ISR, the "last known leader"
> > behavior
> > > is
> > > >    maintained.*
> > > >
> > > > What does "last known leader behavior maintained" mean here? I thought
> > we
> > > > said *"*The above “*Last Leader” behavior can’t be maintained with an
> > > empty
> > > > ISR and it should be removed." *My understanding is once metadata
> > version
> > > > is updated we will always take the more thoughtful unclean election
> > > process
> > > > (ie, inspect the logs)
> > > >
> > > > Overall though, the general KIP is pretty solid. Looking at the
> > rejected
> > > > alternatives, it looks like a lot was considered, so it's nice to see
> > the
> > > > final proposal.
> > > >
> > > > Justine
> > > >
> > > > On Mon, Aug 14, 2023 at 8:50 AM Calvin Liu <caliu@confluent.io.invalid
> > >
> > > > wrote:
> > > >
> > > > >    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.kim@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
> > > <caliu@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
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> 

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Calvin Liu <ca...@confluent.io.INVALID>.
Hi Justine
1. With the new proposal, in order to let the consumer consume a message
when only 1 replica commits it to its log, the min ISR has to be set to 1.
2. Yes, we will need an unclean recovery if the leader has an unclean
shutdown.
3. If the min ISR config is changed to a larger value, the ISR and ELR will
not be updated. ISR members are always valid no matter how min ISR changes.
If ELR is not empty, then the HWM can't advance as well after the min ISR
increase, so the ELR members are safe to stay.
4. I will highlight the explanation. Thanks.

On Thu, Aug 31, 2023 at 4:35 PM Justine Olshan <jo...@confluent.io.invalid>
wrote:

> Hey Calvin,
>
> Thanks for the responses. I think I understood most of it, but had a few
> follow up questions
>
> 1. For the acks=1 case, I was wondering if there is any way to continue
> with the current behavior (ie -- we only need one ack to produce to the log
> and consider the request complete.) My understanding is that we can also
> consume from such topics at that point.
> If users wanted this lower durability could they set min.insync.replicas to
> 1?
>
> 2. For the case where we elect a leader that was unknowingly offline. Say
> this replica was the only one in ELR. My understanding is that we would
> promote it to ISR and remove it from ELR when it is the leader, but then we
> would remove it from ISR and have no brokers in ISR or ELR. From there we
> would need to do unclean recovery right?
>
> 3. Did we address the case where dynamically min isr is increased?
>
> 4. I think my comment was more about confusion on the KIP. It was not clear
> to me that the section was describing points if one was done before the
> other. But I now see the sentence explaining that. I think I skipped from
> "delivery plan" to the bullet points.
>
> Justine
>
> On Thu, Aug 31, 2023 at 4:04 PM Calvin Liu <ca...@confluent.io.invalid>
> wrote:
>
> > Hi Justine
> > Thanks for the questions!
> >   *a. For my understanding, will we block replication? Or just the high
> > watermark advancement?*
> >   - The replication will not be blocked. The followers are free to
> > replicate messages above the HWM. Only HWM advancement is blocked.
> >
> >   b. *Also in the acks=1 case, if folks want to continue the previous
> > behavior, they also need to set min.insync.replicas to 1, correct?*
> >   - If the clients only send ack=1 messages and minISR=2. The HWM
> behavior
> > will only be different when there is 1 replica in the ISR. In this case,
> > the min ISR does not do much in the current system. It is kind of a
> > trade-off but we think it is ok.
> >
> >   c. *The KIP seems to suggest that we remove from ELR when we start up
> > again and notice we do not have the clean shutdown file. Is there a
> chance
> > we have an offline broker in ELR that had an unclean shutdown that we
> elect
> > as a leader before we get the change to realize the shutdown was
> unclean?*
> > *  - *The controller will only elect an unfenced(online) replica as the
> > leader. If a broker has an unclean shutdown, it should register to the
> > controller first(where it has to declair whether it is a clean/unclean
> > shutdown) and then start to serve broker requests. So
> >      1. If the broker has an unclean shutdown before the controller is
> > aware that the replica is offline, then the broker can become the leader
> > temporarily. But it can't serve any Fetch requests before it registers
> > again, and that's when the controller will re-elect a leader.
> >      2. If the controller knows the replica is offline(missing heartbeats
> > from the broker for a while) before the broker re-registers, the broker
> > can't be elected as a leader.
> >
> > d. *Would this be the case for strictly a smaller min ISR?*
> > - Yes, only when we have a smaller min ISR. Once the leader is aware of
> the
> > minISR change, the HWM can advance and make the current ELR obsolete. So
> > the controller should clear the ELR if the ISR >= the new min ISR.
> >
> > e. *I thought we said the above "Last Leader” behavior can’t be
> maintained
> > with an empty ISR and it should be removed."*
> > -  As the Kip is a big one, we have to consider delivering it in phases.
> If
> > only the Unclean Recovery is delivered, we do not touch the ISR then the
> > ISR behavior will be the same as the current. I am open to the proposal
> > that directly starting unclean recovery if the last leader fails. Let's
> see
> > if other folks hope to have more if Unclean Recover delivers first.
> >
> > On Tue, Aug 29, 2023 at 4:53 PM Justine Olshan
> > <jo...@confluent.io.invalid>
> > wrote:
> >
> > > Hey Calvin,
> > >
> > > Thanks for the KIP. This will close some of the gaps in leader
> election!
> > I
> > > has a few questions:
> > >
> > > *>* *High Watermark can only advance if the ISR size is larger or equal
> > > to min.insync.replicas*.
> > >
> > > For my understanding, will we block replication? Or just the high
> > watermark
> > > advancement?
> > > Also in the acks=1 case, if folks want to continue the previous
> behavior,
> > > they also need to set min.insync.replicas to 1, correct? It seems like
> > this
> > > change takes some control away from clients when it comes to durability
> > vs
> > > availability.
> > >
> > > *> *
> > > *ELR + ISR size will not be dropped below the min ISR unless the
> > controller
> > > discovers an ELR member has an unclean shutdown. *
> > > The KIP seems to suggest that we remove from ELR when we start up again
> > and
> > > notice we do not have the clean shutdown file. Is there a chance we
> have
> > an
> > > offline broker in ELR that had an unclean shutdown that we elect as a
> > > leader before we get the change to realize the shutdown was unclean?
> > > This seems like it could cause some problems. I may have missed how we
> > > avoid this scenario though.
> > >
> > > *> When updating the config **min.insync.replicas, *
> > > *if the new min ISR <= current ISR, the ELR will be removed.*Would this
> > be
> > > the case for strictly a smaller min ISR? I suppose if we increase the
> > ISR,
> > > we can't reason about ELR. Can we reason about high water mark in this
> > > case--seems like we will have the broker out of ISR not in ISR or ELR?
> > > (Forgive me if we can't increase min ISR if the increase will put us
> > under
> > > it)
> > >
> > > *> Unclean recovery. *
> > >
> > >    - *The unclean leader election will be replaced by the unclean
> > > recovery.*
> > >    - *unclean.leader.election.enable will only be replaced by
> > >    the unclean.recovery.strategy after ELR is delivered.*
> > >    - *As there is no change to the ISR, the "last known leader"
> behavior
> > is
> > >    maintained.*
> > >
> > > What does "last known leader behavior maintained" mean here? I thought
> we
> > > said *"*The above “*Last Leader” behavior can’t be maintained with an
> > empty
> > > ISR and it should be removed." *My understanding is once metadata
> version
> > > is updated we will always take the more thoughtful unclean election
> > process
> > > (ie, inspect the logs)
> > >
> > > Overall though, the general KIP is pretty solid. Looking at the
> rejected
> > > alternatives, it looks like a lot was considered, so it's nice to see
> the
> > > final proposal.
> > >
> > > Justine
> > >
> > > On Mon, Aug 14, 2023 at 8:50 AM Calvin Liu <caliu@confluent.io.invalid
> >
> > > wrote:
> > >
> > > >    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.kim@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
> > <caliu@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
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
Hey Calvin,

Thanks for the responses. I think I understood most of it, but had a few
follow up questions

1. For the acks=1 case, I was wondering if there is any way to continue
with the current behavior (ie -- we only need one ack to produce to the log
and consider the request complete.) My understanding is that we can also
consume from such topics at that point.
If users wanted this lower durability could they set min.insync.replicas to
1?

2. For the case where we elect a leader that was unknowingly offline. Say
this replica was the only one in ELR. My understanding is that we would
promote it to ISR and remove it from ELR when it is the leader, but then we
would remove it from ISR and have no brokers in ISR or ELR. From there we
would need to do unclean recovery right?

3. Did we address the case where dynamically min isr is increased?

4. I think my comment was more about confusion on the KIP. It was not clear
to me that the section was describing points if one was done before the
other. But I now see the sentence explaining that. I think I skipped from
"delivery plan" to the bullet points.

Justine

On Thu, Aug 31, 2023 at 4:04 PM Calvin Liu <ca...@confluent.io.invalid>
wrote:

> Hi Justine
> Thanks for the questions!
>   *a. For my understanding, will we block replication? Or just the high
> watermark advancement?*
>   - The replication will not be blocked. The followers are free to
> replicate messages above the HWM. Only HWM advancement is blocked.
>
>   b. *Also in the acks=1 case, if folks want to continue the previous
> behavior, they also need to set min.insync.replicas to 1, correct?*
>   - If the clients only send ack=1 messages and minISR=2. The HWM behavior
> will only be different when there is 1 replica in the ISR. In this case,
> the min ISR does not do much in the current system. It is kind of a
> trade-off but we think it is ok.
>
>   c. *The KIP seems to suggest that we remove from ELR when we start up
> again and notice we do not have the clean shutdown file. Is there a chance
> we have an offline broker in ELR that had an unclean shutdown that we elect
> as a leader before we get the change to realize the shutdown was unclean?*
> *  - *The controller will only elect an unfenced(online) replica as the
> leader. If a broker has an unclean shutdown, it should register to the
> controller first(where it has to declair whether it is a clean/unclean
> shutdown) and then start to serve broker requests. So
>      1. If the broker has an unclean shutdown before the controller is
> aware that the replica is offline, then the broker can become the leader
> temporarily. But it can't serve any Fetch requests before it registers
> again, and that's when the controller will re-elect a leader.
>      2. If the controller knows the replica is offline(missing heartbeats
> from the broker for a while) before the broker re-registers, the broker
> can't be elected as a leader.
>
> d. *Would this be the case for strictly a smaller min ISR?*
> - Yes, only when we have a smaller min ISR. Once the leader is aware of the
> minISR change, the HWM can advance and make the current ELR obsolete. So
> the controller should clear the ELR if the ISR >= the new min ISR.
>
> e. *I thought we said the above "Last Leader” behavior can’t be maintained
> with an empty ISR and it should be removed."*
> -  As the Kip is a big one, we have to consider delivering it in phases. If
> only the Unclean Recovery is delivered, we do not touch the ISR then the
> ISR behavior will be the same as the current. I am open to the proposal
> that directly starting unclean recovery if the last leader fails. Let's see
> if other folks hope to have more if Unclean Recover delivers first.
>
> On Tue, Aug 29, 2023 at 4:53 PM Justine Olshan
> <jo...@confluent.io.invalid>
> wrote:
>
> > Hey Calvin,
> >
> > Thanks for the KIP. This will close some of the gaps in leader election!
> I
> > has a few questions:
> >
> > *>* *High Watermark can only advance if the ISR size is larger or equal
> > to min.insync.replicas*.
> >
> > For my understanding, will we block replication? Or just the high
> watermark
> > advancement?
> > Also in the acks=1 case, if folks want to continue the previous behavior,
> > they also need to set min.insync.replicas to 1, correct? It seems like
> this
> > change takes some control away from clients when it comes to durability
> vs
> > availability.
> >
> > *> *
> > *ELR + ISR size will not be dropped below the min ISR unless the
> controller
> > discovers an ELR member has an unclean shutdown. *
> > The KIP seems to suggest that we remove from ELR when we start up again
> and
> > notice we do not have the clean shutdown file. Is there a chance we have
> an
> > offline broker in ELR that had an unclean shutdown that we elect as a
> > leader before we get the change to realize the shutdown was unclean?
> > This seems like it could cause some problems. I may have missed how we
> > avoid this scenario though.
> >
> > *> When updating the config **min.insync.replicas, *
> > *if the new min ISR <= current ISR, the ELR will be removed.*Would this
> be
> > the case for strictly a smaller min ISR? I suppose if we increase the
> ISR,
> > we can't reason about ELR. Can we reason about high water mark in this
> > case--seems like we will have the broker out of ISR not in ISR or ELR?
> > (Forgive me if we can't increase min ISR if the increase will put us
> under
> > it)
> >
> > *> Unclean recovery. *
> >
> >    - *The unclean leader election will be replaced by the unclean
> > recovery.*
> >    - *unclean.leader.election.enable will only be replaced by
> >    the unclean.recovery.strategy after ELR is delivered.*
> >    - *As there is no change to the ISR, the "last known leader" behavior
> is
> >    maintained.*
> >
> > What does "last known leader behavior maintained" mean here? I thought we
> > said *"*The above “*Last Leader” behavior can’t be maintained with an
> empty
> > ISR and it should be removed." *My understanding is once metadata version
> > is updated we will always take the more thoughtful unclean election
> process
> > (ie, inspect the logs)
> >
> > Overall though, the general KIP is pretty solid. Looking at the rejected
> > alternatives, it looks like a lot was considered, so it's nice to see the
> > final proposal.
> >
> > Justine
> >
> > On Mon, Aug 14, 2023 at 8:50 AM Calvin Liu <ca...@confluent.io.invalid>
> > wrote:
> >
> > >    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.kim@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
> <caliu@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
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Calvin Liu <ca...@confluent.io.INVALID>.
Hi Justine
Thanks for the questions!
  *a. For my understanding, will we block replication? Or just the high
watermark advancement?*
  - The replication will not be blocked. The followers are free to
replicate messages above the HWM. Only HWM advancement is blocked.

  b. *Also in the acks=1 case, if folks want to continue the previous
behavior, they also need to set min.insync.replicas to 1, correct?*
  - If the clients only send ack=1 messages and minISR=2. The HWM behavior
will only be different when there is 1 replica in the ISR. In this case,
the min ISR does not do much in the current system. It is kind of a
trade-off but we think it is ok.

  c. *The KIP seems to suggest that we remove from ELR when we start up
again and notice we do not have the clean shutdown file. Is there a chance
we have an offline broker in ELR that had an unclean shutdown that we elect
as a leader before we get the change to realize the shutdown was unclean?*
*  - *The controller will only elect an unfenced(online) replica as the
leader. If a broker has an unclean shutdown, it should register to the
controller first(where it has to declair whether it is a clean/unclean
shutdown) and then start to serve broker requests. So
     1. If the broker has an unclean shutdown before the controller is
aware that the replica is offline, then the broker can become the leader
temporarily. But it can't serve any Fetch requests before it registers
again, and that's when the controller will re-elect a leader.
     2. If the controller knows the replica is offline(missing heartbeats
from the broker for a while) before the broker re-registers, the broker
can't be elected as a leader.

d. *Would this be the case for strictly a smaller min ISR?*
- Yes, only when we have a smaller min ISR. Once the leader is aware of the
minISR change, the HWM can advance and make the current ELR obsolete. So
the controller should clear the ELR if the ISR >= the new min ISR.

e. *I thought we said the above "Last Leader” behavior can’t be maintained
with an empty ISR and it should be removed."*
-  As the Kip is a big one, we have to consider delivering it in phases. If
only the Unclean Recovery is delivered, we do not touch the ISR then the
ISR behavior will be the same as the current. I am open to the proposal
that directly starting unclean recovery if the last leader fails. Let's see
if other folks hope to have more if Unclean Recover delivers first.

On Tue, Aug 29, 2023 at 4:53 PM Justine Olshan <jo...@confluent.io.invalid>
wrote:

> Hey Calvin,
>
> Thanks for the KIP. This will close some of the gaps in leader election! I
> has a few questions:
>
> *>* *High Watermark can only advance if the ISR size is larger or equal
> to min.insync.replicas*.
>
> For my understanding, will we block replication? Or just the high watermark
> advancement?
> Also in the acks=1 case, if folks want to continue the previous behavior,
> they also need to set min.insync.replicas to 1, correct? It seems like this
> change takes some control away from clients when it comes to durability vs
> availability.
>
> *> *
> *ELR + ISR size will not be dropped below the min ISR unless the controller
> discovers an ELR member has an unclean shutdown. *
> The KIP seems to suggest that we remove from ELR when we start up again and
> notice we do not have the clean shutdown file. Is there a chance we have an
> offline broker in ELR that had an unclean shutdown that we elect as a
> leader before we get the change to realize the shutdown was unclean?
> This seems like it could cause some problems. I may have missed how we
> avoid this scenario though.
>
> *> When updating the config **min.insync.replicas, *
> *if the new min ISR <= current ISR, the ELR will be removed.*Would this be
> the case for strictly a smaller min ISR? I suppose if we increase the ISR,
> we can't reason about ELR. Can we reason about high water mark in this
> case--seems like we will have the broker out of ISR not in ISR or ELR?
> (Forgive me if we can't increase min ISR if the increase will put us under
> it)
>
> *> Unclean recovery. *
>
>    - *The unclean leader election will be replaced by the unclean
> recovery.*
>    - *unclean.leader.election.enable will only be replaced by
>    the unclean.recovery.strategy after ELR is delivered.*
>    - *As there is no change to the ISR, the "last known leader" behavior is
>    maintained.*
>
> What does "last known leader behavior maintained" mean here? I thought we
> said *"*The above “*Last Leader” behavior can’t be maintained with an empty
> ISR and it should be removed." *My understanding is once metadata version
> is updated we will always take the more thoughtful unclean election process
> (ie, inspect the logs)
>
> Overall though, the general KIP is pretty solid. Looking at the rejected
> alternatives, it looks like a lot was considered, so it's nice to see the
> final proposal.
>
> Justine
>
> On Mon, Aug 14, 2023 at 8:50 AM Calvin Liu <ca...@confluent.io.invalid>
> wrote:
>
> >    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.kim@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 <caliu@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
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
Hey Calvin,

Thanks for the KIP. This will close some of the gaps in leader election! I
has a few questions:

*>* *High Watermark can only advance if the ISR size is larger or equal
to min.insync.replicas*.

For my understanding, will we block replication? Or just the high watermark
advancement?
Also in the acks=1 case, if folks want to continue the previous behavior,
they also need to set min.insync.replicas to 1, correct? It seems like this
change takes some control away from clients when it comes to durability vs
availability.

*> *
*ELR + ISR size will not be dropped below the min ISR unless the controller
discovers an ELR member has an unclean shutdown. *
The KIP seems to suggest that we remove from ELR when we start up again and
notice we do not have the clean shutdown file. Is there a chance we have an
offline broker in ELR that had an unclean shutdown that we elect as a
leader before we get the change to realize the shutdown was unclean?
This seems like it could cause some problems. I may have missed how we
avoid this scenario though.

*> When updating the config **min.insync.replicas, *
*if the new min ISR <= current ISR, the ELR will be removed.*Would this be
the case for strictly a smaller min ISR? I suppose if we increase the ISR,
we can't reason about ELR. Can we reason about high water mark in this
case--seems like we will have the broker out of ISR not in ISR or ELR?
(Forgive me if we can't increase min ISR if the increase will put us under
it)

*> Unclean recovery. *

   - *The unclean leader election will be replaced by the unclean recovery.*
   - *unclean.leader.election.enable will only be replaced by
   the unclean.recovery.strategy after ELR is delivered.*
   - *As there is no change to the ISR, the "last known leader" behavior is
   maintained.*

What does "last known leader behavior maintained" mean here? I thought we
said *"*The above “*Last Leader” behavior can’t be maintained with an empty
ISR and it should be removed." *My understanding is once metadata version
is updated we will always take the more thoughtful unclean election process
(ie, inspect the logs)

Overall though, the general KIP is pretty solid. Looking at the rejected
alternatives, it looks like a lot was considered, so it's nice to see the
final proposal.

Justine

On Mon, Aug 14, 2023 at 8:50 AM Calvin Liu <ca...@confluent.io.invalid>
wrote:

>    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 <je...@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
> > >
> >
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Calvin Liu <ca...@confluent.io.INVALID>.
   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 <je...@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
> >
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Jeff Kim <je...@confluent.io.INVALID>.
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
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Colin McCabe <cm...@apache.org>.
On Mon, Sep 25, 2023, at 15:48, Jun Rao wrote:
> Hi, Calvin,
>
> Thanks for the updated KIP. Made another pass. A few more comments below.
>
> 40. unclean.leader.election.enable.false ->
> unclean.recovery.strategy.Balanced: The Balanced mode could still lead to
> data loss. So, I am wondering if unclean.leader.election.enable.false
> should map to None?
>

Hi Jun,

Thanks for the re-review.

The issue with mapping unclean.leader.election.enable=false to unclean.recovery.strategy=None is that it will be a large behavior change for existing users. For example, in the scenario where 3 or more brokers lose power at the same time, if unclean.recovery.strategy=None, the cluster will never recover without manual intervention. I don't think this really is viable for us since users who upgrade will see it as just "broken." In contrast, unclean.recovery.strategy=Balanced implement the current behavior (basically).

We all agree that it's awkward to keep around unclean.leader.election.enable, which is why this KIP deprecates it. We should be able to remove it in 4.0 and just have unclean.recovery.strategy, which will make things clearer.

Agreed on your other comments -- thanks again for reviewing.

best,
Colin


> 41. unclean.recovery.manager.enabled: I am not sure why we introduce this
> additional config. Is it the same as unclean.recovery.strategy=None?
>
> 42. DescribeTopicResponse.TopicAuthorizedOperations: Should this be at the
> topic level?
>
> 43. "Limit: 20 topics max per request": Could we describe what happens if
> the request includes more than 20 topics?
>
> 44. ElectLeadersRequest.DesiredLeaders: Could we describe whether the
> ordering matters?
>
> 45. GetReplicaLogInfo.TopicPartitions: "about": "The topic partitions to
> elect leaders.": The description in "about" is incorrect.
>
> 46. GetReplicaLogInfoResponse: Should we nest partitions under topicId to
> be consistent with other types of responses?
>
> 47. kafka-leader-election.sh:
> 47.1 Could we explain DESIGNATION?
> 47.2 desiredLeader: Should it be a list to match the field in
> ElectLeadersRequest?
>
> 48. We could add a section on downgrade?
>
> 49. LastKnownLeader: This seems only needed in the first phase of
> delivering ELR. Will it be removed when the complete KIP is delivered?
>
> Thanks,
>
> Jun
>
> On Tue, Sep 19, 2023 at 1:30 PM Colin McCabe <cm...@apache.org> wrote:
>
>> Hi Calvin,
>>
>> Thanks for the explanations. I like the idea of using none, balanced,
>> aggressive. We also had an offline discussion about why it is good to use a
>> new config key (basically, so that we can deprecate the old one which had
>> only false/true values in 4.0) With these changes, I am +1.
>>
>> best,
>> Colin
>>
>> On Mon, Sep 18, 2023, at 15:54, Calvin Liu wrote:
>> > Hi Colin,
>> > Also, can we deprecate unclean.leader.election.enable in 4.0? Before
>> that,
>> > we can have both the config unclean.recovery.strategy and
>> > unclean.leader.election.enable
>> > and using the unclean.recovery.Enabled to determine which config to use
>> > during the unclean leader election.
>> >
>> > On Mon, Sep 18, 2023 at 3:51 PM Calvin Liu <ca...@confluent.io> wrote:
>> >
>> >> Hi Colin,
>> >> For the unclean.recovery.strategy config name, how about we use the
>> >> following
>> >> None. It basically means no unclean recovery will be performed.
>> >> Aggressive. It means availability goes first. Whenever the partition
>> can't
>> >> elect a durable replica, the controller will try the unclean recovery.
>> >> Balanced. It is the balance point of the availability first(Aggressive)
>> >> and least availability(None). The controller performs unclean recovery
>> when
>> >> both ISR and ELR are empty.
>> >>
>> >>
>> >> On Fri, Sep 15, 2023 at 11:42 AM Calvin Liu <ca...@confluent.io> wrote:
>> >>
>> >>> Hi Colin,
>> >>>
>> >>> > So, the proposal is that if someone sets
>> "unclean.leader.election.enable
>> >>> = true"...
>> >>>
>> >>>
>> >>> The idea is to use one of the unclean.leader.election.enable and
>> >>> unclean.recovery.strategy based on the unclean.recovery.Enabled. A
>> possible
>> >>> version can be
>> >>>
>> >>> If unclean.recovery.Enabled:
>> >>>
>> >>> {
>> >>>
>> >>> Check unclean.recovery.strategy. If set, use it. Otherwise, check
>> >>> unclean.leader.election.enable and translate it to
>> >>> unclean.recovery.strategy.
>> >>>
>> >>> } else {
>> >>>
>> >>> Use unclean.leader.election.enable
>> >>>
>> >>> }
>> >>>
>> >>>
>> >>> —--------
>> >>>
>> >>> >The configuration key should be "unclean.recovery.manager.enabled",
>> >>> right?
>> >>>
>> >>>
>> >>> I think we have two ways of choosing a leader uncleanly, unclean leader
>> >>> election and unclean recovery(log inspection) and we try to switch
>> between
>> >>> them.
>> >>>
>> >>> Do you mean we want to develop two ways of performing the unclean
>> >>> recovery and one of them is using “unclean recovery manager”? I guess
>> we
>> >>> haven’t discussed the second way.
>> >>>
>> >>>
>> >>> —-------
>> >>>
>> >>> >How do these 4 levels of overrides interact with your new
>> >>> configurations?
>> >>>
>> >>>
>> >>> I do notice in the Kraft controller code, the method to check whether
>> >>> perform unclean leader election is hard coded to false since
>> >>> 2021(uncleanLeaderElectionEnabledForTopic). Isn’t it a good chance to
>> >>> completely deprecate the unclean.leader.election.enable? We don’t even
>> have
>> >>> to worry about the config conversion.
>> >>>
>> >>> On the other hand, whatever the override is, as long as the controller
>> >>> can have the final effective unclean.leader.election.enable, the topic
>> >>> level config unclean.recovery.strategy, the cluster level config
>> >>> unclean.recovery.Enabled, the controller can calculate the correct
>> methods
>> >>> to use right?
>> >>>
>> >>>
>> >>> On Fri, Sep 15, 2023 at 10:02 AM Colin McCabe <cm...@apache.org>
>> wrote:
>> >>>
>> >>>> On Thu, Sep 14, 2023, at 22:23, Calvin Liu wrote:
>> >>>> > Hi Colin
>> >>>> > 1. I think using the new config name is more clear.
>> >>>> >        a. The unclean leader election is actually removed if unclean
>> >>>> > recovery is in use.
>> >>>> >        b. Using multiple values in unclean.leader.election.enable is
>> >>>> > confusing and it will be more confusing after people forget about
>> this
>> >>>> > discussion.
>> >>>>
>> >>>> Hi Calvin,
>> >>>>
>> >>>> So, the proposal is that if someone sets
>> "unclean.leader.election.enable
>> >>>> = true" but then sets one of your new configurations, the value of
>> >>>> unclean.leader.election.enable is ignored? That seems less clear to
>> me, not
>> >>>> more. Just in general, having multiple configuration keys to control
>> the
>> >>>> same thing confuses users. Basically, they are sitting at a giant
>> control
>> >>>> panel, and some of the levers do nothing.
>> >>>>
>> >>>> > 2. Sorry I forgot to mention in the response that I did add the
>> >>>> > unclean.recovery.Enabled flag.
>> >>>>
>> >>>> The configuration key should be "unclean.recovery.manager.enabled",
>> >>>> right? Becuase we can do "unclean recovery" without the manager.
>> Disabling
>> >>>> the manager just means we use a different mechanism for recovery.
>> >>>>
>> >>>> >        c. Maybe I underestimated the challenge of replacing the
>> >>>> config. Any
>> >>>> > implementation problems ahead?
>> >>>>
>> >>>> There are four levels of overrides for unclean.leader.election.enable.
>> >>>>
>> >>>> 1. static configuration for node.
>> >>>>     This goes in the configuration file, typically named
>> >>>> server.properties
>> >>>>
>> >>>> 2. dynamic configuration for node default
>> >>>>   ConfigResource(type=BROKER, name="")
>> >>>>
>> >>>> 3. dynamic configuration for node
>> >>>>   ConfigResource(type=BROKER, name=<controller id>)
>> >>>>
>> >>>> 4. dynamic configuration for topic
>> >>>>   ConfigResource(type=TOPIC, name=<topic-name>)
>> >>>>
>> >>>> How do these 4 levels of overrides interact with your new
>> >>>> configurations? If the new configurations dominate over the old ones,
>> it
>> >>>> seems like this will get a lot more confusing to implement (and also
>> to
>> >>>> use.)
>> >>>>
>> >>>> Again, I'd recommend just adding some new values to
>> >>>> unclean.leader.election.enable. It's simple and will prevent user
>> confusion
>> >>>> (as well as developer confusion.)
>> >>>>
>> >>>> best,
>> >>>> Colin
>> >>>>
>> >>>>
>> >>>> > 3. About the admin client, I mentioned 3 changes in the client.
>> >>>> Anything
>> >>>> > else I missed in the KIP?
>> >>>> >       a. The client will switch to using the new RPC instead of
>> >>>> > MetadataRequest for the topics.
>> >>>> >       b. The TopicPartitionInfo used in TopicDescription needs to
>> add
>> >>>> new
>> >>>> > fields related to the ELR.
>> >>>> >       c. The outputs will add the ELR related fields.
>> >>>> >
>> >>>> > On Thu, Sep 14, 2023 at 9:19 PM Colin McCabe <cm...@apache.org>
>> >>>> wrote:
>> >>>> >
>> >>>> >> Hi Calvin,
>> >>>> >>
>> >>>> >> Thanks for the changes.
>> >>>> >>
>> >>>> >> 1. Earlier I commented that creating "unclean.recovery.strategy "
>> is
>> >>>> not
>> >>>> >> necessary, and we can just reuse the existing
>> >>>> >> "unclean.leader.election.enable" configuration key. Let's discuss
>> >>>> that.
>> >>>> >>
>> >>>> >> 2.I also don't understand why you didn't add a configuration to
>> >>>> enable or
>> >>>> >> disable the Unclean Recovery Manager. This seems like a very simple
>> >>>> way to
>> >>>> >> handle the staging issue which we discussed. The URM can just be
>> >>>> turned off
>> >>>> >> until it is production ready. Let's discuss this.
>> >>>> >>
>> >>>> >> 3. You still need to describe the changes to AdminClient that are
>> >>>> needed
>> >>>> >> to use DescribeTopicRequest.
>> >>>> >>
>> >>>> >> Keep at it. It's looking better. :)
>> >>>> >>
>> >>>> >> best,
>> >>>> >> Colin
>> >>>> >>
>> >>>> >>
>> >>>> >> On Thu, Sep 14, 2023, at 11:03, Calvin Liu wrote:
>> >>>> >> > Hi Colin
>> >>>> >> > Thanks for the comments!
>> >>>> >> >
>> >>>> >> > I did the following changes
>> >>>> >> >
>> >>>> >> >    1.
>> >>>> >> >
>> >>>> >> >    Simplified the API spec section to only include the diff.
>> >>>> >> >    2.
>> >>>> >> >
>> >>>> >> >    Reordered the HWM requirement section.
>> >>>> >> >    3.
>> >>>> >> >
>> >>>> >> >    Removed the URM implementation details to keep the necessary
>> >>>> >> >    characteristics to perform the unclean recovery.
>> >>>> >> >    1.
>> >>>> >> >
>> >>>> >> >       When to perform the unclean recovery
>> >>>> >> >       2.
>> >>>> >> >
>> >>>> >> >       Under different config, how the unclean recovery finds the
>> >>>> leader.
>> >>>> >> >       3.
>> >>>> >> >
>> >>>> >> >       How the config unclean.leader.election.enable and
>> >>>> >> >       unclean.recovery.strategy are converted when users
>> >>>> enable/disable
>> >>>> >> the
>> >>>> >> >       unclean recovery.
>> >>>> >> >       4.
>> >>>> >> >
>> >>>> >> >    More details about how we change admin client.
>> >>>> >> >    5.
>> >>>> >> >
>> >>>> >> >    API limits on the GetReplicaLogInfoRequest and
>> >>>> DescribeTopicRequest.
>> >>>> >> >    6.
>> >>>> >> >
>> >>>> >> >    Two metrics added
>> >>>> >> >    1.
>> >>>> >> >
>> >>>> >> >       Kafka.controller.global_under_min_isr_partition_count
>> >>>> >> >       2.
>> >>>> >> >
>> >>>> >> >       kafka.controller.unclean_recovery_finished_count
>> >>>> >> >
>> >>>> >> >
>> >>>> >> > On Wed, Sep 13, 2023 at 10:46 AM Colin McCabe <
>> cmccabe@apache.org>
>> >>>> >> wrote:
>> >>>> >> >
>> >>>> >> >> On Tue, Sep 12, 2023, at 17:21, Calvin Liu wrote:
>> >>>> >> >> > Hi Colin
>> >>>> >> >> > Thanks for the comments!
>> >>>> >> >> >
>> >>>> >> >>
>> >>>> >> >> Hi Calvin,
>> >>>> >> >>
>> >>>> >> >> Thanks again for the KIP.
>> >>>> >> >>
>> >>>> >> >> One meta-comment: it's usually better to just do a diff on a
>> >>>> message
>> >>>> >> spec
>> >>>> >> >> file or java file if you're including changes to it in the KIP.
>> >>>> This is
>> >>>> >> >> easier to read than looking for "new fields begin" etc. in the
>> >>>> text, and
>> >>>> >> >> gracefully handles the case where existing fields were changed.
>> >>>> >> >>
>> >>>> >> >> > Rewrite the Additional High Watermark advancement requirement
>> >>>> >> >> > There was feedback on this section that some readers may not
>> be
>> >>>> >> familiar
>> >>>> >> >> > with HWM and Ack=0,1,all requests. This can help them
>> understand
>> >>>> the
>> >>>> >> >> > proposal. I will rewrite this part for more readability.
>> >>>> >> >> >
>> >>>> >> >>
>> >>>> >> >> To be clear, I wasn't suggesting dropping either section. I
>> agree
>> >>>> that
>> >>>> >> >> they add useful background. I was just suggesting that we should
>> >>>> discuss
>> >>>> >> >> the "acks" setting AFTER discussing the new high watermark
>> >>>> advancement
>> >>>> >> >> conditions. We also should discuss acks=0. While it isn't
>> >>>> conceptually
>> >>>> >> much
>> >>>> >> >> different than acks=1 here, its omission from this section is
>> >>>> confusing.
>> >>>> >> >>
>> >>>> >> >> > Unclean recovery
>> >>>> >> >> >
>> >>>> >> >> > The plan is to replace the unclean.leader.election.enable with
>> >>>> >> >> > unclean.recovery.strategy. If the Unclean Recovery is enabled
>> >>>> then it
>> >>>> >> >> deals
>> >>>> >> >> > with the three options in the unclean.recovery.strategy.
>> >>>> >> >> >
>> >>>> >> >> >
>> >>>> >> >> > Let’s refine the Unclean Recovery. We have already taken a
>> lot of
>> >>>> >> >> > suggestions and I hope to enhance the durability of Kafka to
>> the
>> >>>> next
>> >>>> >> >> level
>> >>>> >> >> > with this KIP.
>> >>>> >> >>
>> >>>> >> >> I am OK with doing the unclean leader recovery improvements in
>> >>>> this KIP.
>> >>>> >> >> However, I think we need to really work on the configuration
>> >>>> settings.
>> >>>> >> >>
>> >>>> >> >> Configuration overrides are often quite messy. For example, the
>> >>>> cases
>> >>>> >> >> where we have log.roll.hours and log.roll.segment.ms, the user
>> >>>> has to
>> >>>> >> >> remember which one takes precedence, and it is not obvious. So,
>> >>>> rather
>> >>>> >> than
>> >>>> >> >> creating a new configuration, why not add additional values to
>> >>>> >> >> "unclean.leader.election.enable"? I think this will be simpler
>> for
>> >>>> >> people
>> >>>> >> >> to understand, and simpler in the code as well.
>> >>>> >> >>
>> >>>> >> >> What if we continued to use "unclean.leader.election.enable" but
>> >>>> >> extended
>> >>>> >> >> it so that it took a string? Then the string could have these
>> >>>> values:
>> >>>> >> >>
>> >>>> >> >> never
>> >>>> >> >>     never automatically do an unclean leader election under any
>> >>>> >> conditions
>> >>>> >> >>
>> >>>> >> >> false / default
>> >>>> >> >>     only do an unclean leader election if there may be possible
>> >>>> data
>> >>>> >> loss
>> >>>> >> >>
>> >>>> >> >> true / always
>> >>>> >> >>     always do an unclean leader election if we can't immediately
>> >>>> elect a
>> >>>> >> >> leader
>> >>>> >> >>
>> >>>> >> >> It's a bit awkward that false maps to default rather than to
>> >>>> never. But
>> >>>> >> >> this awkwardness exists if we use two different configuration
>> keys
>> >>>> as
>> >>>> >> well.
>> >>>> >> >> The reason for the awkwardness is that we simply don't want most
>> >>>> of the
>> >>>> >> >> people currently setting unclean.leader.election.enable=false to
>> >>>> get the
>> >>>> >> >> "never" behavior. We have to bite that bullet. Better to be
>> clear
>> >>>> and
>> >>>> >> >> explicit than hide it.
>> >>>> >> >>
>> >>>> >> >> Another thing that's a bit awkward is having two different ways
>> to
>> >>>> do
>> >>>> >> >> unclean leader election specified in the KIP. You descirbe two
>> >>>> methods:
>> >>>> >> the
>> >>>> >> >> simple "choose the last leader" method, and the "unclean
>> recovery
>> >>>> >> manager"
>> >>>> >> >> method. I understand why you did it this way -- "choose the last
>> >>>> >> leader" is
>> >>>> >> >> simple, and will help us deliver an implementation quickly,
>> while
>> >>>> the
>> >>>> >> URM
>> >>>> >> >> is preferable in the long term. My suggestion here is to
>> separate
>> >>>> the
>> >>>> >> >> decision of HOW to do unclean leader election from the decision
>> of
>> >>>> WHEN
>> >>>> >> to
>> >>>> >> >> do it.
>> >>>> >> >>
>> >>>> >> >> So in other words, have "unclean.leader.election.enable" specify
>> >>>> when we
>> >>>> >> >> do unclean leader election, and have a new configuration like
>> >>>> >> >> "unclean.recovery.manager.enable" to determine if we use the
>> URM.
>> >>>> >> >> Presumably the URM will take some time to get fully stable, so
>> >>>> this can
>> >>>> >> >> default to false for a while, and we can flip the default to
>> true
>> >>>> when
>> >>>> >> we
>> >>>> >> >> feel ready.
>> >>>> >> >>
>> >>>> >> >> The URM is somewhat under-described here. I think we need a few
>> >>>> >> >> configurations here for it. For example, we need a
>> configuration to
>> >>>> >> specify
>> >>>> >> >> how long it should wait for a broker to respond to its RPCs
>> before
>> >>>> >> moving
>> >>>> >> >> on. We also need to understand how the URM interacts with
>> >>>> >> >> unclean.leader.election.enable=always. I assume that with
>> "always"
>> >>>> we
>> >>>> >> will
>> >>>> >> >> just unconditionally use the URM rather than choosing randomly.
>> >>>> But this
>> >>>> >> >> should be spelled out in the KIP.
>> >>>> >> >>
>> >>>> >> >> >
>> >>>> >> >> > DescribeTopicRequest
>> >>>> >> >> >
>> >>>> >> >> >    1.
>> >>>> >> >> >    Yes, the plan is to replace the MetadataRequest with the
>> >>>> >> >> >    DescribeTopicRequest for the admin clients. Will check the
>> >>>> details.
>> >>>> >> >>
>> >>>> >> >> Sounds good. But as I said, you need to specify how AdminClient
>> >>>> >> interacts
>> >>>> >> >> with the new request. This will involve adding some fields to
>> >>>> >> >> TopicDescription.java. And you need to specify the changes to
>> the
>> >>>> >> >> kafka-topics.sh command line tool. Otherwise we cannot use the
>> >>>> tool to
>> >>>> >> see
>> >>>> >> >> the new information.
>> >>>> >> >>
>> >>>> >> >> The new requests, DescribeTopicRequest and
>> >>>> GetReplicaLogInfoRequest,
>> >>>> >> need
>> >>>> >> >> to have limits placed on them so that their size can't be
>> >>>> infinite. We
>> >>>> >> >> don't want to propagate the current problems of MetadataRequest,
>> >>>> where
>> >>>> >> >> clients can request massive responses that can mess up the JVM
>> when
>> >>>> >> handled.
>> >>>> >> >>
>> >>>> >> >> Adding limits is simple for GetReplicaLogInfoRequest -- we can
>> >>>> just say
>> >>>> >> >> that only 2000 partitions at a time can be requested. For
>> >>>> >> >> DescribeTopicRequest we can probably just limit to 20 topics or
>> >>>> >> something
>> >>>> >> >> like that, to avoid the complexity of doing pagination in this
>> KIP.
>> >>>> >> >>
>> >>>> >> >> >    2.
>> >>>> >> >> >    I can let the broker load the ELR info so that they can
>> serve
>> >>>> the
>> >>>> >> >> >    DescribeTopicRequest as well.
>> >>>> >> >>
>> >>>> >> >> Yes, it's fine to add to MetadataCache. In fact, you'll be
>> loading
>> >>>> it
>> >>>> >> >> anyway once it's added to PartitionImage.
>> >>>> >> >>
>> >>>> >> >> >    3.
>> >>>> >> >> >    Yeah, it does not make sense to have the topic id if
>> >>>> >> >> >    DescribeTopicRequest is only used by the admin client.
>> >>>> >> >>
>> >>>> >> >> OK. That makes things simpler. We can always create a new API
>> later
>> >>>> >> >> (hopefully not in this KIP!) to query by topic ID.
>> >>>> >> >>
>> >>>> >> >> >
>> >>>> >> >> >
>> >>>> >> >> > Metrics
>> >>>> >> >> >
>> >>>> >> >> > As for overall cluster health metrics, I think under-min-ISR
>> is
>> >>>> still
>> >>>> >> a
>> >>>> >> >> > useful one. ELR is more like a safety belt. When the ELR is
>> >>>> used, the
>> >>>> >> >> > cluster availability has already been impacted.
>> >>>> >> >> >
>> >>>> >> >> > Maybe we can have a metric to count the partitions that
>> sum(ISR,
>> >>>> ELR)
>> >>>> >> <
>> >>>> >> >> min
>> >>>> >> >> > ISR. What do you think?
>> >>>> >> >>
>> >>>> >> >> How about:
>> >>>> >> >>
>> >>>> >> >> A.  a metric for the totoal number of under-min-isr partitions?
>> We
>> >>>> don't
>> >>>> >> >> have that in Apache Kafka at the moment.
>> >>>> >> >>
>> >>>> >> >> B. a metric for the number of unclean leader elections we did
>> (for
>> >>>> >> >> simplicity, it can reset to 0 on controller restart: we expect
>> >>>> people to
>> >>>> >> >> monitor the change over time anyway)
>> >>>> >> >>
>> >>>> >> >> best,
>> >>>> >> >> Colin
>> >>>> >> >>
>> >>>> >> >>
>> >>>> >> >> >
>> >>>> >> >> > Yeah, for the ongoing unclean recoveries, the controller can
>> >>>> keep an
>> >>>> >> >> > accurate count through failover because partition registration
>> >>>> can
>> >>>> >> >> indicate
>> >>>> >> >> > whether a recovery is needed. However, for the happened ones,
>> >>>> unless
>> >>>> >> we
>> >>>> >> >> > want to persist the number somewhere, we can only figure it
>> out
>> >>>> from
>> >>>> >> the
>> >>>> >> >> > log.
>> >>>> >> >> >
>> >>>> >> >> > On Tue, Sep 12, 2023 at 3:16 PM Colin McCabe <
>> cmccabe@apache.org
>> >>>> >
>> >>>> >> wrote:
>> >>>> >> >> >
>> >>>> >> >> >> Also, we should have metrics that show what is going on with
>> >>>> regard
>> >>>> >> to
>> >>>> >> >> the
>> >>>> >> >> >> eligible replica set. I'm not sure exactly what to suggest,
>> but
>> >>>> >> >> something
>> >>>> >> >> >> that could identify when things are going wrong in the
>> clsuter.
>> >>>> >> >> >>
>> >>>> >> >> >> For example, maybe a metric for partitions containing
>> replicas
>> >>>> that
>> >>>> >> are
>> >>>> >> >> >> ineligible to be leader? That would show a spike when a
>> broker
>> >>>> had an
>> >>>> >> >> >> unclean restart.
>> >>>> >> >> >>
>> >>>> >> >> >> Ideally, we'd also have a metric that indicates when an
>> unclear
>> >>>> >> leader
>> >>>> >> >> >> election or a recovery happened. It's a bit tricky because
>> the
>> >>>> simple
>> >>>> >> >> >> thing, of tracking it per controller, may be a bit confusing
>> >>>> during
>> >>>> >> >> >> failovers.
>> >>>> >> >> >>
>> >>>> >> >> >> best,
>> >>>> >> >> >> Colin
>> >>>> >> >> >>
>> >>>> >> >> >>
>> >>>> >> >> >> On Tue, Sep 12, 2023, at 14:25, Colin McCabe wrote:
>> >>>> >> >> >> > Hi Calvin,
>> >>>> >> >> >> >
>> >>>> >> >> >> > Thanks for the KIP. I think this is a great improvement.
>> >>>> >> >> >> >
>> >>>> >> >> >> >> Additional High Watermark advance requirement
>> >>>> >> >> >> >
>> >>>> >> >> >> > Typo: change "advance" to "advancement"
>> >>>> >> >> >> >
>> >>>> >> >> >> >> A bit recap of some key concepts.
>> >>>> >> >> >> >
>> >>>> >> >> >> > Typo: change "bit" to "quick"
>> >>>> >> >> >> >
>> >>>> >> >> >> >> Ack=1/all produce request. It defines when the Kafka
>> server
>> >>>> should
>> >>>> >> >> >> respond to the produce request
>> >>>> >> >> >> >
>> >>>> >> >> >> > I think this section would be clearer if we talked about
>> the
>> >>>> new
>> >>>> >> high
>> >>>> >> >> >> > watermark advancement requirement first, and THEN talked
>> >>>> about its
>> >>>> >> >> >> > impact on acks=0, acks=1, and     acks=all.  acks=all is of
>> >>>> course
>> >>>> >> the
>> >>>> >> >> >> > main case we care about here, so it would be good to lead
>> with
>> >>>> >> that,
>> >>>> >> >> >> > rather than delving into the technicalities of acks=0/1
>> first.
>> >>>> >> >> >> >
>> >>>> >> >> >> >> Unclean recovery
>> >>>> >> >> >> >
>> >>>> >> >> >> > So, here you are introducing a new configuration,
>> >>>> >> >> >> > unclean.recovery.strategy. The difficult thing here is that
>> >>>> there
>> >>>> >> is a
>> >>>> >> >> >> > lot of overlap with unclean.leader.election.enable. So we
>> >>>> have 3
>> >>>> >> >> >> > different settings for unclean.recovery.strategy, plus 2
>> >>>> different
>> >>>> >> >> >> > settings for unclean.leader.election.enable, giving a cross
>> >>>> >> product of
>> >>>> >> >> >> > 6 different options. The following "unclean recovery
>> manager"
>> >>>> >> section
>> >>>> >> >> >> > only applies to one fo those 6 different possibilities (I
>> >>>> think?)
>> >>>> >> >> >> >
>> >>>> >> >> >> > I simply don't think we need so many different election
>> types.
>> >>>> >> Really
>> >>>> >> >> >> > the use-cases we need are people who want NO unclean
>> >>>> elections,
>> >>>> >> people
>> >>>> >> >> >> > who want "the reasonable thing" and people who want
>> >>>> avaialbility at
>> >>>> >> >> all
>> >>>> >> >> >> > costs.
>> >>>> >> >> >> >
>> >>>> >> >> >> > Overall, I feel like the first half of the KIP is about the
>> >>>> ELR,
>> >>>> >> and
>> >>>> >> >> >> > the second half is about reworking unclean leader
>> election. It
>> >>>> >> might
>> >>>> >> >> be
>> >>>> >> >> >> > better to move that second half to a separate KIP so that
>> we
>> >>>> can
>> >>>> >> >> figure
>> >>>> >> >> >> > it out fully. It should be fine to punt this until later
>> and
>> >>>> just
>> >>>> >> have
>> >>>> >> >> >> > the current behavior on empty ELR be waiting for the last
>> >>>> known
>> >>>> >> leader
>> >>>> >> >> >> > to return. After all, that's what we do today.
>> >>>> >> >> >> >
>> >>>> >> >> >> >> DescribeTopicRequest
>> >>>> >> >> >> >
>> >>>> >> >> >> > Is the intention for AdminClient to use this RPC for
>> >>>> >> >> >> > Admin#describeTopics ? If so, we need to describe all of
>> the
>> >>>> >> changes
>> >>>> >> >> to
>> >>>> >> >> >> > the admin client API, as well as changes to command-line
>> >>>> tools like
>> >>>> >> >> >> > kafka-topics.sh (if there are any). For example, you will
>> >>>> probably
>> >>>> >> >> need
>> >>>> >> >> >> > changes to TopicDescription.java. You will also need to
>> >>>> provide
>> >>>> >> all of
>> >>>> >> >> >> > the things that admin client needs -- for example,
>> >>>> >> >> >> > TopicAuthorizedOperations.
>> >>>> >> >> >> >
>> >>>> >> >> >> > I also don't think the controller should serve this
>> request.
>> >>>> We
>> >>>> >> want
>> >>>> >> >> to
>> >>>> >> >> >> > minimize load on the controller. Just like with the other
>> >>>> metadata
>> >>>> >> >> >> > requests like MetadataRequest, this should be served by
>> >>>> brokers.
>> >>>> >> >> >> >
>> >>>> >> >> >> > It's a bit confusing why both topic ID and topic name are
>> >>>> provided
>> >>>> >> to
>> >>>> >> >> >> > this API. Is the intention that callers should set one but
>> >>>> not the
>> >>>> >> >> >> > other? Or both? This needs to be clarified. Also, if we do
>> >>>> want to
>> >>>> >> >> >> > support lookups by UUID, that is another thing that needs
>> to
>> >>>> be
>> >>>> >> added
>> >>>> >> >> >> > to adminclient.
>> >>>> >> >> >> >
>> >>>> >> >> >> > In general, I feel like this should also probably be its
>> own
>> >>>> KIP
>> >>>> >> since
>> >>>> >> >> >> > it's fairly complex
>> >>>> >> >> >> >
>> >>>> >> >> >> > best,
>> >>>> >> >> >> > Colin
>> >>>> >> >> >> >
>> >>>> >> >> >> >
>> >>>> >> >> >> > On Thu, Aug 10, 2023, at 15:46, Calvin Liu 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
>> >>>> >> >> >>
>> >>>> >> >>
>> >>>> >>
>> >>>>
>> >>>
>>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Andrew Schofield <an...@outlook.com>.
Hi David,
Thanks for the summary. It seems to me that the Flow-like option is best because
it can easily handle cancellations and exceptions, returning the topic partition information
and signalling when the last of the results have been returned. I think it’s also equally
applicable to any of the other KafkaAdminClient methods which could benefit from
returning results progressively from the broker such as describing all of the consumer groups
on a massive cluster. That would of course be another KIP :)

Thanks,
Andrew

> On 28 Feb 2024, at 22:30, David Arthur <da...@confluent.io.INVALID> wrote:
>
> Andrew/Jose, I like the suggested Flow API. It's also similar to the stream
> observers in GPRC. I'm not sure we should expose something as complex as
> the Flow API directly in KafkaAdminClient, but certainly we can provide a
> similar interface.
>
> ---
> Cancellations:
>
> Another thing not yet discussed is how to cancel in-flight requests. For
> other calls in KafkaAdminClient, we use KafkaFuture which has a "cancel"
> method. With the callback approach, we need to be able to cancel the
> request from within the callback as well as externally. Looking to the Flow
> API again for inspiration, we could have the admin client pass an object to
> the callback which can be used for cancellation. In the simple case, users
> can ignore this object. In the advanced case, they can create a concrete
> class for the callback and cache the cancellation object so it can be
> accessed externally. This would be similar to the Subscription in the Flow
> API.
>
> ---
> Topics / Partitions:
>
> For the case of topic descriptions, we actually have two data types
> interleaved in one stream (topics and partitions). This means if we go with
> TopicDescription in the "onNext" method, we will have a partial set of
> topics in some cases. Also, we will end up calling "onNext" more than once
> for each RPC in the case that a single RPC response spans multiple topics.
>
> One alternative to a single "onNext" would be an interface more tailored to
> the RPC like:
>
> interface DescribeTopicsStreamObserver {
>  // Called for each topic in the result stream.
>  void onTopic(TopicInfo topic);
>
>  // Called for each partition of the topic last handled by onTopic
>  void onPartition(TopicPartitionInfo partition);
>
>  // Called once the broker has finished streaming results to the admin
> client. This marks the end of the stream.
>  void onComplete();
>
>  // Called if an error occurs on the underlying stream. This marks the end
> of the stream.
>  void onError(Throwable t);
> }
>
> ---
> Consumer API:
>
> Offline, there was some discussion about using a simple SAM consumer-like
> interface:
>
> interface AdminResultsConsumer<T> {
>  void onNext(T next, Throwable t);
> }
>
> This has the benefit of being quite simple and letting callers supply a
> lambda instead of a full anonymous class definition. This would use
> nullable arguments like CompletableFuture#whenComplete. We could also use
> an Optional pattern here instead of nullables.
>
> ---
> Summary:
>
> So far, it seems like we are looking at these different options. The main
> difference in terms of API design is if the user will need to implement
> more than one method, or if a lambda can suffice.
>
> 1. Generic, Flow-like interface: AdminResultsSubscriber
> 2. DescribeTopicsStreamObserver (in this message above)
> 3. AdminResultsConsumer
> 4. AdminResultsConsumer with an Optional-like type instead of nullable
> arguments
>
>
>
> -David
>
>
>
>
> On Fri, Feb 23, 2024 at 4:00 PM José Armando García Sancio
> <js...@confluent.io.invalid> wrote:
>
>> Hi Calvin
>>
>> On Fri, Feb 23, 2024 at 9:23 AM Calvin Liu <ca...@confluent.io.invalid>
>> wrote:
>>> As we agreed to implement the pagination for the new API
>>> DescribeTopicPartitions, the client side must also add a proper interface
>>> to handle the pagination.
>>> The current KafkaAdminClient.describeTopics returns
>>> the DescribeTopicsResult which is the future for querying all the topics.
>>> It is awkward to fit the pagination into it because
>>
>> I suggest taking a look at Java's Flow API:
>>
>> https://docs.oracle.com/en/java/javase/21/docs/api/java.base/java/util/concurrent/Flow.html
>> It was design for this specific use case and many libraries integrate with
>> it.
>>
>> If the Kafka client cannot be upgraded to support the Java 9 which
>> introduced that API, you can copy the same interface and semantics.
>> This would allow users to easily integrate with reactive libraries
>> since they all integrate with Java Flow.
>>
>> Thanks,
>> --
>> -José
>>
>
>
> --
> -David



Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by David Arthur <da...@confluent.io.INVALID>.
Andrew/Jose, I like the suggested Flow API. It's also similar to the stream
observers in GPRC. I'm not sure we should expose something as complex as
the Flow API directly in KafkaAdminClient, but certainly we can provide a
similar interface.

---
Cancellations:

Another thing not yet discussed is how to cancel in-flight requests. For
other calls in KafkaAdminClient, we use KafkaFuture which has a "cancel"
method. With the callback approach, we need to be able to cancel the
request from within the callback as well as externally. Looking to the Flow
API again for inspiration, we could have the admin client pass an object to
the callback which can be used for cancellation. In the simple case, users
can ignore this object. In the advanced case, they can create a concrete
class for the callback and cache the cancellation object so it can be
accessed externally. This would be similar to the Subscription in the Flow
API.

---
Topics / Partitions:

For the case of topic descriptions, we actually have two data types
interleaved in one stream (topics and partitions). This means if we go with
TopicDescription in the "onNext" method, we will have a partial set of
topics in some cases. Also, we will end up calling "onNext" more than once
for each RPC in the case that a single RPC response spans multiple topics.

One alternative to a single "onNext" would be an interface more tailored to
the RPC like:

interface DescribeTopicsStreamObserver {
  // Called for each topic in the result stream.
  void onTopic(TopicInfo topic);

  // Called for each partition of the topic last handled by onTopic
  void onPartition(TopicPartitionInfo partition);

  // Called once the broker has finished streaming results to the admin
client. This marks the end of the stream.
  void onComplete();

  // Called if an error occurs on the underlying stream. This marks the end
of the stream.
  void onError(Throwable t);
}

---
Consumer API:

Offline, there was some discussion about using a simple SAM consumer-like
interface:

interface AdminResultsConsumer<T> {
  void onNext(T next, Throwable t);
}

This has the benefit of being quite simple and letting callers supply a
lambda instead of a full anonymous class definition. This would use
nullable arguments like CompletableFuture#whenComplete. We could also use
an Optional pattern here instead of nullables.

---
Summary:

So far, it seems like we are looking at these different options. The main
difference in terms of API design is if the user will need to implement
more than one method, or if a lambda can suffice.

1. Generic, Flow-like interface: AdminResultsSubscriber
2. DescribeTopicsStreamObserver (in this message above)
3. AdminResultsConsumer
4. AdminResultsConsumer with an Optional-like type instead of nullable
arguments



-David




On Fri, Feb 23, 2024 at 4:00 PM José Armando García Sancio
<js...@confluent.io.invalid> wrote:

> Hi Calvin
>
> On Fri, Feb 23, 2024 at 9:23 AM Calvin Liu <ca...@confluent.io.invalid>
> wrote:
> > As we agreed to implement the pagination for the new API
> > DescribeTopicPartitions, the client side must also add a proper interface
> > to handle the pagination.
> > The current KafkaAdminClient.describeTopics returns
> > the DescribeTopicsResult which is the future for querying all the topics.
> > It is awkward to fit the pagination into it because
>
> I suggest taking a look at Java's Flow API:
>
> https://docs.oracle.com/en/java/javase/21/docs/api/java.base/java/util/concurrent/Flow.html
> It was design for this specific use case and many libraries integrate with
> it.
>
> If the Kafka client cannot be upgraded to support the Java 9 which
> introduced that API, you can copy the same interface and semantics.
> This would allow users to easily integrate with reactive libraries
> since they all integrate with Java Flow.
>
> Thanks,
> --
> -José
>


-- 
-David

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by José Armando García Sancio <js...@confluent.io.INVALID>.
Hi Calvin

On Fri, Feb 23, 2024 at 9:23 AM Calvin Liu <ca...@confluent.io.invalid> wrote:
> As we agreed to implement the pagination for the new API
> DescribeTopicPartitions, the client side must also add a proper interface
> to handle the pagination.
> The current KafkaAdminClient.describeTopics returns
> the DescribeTopicsResult which is the future for querying all the topics.
> It is awkward to fit the pagination into it because

I suggest taking a look at Java's Flow API:
https://docs.oracle.com/en/java/javase/21/docs/api/java.base/java/util/concurrent/Flow.html
It was design for this specific use case and many libraries integrate with it.

If the Kafka client cannot be upgraded to support the Java 9 which
introduced that API, you can copy the same interface and semantics.
This would allow users to easily integrate with reactive libraries
since they all integrate with Java Flow.

Thanks,
-- 
-José

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Andrew Schofield <an...@outlook.com>.
This is an interesting problem. While it would be possible to use a Consumer,
it can’t handle exceptions really. A java.util.Stream has a similar problem.

I wonder whether an interface which looks like java.util.concurrent.Flow.Subscriber
would be good. Something like:

public interface AdminResultsSubscriber<T> {
  void onComplete();
  void onError(Exception e):
  void onNext(T result);
}

And then:
Admin.describeTopics(TopicCollection, DescribeTopicOptions, AdminResultsSubscriber<TopicDescription>)

Thanks,
Andrew


> On 23 Feb 2024, at 17:56, David Arthur <da...@confluent.io.INVALID> wrote:
> 
> Thanks for raising this here, Calvin. Since this is the first "streaming
> results" type API in KafkaAdminClient (as far as I know), we're treading
> new ground here.
> 
> As you mentioned, we can either accept a consumer or return some iterable
> result. Returning a java.util.Stream is also an option, and a bit more
> modern/convenient than java.util.Iterator. Personally, I like the consumer
> approach, but I'm interested in hearing other's opinions.
> 
> This actually brings up another question: Do we think it's safe to assume
> that one topic's description can fit into memory? The RPC supports paging
> across partitions within a single topic, so maybe the admin API should as
> well?
> 
> -David
> 
> On Fri, Feb 23, 2024 at 12:22 PM Calvin Liu <ca...@confluent.io> wrote:
> 
>> Hey,
>> As we agreed to implement the pagination for the new API
>> DescribeTopicPartitions, the client side must also add a proper interface
>> to handle the pagination.
>> The current KafkaAdminClient.describeTopics returns
>> the DescribeTopicsResult which is the future for querying all the topics.
>> It is awkward to fit the pagination into it because
>> 
>>   1. Each future corresponds to a topic. We also want to have the
>>   pagination on huge topics for their partitions.
>>   2. To avoid OOM, we should only fetch the new topics when we need them
>>   and release the used topics. Especially the main use case of looping the
>>   topic list is when the client prints all the topics.
>> 
>> So, to better serve the pagination, @David Arthur
>> <da...@confluent.io> suggested to add a new interface in the Admin
>> client between the following 2.
>> 
>> describeTopics(TopicCollection topics, DescribeTopicsOptions options, Consumer<TopicDescription>);
>> 
>> Iterator<TopicDescription> describeTopics(TopicCollection topics, DescribeTopicsOptions options);
>> 
>> David and I would prefer the first Consumer version which works better as a stream purposes.
>> 
>> 
>> On Wed, Oct 11, 2023 at 4:28 PM Calvin Liu <ca...@confluent.io> wrote:
>> 
>>> Hi David,
>>> Thanks for the comment.
>>> Yes, we can separate the ELR enablement from the metadata version. It is
>>> also helpful to avoid blocking the following MV releases if the user is not
>>> ready for ELR.
>>> One thing to correct is that, the Unclean recovery is controlled
>>> by unclean.recovery.manager.enabled, a separate config
>>> from unclean.recovery.strategy. It determines whether unclean recovery will
>>> be used in an unclean leader election.
>>> Thanks
>>> 
>>> On Wed, Oct 11, 2023 at 4:11 PM David Arthur <mu...@gmail.com> wrote:
>>> 
>>>> One thing we should consider is a static config to totally enable/disable
>>>> the ELR feature. If I understand the KIP correctly, we can effectively
>>>> disable the unclean recovery by setting the recovery strategy config to
>>>> "none".
>>>> 
>>>> This would make development and rollout of this feature a bit smoother.
>>>> Consider the case that we find bugs in ELR after a cluster has updated to
>>>> its MetadataVersion. It's simpler to disable the feature through config
>>>> rather than going through a MetadataVersion downgrade (once that's
>>>> supported).
>>>> 
>>>> Does that make sense?
>>>> 
>>>> -David
>>>> 
>>>> On Wed, Oct 11, 2023 at 1:40 PM Calvin Liu <ca...@confluent.io.invalid>
>>>> wrote:
>>>> 
>>>>> Hi Jun
>>>>> -Good catch, yes, we don't need the -1 in the DescribeTopicRequest.
>>>>> -No new value is added. The LeaderRecoveryState will still be set to 1
>>>> if
>>>>> we have an unclean leader election. The unclean leader election
>>>> includes
>>>>> the old random way and the unclean recovery. During the unclean
>>>> recovery,
>>>>> the LeaderRecoveryState will not change until the controller decides to
>>>>> update the records with the new leader.
>>>>> Thanks
>>>>> 
>>>>> On Wed, Oct 11, 2023 at 9:02 AM Jun Rao <ju...@confluent.io.invalid>
>>>> wrote:
>>>>> 
>>>>>> Hi, Calvin,
>>>>>> 
>>>>>> Another thing. Currently, when there is an unclean leader election,
>>>> we
>>>>> set
>>>>>> the LeaderRecoveryState in PartitionRecord and PartitionChangeRecord
>>>> to
>>>>> 1.
>>>>>> With the KIP, will there be new values for LeaderRecoveryState? If
>>>> not,
>>>>>> when will LeaderRecoveryState be set to 1?
>>>>>> 
>>>>>> Thanks,
>>>>>> 
>>>>>> Jun
>>>>>> 
>>>>>> On Tue, Oct 10, 2023 at 4:24 PM Jun Rao <ju...@confluent.io> wrote:
>>>>>> 
>>>>>>> Hi, Calvin,
>>>>>>> 
>>>>>>> One more comment.
>>>>>>> 
>>>>>>> "The first partition to fetch details for. -1 means to fetch all
>>>>>>> partitions." It seems that FirstPartitionId of 0 naturally means
>>>>> fetching
>>>>>>> all partitions?
>>>>>>> 
>>>>>>> Thanks,
>>>>>>> 
>>>>>>> Jun
>>>>>>> 
>>>>>>> On Tue, Oct 10, 2023 at 12:40 PM Calvin Liu
>>>> <caliu@confluent.io.invalid
>>>>>> 
>>>>>>> wrote:
>>>>>>> 
>>>>>>>> Hi Jun,
>>>>>>>> Yeah, with the current Metadata request handling, we only return
>>>>> errors
>>>>>> on
>>>>>>>> the Topic level, like topic not found. It seems that querying a
>>>>> specific
>>>>>>>> partition is not a valid use case. Will update.
>>>>>>>> Thanks
>>>>>>>> 
>>>>>>>> On Tue, Oct 10, 2023 at 11:55 AM Jun Rao <jun@confluent.io.invalid
>>>>> 
>>>>>>>> wrote:
>>>>>>>> 
>>>>>>>>> Hi, Calvin,
>>>>>>>>> 
>>>>>>>>> 60.  If the range query has errors for some of the partitions,
>>>> do we
>>>>>>>> expect
>>>>>>>>> different responses when querying particular partitions?
>>>>>>>>> 
>>>>>>>>> Thanks,
>>>>>>>>> 
>>>>>>>>> Jun
>>>>>>>>> 
>>>>>>>>> On Tue, Oct 10, 2023 at 10:50 AM Calvin Liu
>>>>>> <caliu@confluent.io.invalid
>>>>>>>>> 
>>>>>>>>> wrote:
>>>>>>>>> 
>>>>>>>>>> Hi Jun
>>>>>>>>>> 60. Yes, it is a good question. I was thinking the API could
>>>> be
>>>>>>>> flexible
>>>>>>>>> to
>>>>>>>>>> query the particular partitions if the range query has errors
>>>> for
>>>>>>>> some of
>>>>>>>>>> the partitions. Not sure whether it is a valid assumption,
>>>> what do
>>>>>> you
>>>>>>>>>> think?
>>>>>>>>>> 
>>>>>>>>>> 61. Good point, I will update them to partition level with the
>>>>> same
>>>>>>>>> limit.
>>>>>>>>>> 
>>>>>>>>>> 62. Sure, will do.
>>>>>>>>>> 
>>>>>>>>>> Thanks
>>>>>>>>>> 
>>>>>>>>>> On Tue, Oct 10, 2023 at 10:12 AM Jun Rao
>>>> <jun@confluent.io.invalid
>>>>>> 
>>>>>>>>> wrote:
>>>>>>>>>> 
>>>>>>>>>>> Hi, Calvin,
>>>>>>>>>>> 
>>>>>>>>>>> A few more minor comments on your latest update.
>>>>>>>>>>> 
>>>>>>>>>>> 60. DescribeTopicRequest: When will the Partitions field be
>>>>> used?
>>>>>> It
>>>>>>>>>> seems
>>>>>>>>>>> that the FirstPartitionId field is enough for AdminClient
>>>> usage.
>>>>>>>>>>> 
>>>>>>>>>>> 61. Could we make the limit for DescribeTopicRequest,
>>>>>>>>>> ElectLeadersRequest,
>>>>>>>>>>> GetReplicaLogInfo consistent? Currently,
>>>> ElectLeadersRequest's
>>>>>>>> limit is
>>>>>>>>>> at
>>>>>>>>>>> topic level and GetReplicaLogInfo has a different partition
>>>>> level
>>>>>>>> limit
>>>>>>>>>>> from DescribeTopicRequest.
>>>>>>>>>>> 
>>>>>>>>>>> 62. Should ElectLeadersRequest.DesiredLeaders be at the same
>>>>> level
>>>>>>>> as
>>>>>>>>>>> ElectLeadersRequest.TopicPartitions.Partitions? In the KIP,
>>>> it
>>>>>> looks
>>>>>>>>> like
>>>>>>>>>>> it's at the same level as
>>>> ElectLeadersRequest.TopicPartitions.
>>>>>>>>>>> 
>>>>>>>>>>> Thanks,
>>>>>>>>>>> 
>>>>>>>>>>> Jun
>>>>>>>>>>> 
>>>>>>>>>>> On Wed, Oct 4, 2023 at 3:55 PM Calvin Liu
>>>>>>>> <ca...@confluent.io.invalid>
>>>>>>>>>>> wrote:
>>>>>>>>>>> 
>>>>>>>>>>>> Hi David,
>>>>>>>>>>>> Thanks for the comments.
>>>>>>>>>>>> ----
>>>>>>>>>>>> I thought that a new snapshot with the downgraded MV is
>>>>> created
>>>>>> in
>>>>>>>>> this
>>>>>>>>>>>> case. Isn’t it the case?
>>>>>>>>>>>> Yes, you are right, a metadata delta will be generated
>>>> after
>>>>> the
>>>>>>>> MV
>>>>>>>>>>>> downgrade. Then the user can start the software downgrade.
>>>>>>>>>>>> -----
>>>>>>>>>>>> Could you also elaborate a bit more on the reasoning
>>>> behind
>>>>>> adding
>>>>>>>>> the
>>>>>>>>>>>> limits to the admin RPCs? This is a new pattern in Kafka
>>>> so it
>>>>>>>> would
>>>>>>>>> be
>>>>>>>>>>>> good to clear on the motivation.
>>>>>>>>>>>> Thanks to Colin for bringing it up. The current
>>>>> MetadataRequest
>>>>>>>> does
>>>>>>>>>> not
>>>>>>>>>>>> have a limit on the number of topics to query in a single
>>>>>> request.
>>>>>>>>>>> Massive
>>>>>>>>>>>> requests can mess up the JVM. We want to have some sort of
>>>>>>>> throttle
>>>>>>>>> on
>>>>>>>>>>> the
>>>>>>>>>>>> new APIs.
>>>>>>>>>>>> -----
>>>>>>>>>>>> Could you also explain how the client is supposed to
>>>> handle
>>>>> the
>>>>>>>>>>>> topics/partitions above the limit? I suppose that it will
>>>> have
>>>>>> to
>>>>>>>>> retry
>>>>>>>>>>>> those, correct?
>>>>>>>>>>>> Corrent. For the official admin clients, it will split the
>>>>> large
>>>>>>>>>> request
>>>>>>>>>>>> into proper pieces and query one after another.
>>>>>>>>>>>> -----
>>>>>>>>>>>> My understanding is that the topics/partitions above the
>>>> limit
>>>>>>>> will
>>>>>>>>> be
>>>>>>>>>>>> failed with an invalid exception error. I wonder if this
>>>>> choice
>>>>>> is
>>>>>>>>>>>> judicious because the invalide request exception is
>>>> usually
>>>>>>>> fatal. It
>>>>>>>>>> may
>>>>>>>>>>>> be better to use an new and explicit error for this case.
>>>>>>>>>>>> 
>>>>>>>>>>>> Thanks for bringing this up. How about
>>>>> "REQUEST_LIMIT_REACHED"?
>>>>>>>>>>>> --------
>>>>>>>>>>>> It seems that we still need to specify the changes to the
>>>>> admin
>>>>>>>> api
>>>>>>>>> to
>>>>>>>>>>>> accommodate the new or updated apis. Do you plan to add
>>>> them?
>>>>>>>>>>>> Try to cover the following
>>>>>>>>>>>> 1. The admin client will use the new DescribeTopicRequest
>>>> to
>>>>>> query
>>>>>>>>> the
>>>>>>>>>>>> topics
>>>>>>>>>>>> 2. Mention the API limit and the new retriable error.
>>>>>>>>>>>> 3. Output changes for the admin client when describing a
>>>> topic
>>>>>>>> (new
>>>>>>>>>>> fields
>>>>>>>>>>>> of ELR...)
>>>>>>>>>>>> 4. Changes to data structures like TopicPartitionInfo to
>>>>> include
>>>>>>>> the
>>>>>>>>>> ELR.
>>>>>>>>>>>> Anything else I missed?
>>>>>>>>>>>> 
>>>>>>>>>>>> Thanks!
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> On Wed, Oct 4, 2023 at 12:27 PM David Jacot <
>>>>>>>> david.jacot@gmail.com>
>>>>>>>>>>> wrote:
>>>>>>>>>>>> 
>>>>>>>>>>>>> Hi Calvin,
>>>>>>>>>>>>> 
>>>>>>>>>>>>> I thought that a new snapshot with the downgraded MV is
>>>>>> created
>>>>>>>> in
>>>>>>>>>> this
>>>>>>>>>>>>> case. Isn’t it the case?
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Could you also elaborate a bit more on the reasoning
>>>> behind
>>>>>>>> adding
>>>>>>>>>> the
>>>>>>>>>>>>> limits to the admin RPCs? This is a new pattern in
>>>> Kafka so
>>>>> it
>>>>>>>>> would
>>>>>>>>>> be
>>>>>>>>>>>>> good to clear on the motivation.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Could you also explain how the client is supposed to
>>>> handle
>>>>>> the
>>>>>>>>>>>>> topics/partitions above the limit? I suppose that it
>>>> will
>>>>> have
>>>>>>>> to
>>>>>>>>>> retry
>>>>>>>>>>>>> those, correct?
>>>>>>>>>>>>> 
>>>>>>>>>>>>> My understanding is that the topics/partitions above the
>>>>> limit
>>>>>>>> will
>>>>>>>>>> be
>>>>>>>>>>>>> failed with an invalid exception error. I wonder if this
>>>>>> choice
>>>>>>>> is
>>>>>>>>>>>>> judicious because the invalide request exception is
>>>> usually
>>>>>>>> fatal.
>>>>>>>>> It
>>>>>>>>>>> may
>>>>>>>>>>>>> be better to use an new and explicit error for this
>>>> case.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> It seems that we still need to specify the changes to
>>>> the
>>>>>> admin
>>>>>>>> api
>>>>>>>>>> to
>>>>>>>>>>>>> accommodate the new or updated apis. Do you plan to add
>>>>> them?
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Best,
>>>>>>>>>>>>> David
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Le mer. 4 oct. 2023 à 20:39, Calvin Liu
>>>>>>>> <caliu@confluent.io.invalid
>>>>>>>>>> 
>>>>>>>>>> a
>>>>>>>>>>>>> écrit :
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Hi Jun,
>>>>>>>>>>>>>> After the MV downgrade, the controller will write in
>>>> the
>>>>> old
>>>>>>>>>> version
>>>>>>>>>>> of
>>>>>>>>>>>>> the
>>>>>>>>>>>>>> PartitionRecord/PartitionChangeRecord. If I understand
>>>>>>>> correctly,
>>>>>>>>>> it
>>>>>>>>>>> is
>>>>>>>>>>>>>> possible to downgrade the software version if the
>>>>> controller
>>>>>>>> only
>>>>>>>>>> has
>>>>>>>>>>>> to
>>>>>>>>>>>>>> handle old version records.
>>>>>>>>>>>>>> However, the controller will not automatically
>>>> rewrite the
>>>>>>>>>>>>> PartitionRecord
>>>>>>>>>>>>>> with the old version unless there is a partition
>>>> update.
>>>>>> Then,
>>>>>>>>> the
>>>>>>>>>>> user
>>>>>>>>>>>>> may
>>>>>>>>>>>>>> have to wait an unknown amount of time before the
>>>> software
>>>>>>>>>> downgrades
>>>>>>>>>>>>>> unless they do a roll to force update every
>>>> partition. If
>>>>> it
>>>>>>>>> makes
>>>>>>>>>>>>> sense, I
>>>>>>>>>>>>>> can mention these steps to do a software downgrade.
>>>>>>>>>>>>>> Thanks
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> On Wed, Oct 4, 2023 at 11:20 AM Jun Rao
>>>>>>>> <jun@confluent.io.invalid
>>>>>>>>>> 
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Hi, Calvin and Justine,
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Historically, when we change the record format in
>>>> the
>>>>> log,
>>>>>>>> we
>>>>>>>>>> don't
>>>>>>>>>>>>>> support
>>>>>>>>>>>>>>> software version downgrading.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> For the record format change in the metadata log,
>>>> have
>>>>> we
>>>>>>>>> thought
>>>>>>>>>>>> about
>>>>>>>>>>>>>>> forcing the write of the latest metadata records
>>>> with
>>>>> the
>>>>>>>> old
>>>>>>>>>>> version
>>>>>>>>>>>>>>> during MV downgrading? This will in theory allow
>>>> the old
>>>>>>>>> version
>>>>>>>>>> of
>>>>>>>>>>>> the
>>>>>>>>>>>>>>> software to obtain the latest metadata.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Jun
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> On Wed, Oct 4, 2023 at 9:53 AM Justine Olshan
>>>>>>>>>>>>>> <jolshan@confluent.io.invalid
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Sorry -- not MV but software version.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> On Wed, Oct 4, 2023 at 9:51 AM Justine Olshan <
>>>>>>>>>>>> jolshan@confluent.io>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Catching up with this discussion.
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> I was just curious -- have we had other
>>>> instances
>>>>>> where
>>>>>>>>>>>> downgrading
>>>>>>>>>>>>>> MV
>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>> not supported? I think Kafka typically tries to
>>>>>> support
>>>>>>>>>>>> downgrades,
>>>>>>>>>>>>>>> and I
>>>>>>>>>>>>>>>>> couldn't think of other examples.
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> On Wed, Oct 4, 2023 at 9:40 AM Calvin Liu
>>>>>>>>>>>>> <caliu@confluent.io.invalid
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> Hi Jun,
>>>>>>>>>>>>>>>>>> 54. Marked the software downgrading is not
>>>>> supported.
>>>>>>>> As
>>>>>>>>> the
>>>>>>>>>>> old
>>>>>>>>>>>>>>>>>> controller
>>>>>>>>>>>>>>>>>> will not understand the new PartitionRecord and
>>>>>>>>>>>>>> PartitionChangeRecord.
>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> On Wed, Oct 4, 2023 at 9:12 AM Jun Rao
>>>>>>>>>>> <jun@confluent.io.invalid
>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Hi, Calvin,
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Thanks for the reply. Just one more comment.
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 54. It seems that downgrading MV is
>>>> supported. Is
>>>>>>>>>>> downgrading
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> software
>>>>>>>>>>>>>>>>>>> version supported? It would be useful to
>>>> document
>>>>>>>> that.
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Jun
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> On Tue, Oct 3, 2023 at 4:55 PM Artem Livshits
>>>>>>>>>>>>>>>>>>> <al...@confluent.io.invalid> wrote:
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Hi Colin,
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> I think in your example
>>>> "do_unclean_recovery"
>>>>>> would
>>>>>>>>> need
>>>>>>>>>>> to
>>>>>>>>>>>> do
>>>>>>>>>>>>>>>>>> different
>>>>>>>>>>>>>>>>>>>> things depending on the strategy.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> do_unclean_recovery() {
>>>>>>>>>>>>>>>>>>>>   if (unclean.recovery.manager.enabled) {
>>>>>>>>>>>>>>>>>>>>    if (strategy == Aggressive)
>>>>>>>>>>>>>>>>>>>>      use
>>>>>>>>> UncleanRecoveryManager(waitLastKnownERL=false)
>>>>>>>>>>> //
>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>> inspect
>>>>>>>>>>>>>>>>>>>> logs from whoever is available
>>>>>>>>>>>>>>>>>>>>    else
>>>>>>>>>>>>>>>>>>>>      use
>>>>>>>>> UncleanRecoveryManager(waitLastKnownERL=true)
>>>>>>>>>>> //
>>>>>>>>>>>>>> must
>>>>>>>>>>>>>>>> wait
>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>> at least last known ELR
>>>>>>>>>>>>>>>>>>>>  } else {
>>>>>>>>>>>>>>>>>>>>    if (strategy == Aggressive)
>>>>>>>>>>>>>>>>>>>>      choose the last known leader if that
>>>> is
>>>>>>>>> available,
>>>>>>>>>>> or
>>>>>>>>>>>> a
>>>>>>>>>>>>>>> random
>>>>>>>>>>>>>>>>>>> leader
>>>>>>>>>>>>>>>>>>>> if not)
>>>>>>>>>>>>>>>>>>>>    else
>>>>>>>>>>>>>>>>>>>>      wait for last known leader to get
>>>> back
>>>>>>>>>>>>>>>>>>>>  }
>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> The idea is that the Aggressive strategy
>>>> would
>>>>>>>> kick in
>>>>>>>>>> as
>>>>>>>>>>>> soon
>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>> lost
>>>>>>>>>>>>>>>>>>>> the leader and would pick a leader from
>>>> whoever
>>>>>> is
>>>>>>>>>>>> available;
>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> Balanced will only kick in when ELR is
>>>> empty
>>>>> and
>>>>>>>> will
>>>>>>>>>> wait
>>>>>>>>>>>> for
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> brokers
>>>>>>>>>>>>>>>>>>>> that likely have most data to be available.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> On Tue, Oct 3, 2023 at 3:04 PM Colin
>>>> McCabe <
>>>>>>>>>>>>> cmccabe@apache.org
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> On Tue, Oct 3, 2023, at 10:49, Jun Rao
>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>> Hi, Calvin,
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> Thanks for the update KIP. A few more
>>>>>> comments.
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 41. Why would a user choose the option
>>>> to
>>>>>>>> select a
>>>>>>>>>>>> random
>>>>>>>>>>>>>>>> replica
>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> leader instead of using
>>>>>>>>>>>>> unclean.recovery.strateg=Aggressive?
>>>>>>>>>>>>>>> It
>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>> the latter is strictly better? If
>>>> that's
>>>>> not
>>>>>>>> the
>>>>>>>>>> case,
>>>>>>>>>>>>> could
>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>> fold
>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>> option under unclean.recovery.strategy
>>>>>> instead
>>>>>>>> of
>>>>>>>>>>>>>> introducing
>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>> separate
>>>>>>>>>>>>>>>>>>>>>> config?
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> Hi Jun,
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> I thought the flow of control was:
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> If there is no leader for the partition {
>>>>>>>>>>>>>>>>>>>>>  If (there are unfenced ELR members) {
>>>>>>>>>>>>>>>>>>>>>    choose_an_unfenced_ELR_member
>>>>>>>>>>>>>>>>>>>>>  } else if (there are fenced ELR
>>>> members AND
>>>>>>>>>>>>>>>> strategy=Aggressive) {
>>>>>>>>>>>>>>>>>>>>>    do_unclean_recovery
>>>>>>>>>>>>>>>>>>>>>  } else if (there are no ELR members AND
>>>>>>>> strategy
>>>>>>>>> !=
>>>>>>>>>>>> None)
>>>>>>>>>>>>> {
>>>>>>>>>>>>>>>>>>>>>    do_unclean_recovery
>>>>>>>>>>>>>>>>>>>>>  } else {
>>>>>>>>>>>>>>>>>>>>>    do nothing about the missing leader
>>>>>>>>>>>>>>>>>>>>>  }
>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> do_unclean_recovery() {
>>>>>>>>>>>>>>>>>>>>>   if (unclean.recovery.manager.enabled)
>>>> {
>>>>>>>>>>>>>>>>>>>>>    use UncleanRecoveryManager
>>>>>>>>>>>>>>>>>>>>>  } else {
>>>>>>>>>>>>>>>>>>>>>    choose the last known leader if that
>>>> is
>>>>>>>>> available,
>>>>>>>>>>> or
>>>>>>>>>>>> a
>>>>>>>>>>>>>>> random
>>>>>>>>>>>>>>>>>>> leader
>>>>>>>>>>>>>>>>>>>>> if not)
>>>>>>>>>>>>>>>>>>>>>  }
>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> However, I think this could be clarified,
>>>>>>>> especially
>>>>>>>>>> the
>>>>>>>>>>>>>>> behavior
>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>> unclean.recovery.manager.enabled=false.
>>>>>>>> Inuitively
>>>>>>>>> the
>>>>>>>>>>>> goal
>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>> unclean.recovery.manager.enabled=false
>>>> is to
>>>>> be
>>>>>>>> "the
>>>>>>>>>>> same
>>>>>>>>>>>> as
>>>>>>>>>>>>>>> now,
>>>>>>>>>>>>>>>>>>> mostly"
>>>>>>>>>>>>>>>>>>>>> but it's very underspecified in the KIP,
>>>> I
>>>>>> agree.
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 50. ElectLeadersRequest: "If more than
>>>> 20
>>>>>>>> topics
>>>>>>>>> are
>>>>>>>>>>>>>> included,
>>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> first 20 will be served. Others will be
>>>>>>>> returned
>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>> DesiredLeaders."
>>>>>>>>>>>>>>>>>>>>> Hmm,
>>>>>>>>>>>>>>>>>>>>>> not sure that I understand this.
>>>>>>>>>> ElectLeadersResponse
>>>>>>>>>>>>>> doesn't
>>>>>>>>>>>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>>> DesiredLeaders field.
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 51. GetReplicaLogInfo: "If more than
>>>> 2000
>>>>>>>>> partitions
>>>>>>>>>>> are
>>>>>>>>>>>>>>>> included,
>>>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> first 2000 will be served" Do we
>>>> return an
>>>>>>>> error
>>>>>>>>> for
>>>>>>>>>>> the
>>>>>>>>>>>>>>>> remaining
>>>>>>>>>>>>>>>>>>>>>> partitions? Actually, should we
>>>> include an
>>>>>>>>> errorCode
>>>>>>>>>>>> field
>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>>>>>>>> level in GetReplicaLogInfoResponse to
>>>> cover
>>>>>>>>>>> non-existing
>>>>>>>>>>>>>>>>>> partitions
>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>> no
>>>>>>>>>>>>>>>>>>>>>> authorization, etc?
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 52. The entry should matches => The
>>>> entry
>>>>>>>> should
>>>>>>>>>> match
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 53. ElectLeadersRequest.DesiredLeaders:
>>>>>> Should
>>>>>>>> it
>>>>>>>>> be
>>>>>>>>>>>>>> nullable
>>>>>>>>>>>>>>>>>> since a
>>>>>>>>>>>>>>>>>>>>> user
>>>>>>>>>>>>>>>>>>>>>> may not specify DesiredLeaders?
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 54. Downgrade: Is that indeed
>>>> possible? I
>>>>>>>> thought
>>>>>>>>>>>> earlier
>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>> said
>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>> once the new version of the records
>>>> are in
>>>>>> the
>>>>>>>>>>> metadata
>>>>>>>>>>>>> log,
>>>>>>>>>>>>>>> one
>>>>>>>>>>>>>>>>>>> can't
>>>>>>>>>>>>>>>>>>>>>> downgrade since the old broker doesn't
>>>> know
>>>>>>>> how to
>>>>>>>>>>> parse
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>> version
>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>> the metadata records?
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> MetadataVersion downgrade is currently
>>>> broken
>>>>>>>> but we
>>>>>>>>>>> have
>>>>>>>>>>>>>> fixing
>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>> our
>>>>>>>>>>>>>>>>>>>>> plate for Kafka 3.7.
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> The way downgrade works is that "new
>>>>> features"
>>>>>>>> are
>>>>>>>>>>>> dropped,
>>>>>>>>>>>>>>>> leaving
>>>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>>> the old ones.
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 55. CleanShutdownFile: Should we add a
>>>>>> version
>>>>>>>>> field
>>>>>>>>>>> for
>>>>>>>>>>>>>>> future
>>>>>>>>>>>>>>>>>>>>> extension?
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 56. Config changes are public facing.
>>>> Could
>>>>>> we
>>>>>>>>> have
>>>>>>>>>> a
>>>>>>>>>>>>>> separate
>>>>>>>>>>>>>>>>>>> section
>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>> document all the config changes?
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> +1. A separate section for this would be
>>>>> good.
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> best,
>>>>>>>>>>>>>>>>>>>>> Colin
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> Jun
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> On Mon, Sep 25, 2023 at 4:29 PM Calvin
>>>> Liu
>>>>>>>>>>>>>>>>>>> <caliu@confluent.io.invalid
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> Hi Jun
>>>>>>>>>>>>>>>>>>>>>>> Thanks for the comments.
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 40. If we change to None, it is not
>>>>>> guaranteed
>>>>>>>>> for
>>>>>>>>>> no
>>>>>>>>>>>>> data
>>>>>>>>>>>>>>>> loss.
>>>>>>>>>>>>>>>>>> For
>>>>>>>>>>>>>>>>>>>>> users
>>>>>>>>>>>>>>>>>>>>>>> who are not able to validate the data
>>>> with
>>>>>>>>> external
>>>>>>>>>>>>>>> resources,
>>>>>>>>>>>>>>>>>>> manual
>>>>>>>>>>>>>>>>>>>>>>> intervention does not give a better
>>>> result
>>>>>>>> but a
>>>>>>>>>> loss
>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>> availability.
>>>>>>>>>>>>>>>>>>>>> So
>>>>>>>>>>>>>>>>>>>>>>> practically speaking, the Balance mode
>>>>> would
>>>>>>>> be a
>>>>>>>>>>>> better
>>>>>>>>>>>>>>>> default
>>>>>>>>>>>>>>>>>>>> value.
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 41. No, it represents how we want to
>>>> do
>>>>> the
>>>>>>>>> unclean
>>>>>>>>>>>>> leader
>>>>>>>>>>>>>>>>>> election.
>>>>>>>>>>>>>>>>>>>> If
>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>> is false, the unclean leader election
>>>> will
>>>>>> be
>>>>>>>> the
>>>>>>>>>> old
>>>>>>>>>>>>>> random
>>>>>>>>>>>>>>>> way.
>>>>>>>>>>>>>>>>>>>>>>> Otherwise, the unclean recovery will
>>>> be
>>>>>> used.
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 42. Good catch. Updated.
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 43. Only the first 20 topics will be
>>>>> served.
>>>>>>>>> Others
>>>>>>>>>>>> will
>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>> returned
>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>> InvalidRequestError
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 44. The order matters. The desired
>>>> leader
>>>>>>>> entries
>>>>>>>>>>> match
>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> topic
>>>>>>>>>>>>>>>>>>>>>>> partition list by the index.
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 45. Thanks! Updated.
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 46. Good advice! Updated.
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 47.1, updated the comment. Basically
>>>> it
>>>>> will
>>>>>>>>> elect
>>>>>>>>>>> the
>>>>>>>>>>>>>>> replica
>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> desiredLeader field to be the leader
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 47.2 We can let the admin client do
>>>> the
>>>>>>>>> conversion.
>>>>>>>>>>>> Using
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> desiredLeader
>>>>>>>>>>>>>>>>>>>>>>> field in the json format seems easier
>>>> for
>>>>>>>> users.
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 48. Once the MV version is
>>>> downgraded, all
>>>>>> the
>>>>>>>>> ELR
>>>>>>>>>>>>> related
>>>>>>>>>>>>>>>> fields
>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>> removed on the next partition change.
>>>> The
>>>>>>>>>> controller
>>>>>>>>>>>> will
>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>> ignore
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> ELR fields. Updated the KIP.
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 49. Yes, it would be
>>>> deprecated/removed.
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> On Mon, Sep 25, 2023 at 3:49 PM Jun
>>>> Rao
>>>>>>>>>>>>>>>> <jun@confluent.io.invalid
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> Hi, Calvin,
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the updated KIP. Made
>>>> another
>>>>>>>> pass.
>>>>>>>>> A
>>>>>>>>>>> few
>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>> comments
>>>>>>>>>>>>>>>>>>>>> below.
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 40.
>>>> unclean.leader.election.enable.false
>>>>>> ->
>>>>>>>>>>>>>>>>>>>>>>>> unclean.recovery.strategy.Balanced:
>>>> The
>>>>>>>>> Balanced
>>>>>>>>>>> mode
>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>> still
>>>>>>>>>>>>>>>>>>>>> lead to
>>>>>>>>>>>>>>>>>>>>>>>> data loss. So, I am wondering if
>>>>>>>>>>>>>>>>>>>> unclean.leader.election.enable.false
>>>>>>>>>>>>>>>>>>>>>>>> should map to None?
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 41.
>>>> unclean.recovery.manager.enabled: I
>>>>> am
>>>>>>>> not
>>>>>>>>>> sure
>>>>>>>>>>>> why
>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>> introduce
>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>> additional config. Is it the same as
>>>>>>>>>>>>>>>>>>> unclean.recovery.strategy=None?
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 42.
>>>>>>>>>>> DescribeTopicResponse.TopicAuthorizedOperations:
>>>>>>>>>>>>>> Should
>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> topic level?
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 43. "Limit: 20 topics max per
>>>> request":
>>>>>>>> Could
>>>>>>>>> we
>>>>>>>>>>>>> describe
>>>>>>>>>>>>>>>> what
>>>>>>>>>>>>>>>>>>>>> happens if
>>>>>>>>>>>>>>>>>>>>>>>> the request includes more than 20
>>>>> topics?
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 44.
>>>> ElectLeadersRequest.DesiredLeaders:
>>>>>>>> Could
>>>>>>>>> we
>>>>>>>>>>>>> describe
>>>>>>>>>>>>>>>>>> whether
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> ordering matters?
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 45.
>>>> GetReplicaLogInfo.TopicPartitions:
>>>>>>>> "about":
>>>>>>>>>>> "The
>>>>>>>>>>>>>> topic
>>>>>>>>>>>>>>>>>>>> partitions
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>> elect leaders.": The description in
>>>>>> "about"
>>>>>>>> is
>>>>>>>>>>>>> incorrect.
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 46. GetReplicaLogInfoResponse:
>>>> Should we
>>>>>>>> nest
>>>>>>>>>>>>> partitions
>>>>>>>>>>>>>>>> under
>>>>>>>>>>>>>>>>>>>>> topicId to
>>>>>>>>>>>>>>>>>>>>>>>> be consistent with other types of
>>>>>> responses?
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 47. kafka-leader-election.sh:
>>>>>>>>>>>>>>>>>>>>>>>> 47.1 Could we explain DESIGNATION?
>>>>>>>>>>>>>>>>>>>>>>>> 47.2 desiredLeader: Should it be a
>>>> list
>>>>> to
>>>>>>>>> match
>>>>>>>>>>> the
>>>>>>>>>>>>>> field
>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>> ElectLeadersRequest?
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 48. We could add a section on
>>>> downgrade?
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 49. LastKnownLeader: This seems only
>>>>>> needed
>>>>>>>> in
>>>>>>>>>> the
>>>>>>>>>>>>> first
>>>>>>>>>>>>>>>> phase
>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>> delivering ELR. Will it be removed
>>>> when
>>>>>> the
>>>>>>>>>>> complete
>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>> delivered?
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> Jun
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Sep 19, 2023 at 1:30 PM
>>>> Colin
>>>>>>>> McCabe <
>>>>>>>>>>>>>>>>>> cmccabe@apache.org>
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> Hi Calvin,
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the explanations. I
>>>> like
>>>>> the
>>>>>>>> idea
>>>>>>>>> of
>>>>>>>>>>>> using
>>>>>>>>>>>>>>> none,
>>>>>>>>>>>>>>>>>>>>> balanced,
>>>>>>>>>>>>>>>>>>>>>>>>> aggressive. We also had an offline
>>>>>>>> discussion
>>>>>>>>>>> about
>>>>>>>>>>>>> why
>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>> good
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>> use a
>>>>>>>>>>>>>>>>>>>>>>>>> new config key (basically, so
>>>> that we
>>>>>> can
>>>>>>>>>>> deprecate
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> old
>>>>>>>>>>>>>>>>>> one
>>>>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>>> had
>>>>>>>>>>>>>>>>>>>>>>>>> only false/true values in 4.0)
>>>> With
>>>>>> these
>>>>>>>>>>> changes,
>>>>>>>>>>>> I
>>>>>>>>>>>>> am
>>>>>>>>>>>>>>> +1.
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> best,
>>>>>>>>>>>>>>>>>>>>>>>>> Colin
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Sep 18, 2023, at 15:54,
>>>> Calvin
>>>>>> Liu
>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Colin,
>>>>>>>>>>>>>>>>>>>>>>>>>> Also, can we deprecate
>>>>>>>>>>>>> unclean.leader.election.enable
>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>> 4.0?
>>>>>>>>>>>>>>>>>>>>> Before
>>>>>>>>>>>>>>>>>>>>>>>>> that,
>>>>>>>>>>>>>>>>>>>>>>>>>> we can have both the config
>>>>>>>>>>>>> unclean.recovery.strategy
>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>> unclean.leader.election.enable
>>>>>>>>>>>>>>>>>>>>>>>>>> and using the
>>>>> unclean.recovery.Enabled
>>>>>>>> to
>>>>>>>>>>>> determine
>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>> config
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>> use
>>>>>>>>>>>>>>>>>>>>>>>>>> during the unclean leader
>>>> election.
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Sep 18, 2023 at 3:51 PM
>>>>> Calvin
>>>>>>>> Liu
>>>>>>>>> <
>>>>>>>>>>>>>>>>>>> caliu@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Colin,
>>>>>>>>>>>>>>>>>>>>>>>>>>> For the
>>>> unclean.recovery.strategy
>>>>>>>> config
>>>>>>>>>> name,
>>>>>>>>>>>> how
>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>> use
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> following
>>>>>>>>>>>>>>>>>>>>>>>>>>> None. It basically means no
>>>> unclean
>>>>>>>>> recovery
>>>>>>>>>>>> will
>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>> performed.
>>>>>>>>>>>>>>>>>>>>>>>>>>> Aggressive. It means
>>>> availability
>>>>>> goes
>>>>>>>>>> first.
>>>>>>>>>>>>>> Whenever
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>>>>>>>>>>> can't
>>>>>>>>>>>>>>>>>>>>>>>>>>> elect a durable replica, the
>>>>>> controller
>>>>>>>>> will
>>>>>>>>>>> try
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> unclean
>>>>>>>>>>>>>>>>>>>>>>> recovery.
>>>>>>>>>>>>>>>>>>>>>>>>>>> Balanced. It is the balance
>>>> point
>>>>> of
>>>>>>>> the
>>>>>>>>>>>>>> availability
>>>>>>>>>>>>>>>>>>>>>>>> first(Aggressive)
>>>>>>>>>>>>>>>>>>>>>>>>>>> and least availability(None).
>>>> The
>>>>>>>>> controller
>>>>>>>>>>>>>> performs
>>>>>>>>>>>>>>>>>> unclean
>>>>>>>>>>>>>>>>>>>>>>> recovery
>>>>>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>>>>>> both ISR and ELR are empty.
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Sep 15, 2023 at
>>>> 11:42 AM
>>>>>> Calvin
>>>>>>>>> Liu
>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>> caliu@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Colin,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> So, the proposal is that if
>>>>>> someone
>>>>>>>>> sets
>>>>>>>>>>>>>>>>>>>>>>>>> "unclean.leader.election.enable
>>>>>>>>>>>>>>>>>>>>>>>>>>>> = true"...
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> The idea is to use one of the
>>>>>>>>>>>>>>>>>> unclean.leader.election.enable
>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>> unclean.recovery.strategy
>>>> based on
>>>>>> the
>>>>>>>>>>>>>>>>>>>>> unclean.recovery.Enabled. A
>>>>>>>>>>>>>>>>>>>>>>>>> possible
>>>>>>>>>>>>>>>>>>>>>>>>>>>> version can be
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> If unclean.recovery.Enabled:
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> {
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Check
>>>> unclean.recovery.strategy.
>>>>> If
>>>>>>>> set,
>>>>>>>>>> use
>>>>>>>>>>>> it.
>>>>>>>>>>>>>>>>>> Otherwise,
>>>>>>>>>>>>>>>>>>>>> check
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>> unclean.leader.election.enable and
>>>>>>>>>> translate
>>>>>>>>>>> it
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>> unclean.recovery.strategy.
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> } else {
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Use
>>>> unclean.leader.election.enable
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> —--------
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The configuration key should
>>>> be
>>>>>>>>>>>>>>>>>>>>>>> "unclean.recovery.manager.enabled",
>>>>>>>>>>>>>>>>>>>>>>>>>>>> right?
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think we have two ways of
>>>>>> choosing a
>>>>>>>>>> leader
>>>>>>>>>>>>>>>> uncleanly,
>>>>>>>>>>>>>>>>>>>> unclean
>>>>>>>>>>>>>>>>>>>>>>>> leader
>>>>>>>>>>>>>>>>>>>>>>>>>>>> election and unclean
>>>> recovery(log
>>>>>>>>>> inspection)
>>>>>>>>>>>> and
>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>> try
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> switch
>>>>>>>>>>>>>>>>>>>>>>>>> between
>>>>>>>>>>>>>>>>>>>>>>>>>>>> them.
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Do you mean we want to
>>>> develop two
>>>>>>>> ways
>>>>>>>>> of
>>>>>>>>>>>>>> performing
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> unclean
>>>>>>>>>>>>>>>>>>>>>>>>>>>> recovery and one of them is
>>>> using
>>>>>>>>> “unclean
>>>>>>>>>>>>> recovery
>>>>>>>>>>>>>>>>>>> manager”?
>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>> guess
>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>> haven’t discussed the second
>>>> way.
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> —-------
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> How do these 4 levels of
>>>>> overrides
>>>>>>>>>> interact
>>>>>>>>>>>> with
>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>> configurations?
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> I do notice in the Kraft
>>>>> controller
>>>>>>>> code,
>>>>>>>>>> the
>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> check
>>>>>>>>>>>>>>>>>>>>>>> whether
>>>>>>>>>>>>>>>>>>>>>>>>>>>> perform unclean leader
>>>> election is
>>>>>>>> hard
>>>>>>>>>> coded
>>>>>>>>>>>> to
>>>>>>>>>>>>>>> false
>>>>>>>>>>>>>>>>>> since
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>> 2021(uncleanLeaderElectionEnabledForTopic).
>>>>>>>>>>>> Isn’t
>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>> good
>>>>>>>>>>>>>>>>>>>>> chance
>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>> completely deprecate the
>>>>>>>>>>>>>>>> unclean.leader.election.enable?
>>>>>>>>>>>>>>>>>> We
>>>>>>>>>>>>>>>>>>>>> don’t
>>>>>>>>>>>>>>>>>>>>>>>> even
>>>>>>>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>>> to worry about the config
>>>>>> conversion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> On the other hand, whatever
>>>> the
>>>>>>>> override
>>>>>>>>>> is,
>>>>>>>>>>> as
>>>>>>>>>>>>>> long
>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> controller
>>>>>>>>>>>>>>>>>>>>>>>>>>>> can have the final effective
>>>>>>>>>>>>>>>>>> unclean.leader.election.enable,
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> topic
>>>>>>>>>>>>>>>>>>>>>>>>>>>> level config
>>>>>>>> unclean.recovery.strategy,
>>>>>>>>> the
>>>>>>>>>>>>> cluster
>>>>>>>>>>>>>>>> level
>>>>>>>>>>>>>>>>>>>> config
>>>>>>>>>>>>>>>>>>>>>>>>>>>> unclean.recovery.Enabled, the
>>>>>>>> controller
>>>>>>>>>> can
>>>>>>>>>>>>>>> calculate
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> correct
>>>>>>>>>>>>>>>>>>>>>>>>> methods
>>>>>>>>>>>>>>>>>>>>>>>>>>>> to use right?
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Sep 15, 2023 at
>>>> 10:02 AM
>>>>>> Colin
>>>>>>>>>>> McCabe <
>>>>>>>>>>>>>>>>>>>>> cmccabe@apache.org>
>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Sep 14, 2023, at
>>>> 22:23,
>>>>>>>> Calvin
>>>>>>>>> Liu
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Colin
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1. I think using the new
>>>> config
>>>>>>>> name
>>>>>>>>> is
>>>>>>>>>>> more
>>>>>>>>>>>>>>> clear.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>       a. The unclean
>>>> leader
>>>>>>>> election
>>>>>>>>> is
>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>> removed
>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>> unclean
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> recovery is in use.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>       b. Using multiple
>>>> values
>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>> unclean.leader.election.enable
>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> confusing and it will be
>>>> more
>>>>>>>>> confusing
>>>>>>>>>>>> after
>>>>>>>>>>>>>>> people
>>>>>>>>>>>>>>>>>>> forget
>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Calvin,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> So, the proposal is that if
>>>>> someone
>>>>>>>> sets
>>>>>>>>>>>>>>>>>>>>>>>>> "unclean.leader.election.enable
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> = true" but then sets one of
>>>> your
>>>>>> new
>>>>>>>>>>>>>>> configurations,
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> value of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>> unclean.leader.election.enable is
>>>>>>>>> ignored?
>>>>>>>>>>>> That
>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>> less
>>>>>>>>>>>>>>>>>>>>> clear
>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>> me, not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more. Just in general, having
>>>>>>>> multiple
>>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>>>>> keys
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>> control
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> same thing confuses users.
>>>>>> Basically,
>>>>>>>>> they
>>>>>>>>>>> are
>>>>>>>>>>>>>>> sitting
>>>>>>>>>>>>>>>>>> at a
>>>>>>>>>>>>>>>>>>>>> giant
>>>>>>>>>>>>>>>>>>>>>>>>> control
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> panel, and some of the
>>>> levers do
>>>>>>>>> nothing.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2. Sorry I forgot to
>>>> mention in
>>>>>> the
>>>>>>>>>>> response
>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>> did
>>>>>>>>>>>>>>>>>>> add
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unclean.recovery.Enabled
>>>> flag.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The configuration key should
>>>> be
>>>>>>>>>>>>>>>>>>>>>>> "unclean.recovery.manager.enabled",
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> right? Becuase we can do
>>>> "unclean
>>>>>>>>>> recovery"
>>>>>>>>>>>>>> without
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> manager.
>>>>>>>>>>>>>>>>>>>>>>>>> Disabling
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the manager just means we
>>>> use a
>>>>>>>>> different
>>>>>>>>>>>>>> mechanism
>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>> recovery.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>       c. Maybe I
>>>>> underestimated
>>>>>>>> the
>>>>>>>>>>>> challenge
>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>> replacing
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> config. Any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation problems
>>>> ahead?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> There are four levels of
>>>>> overrides
>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>> unclean.leader.election.enable.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1. static configuration for
>>>> node.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>    This goes in the
>>>>> configuration
>>>>>>>> file,
>>>>>>>>>>>>> typically
>>>>>>>>>>>>>>>> named
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> server.properties
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2. dynamic configuration for
>>>> node
>>>>>>>>> default
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>  ConfigResource(type=BROKER,
>>>>>>>> name="")
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3. dynamic configuration for
>>>> node
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>  ConfigResource(type=BROKER,
>>>>>>>>>>> name=<controller
>>>>>>>>>>>>>> id>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 4. dynamic configuration for
>>>>> topic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>  ConfigResource(type=TOPIC,
>>>>>>>>>>>> name=<topic-name>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> How do these 4 levels of
>>>>> overrides
>>>>>>>>>> interact
>>>>>>>>>>>> with
>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> configurations? If the new
>>>>>>>>> configurations
>>>>>>>>>>>>> dominate
>>>>>>>>>>>>>>>> over
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> old
>>>>>>>>>>>>>>>>>>>>>>>> ones,
>>>>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems like this will get a
>>>> lot
>>>>> more
>>>>>>>>>>> confusing
>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> implement
>>>>>>>>>>>>>>>>>>>> (and
>>>>>>>>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> use.)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Again, I'd recommend just
>>>> adding
>>>>>> some
>>>>>>>>> new
>>>>>>>>>>>> values
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>> unclean.leader.election.enable.
>>>>>> It's
>>>>>>>>>> simple
>>>>>>>>>>>> and
>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>> prevent
>>>>>>>>>>>>>>>>>>>>> user
>>>>>>>>>>>>>>>>>>>>>>>>> confusion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (as well as developer
>>>> confusion.)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> best,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Colin
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3. About the admin client,
>>>> I
>>>>>>>>> mentioned 3
>>>>>>>>>>>>> changes
>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> client.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Anything
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> else I missed in the KIP?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>      a. The client will
>>>> switch
>>>>>> to
>>>>>>>>> using
>>>>>>>>>>> the
>>>>>>>>>>>>> new
>>>>>>>>>>>>>>> RPC
>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> MetadataRequest for the
>>>> topics.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>      b. The
>>>> TopicPartitionInfo
>>>>>>>> used
>>>>>>>>> in
>>>>>>>>>>>>>>>>>> TopicDescription
>>>>>>>>>>>>>>>>>>>>> needs
>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>> add
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fields related to the ELR.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>      c. The outputs will
>>>> add
>>>>> the
>>>>>>>> ELR
>>>>>>>>>>>> related
>>>>>>>>>>>>>>>> fields.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Sep 14, 2023 at
>>>> 9:19 PM
>>>>>>>> Colin
>>>>>>>>>>>> McCabe <
>>>>>>>>>>>>>>>>>>>>>>> cmccabe@apache.org>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Calvin,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the changes.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1. Earlier I commented
>>>> that
>>>>>>>> creating
>>>>>>>>>>>>>>>>>>>>>>> "unclean.recovery.strategy "
>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> necessary, and we can just
>>>>> reuse
>>>>>>>> the
>>>>>>>>>>>> existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>> "unclean.leader.election.enable"
>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>> key.
>>>>>>>>>>>>>>>>>>> Let's
>>>>>>>>>>>>>>>>>>>>>>> discuss
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2.I also don't understand
>>>> why
>>>>>> you
>>>>>>>>>> didn't
>>>>>>>>>>>> add
>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>> configuration to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> enable or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> disable the Unclean
>>>> Recovery
>>>>>>>> Manager.
>>>>>>>>>>> This
>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>> like a
>>>>>>>>>>>>>>>>>>>> very
>>>>>>>>>>>>>>>>>>>>>>>> simple
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> way to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handle the staging issue
>>>> which
>>>>>> we
>>>>>>>>>>>> discussed.
>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>> URM
>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> turned off
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> until it is production
>>>> ready.
>>>>>>>> Let's
>>>>>>>>>>> discuss
>>>>>>>>>>>>>> this.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3. You still need to
>>>> describe
>>>>>> the
>>>>>>>>>> changes
>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> AdminClient
>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> needed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to use
>>>> DescribeTopicRequest.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Keep at it. It's looking
>>>>> better.
>>>>>>>> :)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> best,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Colin
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Sep 14, 2023, at
>>>>> 11:03,
>>>>>>>>> Calvin
>>>>>>>>>>> Liu
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Colin
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the comments!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I did the following
>>>> changes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>   1.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>   Simplified the API
>>>> spec
>>>>>>>> section
>>>>>>>>> to
>>>>>>>>>>>> only
>>>>>>>>>>>>>>>> include
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> diff.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>   2.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>   Reordered the HWM
>>>>>> requirement
>>>>>>>>>>> section.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>   3.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>   Removed the URM
>>>>>>>> implementation
>>>>>>>>>>> details
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>> keep
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> necessary
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>   characteristics to
>>>>> perform
>>>>>>>> the
>>>>>>>>>>> unclean
>>>>>>>>>>>>>>>> recovery.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>   1.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>      When to perform
>>>> the
>>>>>>>> unclean
>>>>>>>>>>>> recovery
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>      2.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>      Under different
>>>>> config,
>>>>>>>> how
>>>>>>>>> the
>>>>>>>>>>>>> unclean
>>>>>>>>>>>>>>>>>> recovery
>>>>>>>>>>>>>>>>>>>>> finds
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> leader.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>      3.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>      How the config
>>>>>>>>>>>>>>>> unclean.leader.election.enable
>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>> unclean.recovery.strategy
>>>>>>>> are
>>>>>>>>>>>>> converted
>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>> users
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> enable/disable
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>      unclean recovery.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>      4.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>   More details about
>>>> how we
>>>>>>>> change
>>>>>>>>>>> admin
>>>>>>>>>>>>>>> client.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>   5.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>   API limits on the
>>>>>>>>>>>>> GetReplicaLogInfoRequest
>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTopicRequest.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>   6.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>   Two metrics added
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>   1.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>> Kafka.controller.global_under_min_isr_partition_count
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>      2.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>> kafka.controller.unclean_recovery_finished_count
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Sep 13, 2023 at
>>>>>> 10:46 AM
>>>>>>>>>> Colin
>>>>>>>>>>>>>> McCabe <
>>>>>>>>>>>>>>>>>>>>>>>>> cmccabe@apache.org>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Sep 12, 2023,
>>>> at
>>>>>> 17:21,
>>>>>>>>>> Calvin
>>>>>>>>>>>> Liu
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Colin
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the
>>>> comments!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Calvin,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks again for the
>>>> KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> One meta-comment: it's
>>>>>> usually
>>>>>>>>>> better
>>>>>>>>>>> to
>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>> do a
>>>>>>>>>>>>>>>>>>> diff
>>>>>>>>>>>>>>>>>>>>> on a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> spec
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> file or java file if
>>>> you're
>>>>>>>>>> including
>>>>>>>>>>>>>> changes
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> easier to read than
>>>> looking
>>>>>> for
>>>>>>>>> "new
>>>>>>>>>>>>> fields
>>>>>>>>>>>>>>>> begin"
>>>>>>>>>>>>>>>>>>> etc.
>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> text, and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> gracefully handles the
>>>> case
>>>>>>>> where
>>>>>>>>>>>> existing
>>>>>>>>>>>>>>>> fields
>>>>>>>>>>>>>>>>>>> were
>>>>>>>>>>>>>>>>>>>>>>>> changed.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Rewrite the
>>>> Additional
>>>>> High
>>>>>>>>>>> Watermark
>>>>>>>>>>>>>>>>>> advancement
>>>>>>>>>>>>>>>>>>>>>>>> requirement
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> There was feedback on
>>>>> this
>>>>>>>>> section
>>>>>>>>>>>> that
>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>> readers
>>>>>>>>>>>>>>>>>>>>> may
>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> familiar
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with HWM and
>>>> Ack=0,1,all
>>>>>>>>> requests.
>>>>>>>>>>>> This
>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>> help
>>>>>>>>>>>>>>>>>>> them
>>>>>>>>>>>>>>>>>>>>>>>>> understand
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposal. I will
>>>> rewrite
>>>>>> this
>>>>>>>>> part
>>>>>>>>>>> for
>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>> readability.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To be clear, I wasn't
>>>>>>>> suggesting
>>>>>>>>>>>> dropping
>>>>>>>>>>>>>>> either
>>>>>>>>>>>>>>>>>>>>> section. I
>>>>>>>>>>>>>>>>>>>>>>>>> agree
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they add useful
>>>>> background. I
>>>>>>>> was
>>>>>>>>>> just
>>>>>>>>>>>>>>>> suggesting
>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discuss
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the "acks" setting
>>>> AFTER
>>>>>>>>> discussing
>>>>>>>>>>> the
>>>>>>>>>>>>> new
>>>>>>>>>>>>>>> high
>>>>>>>>>>>>>>>>>>>>> watermark
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> advancement
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> conditions. We also
>>>> should
>>>>>>>> discuss
>>>>>>>>>>>> acks=0.
>>>>>>>>>>>>>>> While
>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>> isn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> conceptually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> much
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> different than acks=1
>>>> here,
>>>>>> its
>>>>>>>>>>> omission
>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>> section
>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> confusing.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Unclean recovery
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The plan is to
>>>> replace
>>>>> the
>>>>>>>>>>>>>>>>>>>>> unclean.leader.election.enable
>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>> unclean.recovery.strategy.
>>>>>> If
>>>>>>>>> the
>>>>>>>>>>>>> Unclean
>>>>>>>>>>>>>>>>>> Recovery
>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>> enabled
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> then it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deals
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the three
>>>> options in
>>>>>> the
>>>>>>>>>>>>>>>>>>>>> unclean.recovery.strategy.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let’s refine the
>>>> Unclean
>>>>>>>>> Recovery.
>>>>>>>>>>> We
>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>> already
>>>>>>>>>>>>>>>>>>>>> taken a
>>>>>>>>>>>>>>>>>>>>>>>>> lot of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestions and I
>>>> hope to
>>>>>>>>> enhance
>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> durability of
>>>>>>>>>>>>>>>>>>>>> Kafka
>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> level
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I am OK with doing the
>>>>>> unclean
>>>>>>>>>> leader
>>>>>>>>>>>>>> recovery
>>>>>>>>>>>>>>>>>>>>> improvements
>>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, I think we
>>>> need to
>>>>>>>> really
>>>>>>>>>>> work
>>>>>>>>>>>> on
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> settings.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Configuration
>>>> overrides are
>>>>>>>> often
>>>>>>>>>>> quite
>>>>>>>>>>>>>> messy.
>>>>>>>>>>>>>>>> For
>>>>>>>>>>>>>>>>>>>>> example,
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cases
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> where we have
>>>>> log.roll.hours
>>>>>>>> and
>>>>>>>>>>>>>>>>>> log.roll.segment.ms
>>>>>>>>>>>>>>>>>>> ,
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> remember which one
>>>> takes
>>>>>>>>> precedence,
>>>>>>>>>>> and
>>>>>>>>>>>>> it
>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>> obvious.
>>>>>>>>>>>>>>>>>>>>>>>> So,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rather
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> than
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> creating a new
>>>>> configuration,
>>>>>>>> why
>>>>>>>>>> not
>>>>>>>>>>>> add
>>>>>>>>>>>>>>>>>> additional
>>>>>>>>>>>>>>>>>>>>> values
>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>> "unclean.leader.election.enable"?
>>>>>>>>> I
>>>>>>>>>>>> think
>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>> simpler
>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> people
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to understand, and
>>>> simpler
>>>>> in
>>>>>>>> the
>>>>>>>>>> code
>>>>>>>>>>>> as
>>>>>>>>>>>>>>> well.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What if we continued
>>>> to use
>>>>>>>>>>>>>>>>>>>>> "unclean.leader.election.enable"
>>>>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> extended
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it so that it took a
>>>>> string?
>>>>>>>> Then
>>>>>>>>>> the
>>>>>>>>>>>>> string
>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> values:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> never
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>    never
>>>> automatically do
>>>>> an
>>>>>>>>>> unclean
>>>>>>>>>>>>> leader
>>>>>>>>>>>>>>>>>> election
>>>>>>>>>>>>>>>>>>>>> under
>>>>>>>>>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> conditions
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> false / default
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>    only do an unclean
>>>>> leader
>>>>>>>>>> election
>>>>>>>>>>>> if
>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>> may
>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>> possible
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> data
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> loss
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> true / always
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>    always do an
>>>> unclean
>>>>>> leader
>>>>>>>>>>> election
>>>>>>>>>>>>> if
>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>> can't
>>>>>>>>>>>>>>>>>>>>>>>> immediately
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> elect a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> leader
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It's a bit awkward that
>>>>> false
>>>>>>>> maps
>>>>>>>>>> to
>>>>>>>>>>>>>> default
>>>>>>>>>>>>>>>>>> rather
>>>>>>>>>>>>>>>>>>>>> than to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> never. But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this awkwardness
>>>> exists if
>>>>> we
>>>>>>>> use
>>>>>>>>>> two
>>>>>>>>>>>>>>> different
>>>>>>>>>>>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>>>>>>>>>>>> keys
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> well.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The reason for the
>>>>>> awkwardness
>>>>>>>> is
>>>>>>>>>> that
>>>>>>>>>>>> we
>>>>>>>>>>>>>>> simply
>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>> want
>>>>>>>>>>>>>>>>>>>>>>>> most
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> people currently
>>>> setting
>>>>>>>>>>>>>>>>>>>>>>> unclean.leader.election.enable=false
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> get the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "never" behavior. We
>>>> have
>>>>> to
>>>>>>>> bite
>>>>>>>>>> that
>>>>>>>>>>>>>> bullet.
>>>>>>>>>>>>>>>>>> Better
>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>> clear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> explicit than hide it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Another thing that's a
>>>> bit
>>>>>>>> awkward
>>>>>>>>>> is
>>>>>>>>>>>>> having
>>>>>>>>>>>>>>> two
>>>>>>>>>>>>>>>>>>>>> different
>>>>>>>>>>>>>>>>>>>>>>>> ways
>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unclean leader election
>>>>>>>> specified
>>>>>>>>> in
>>>>>>>>>>> the
>>>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>> You
>>>>>>>>>>>>>>>>>>>>> descirbe
>>>>>>>>>>>>>>>>>>>>>>> two
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> methods:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simple "choose the last
>>>>>> leader"
>>>>>>>>>>> method,
>>>>>>>>>>>>> and
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> "unclean
>>>>>>>>>>>>>>>>>>>>>>>>> recovery
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> manager"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method. I understand
>>>> why
>>>>> you
>>>>>>>> did
>>>>>>>>> it
>>>>>>>>>>> this
>>>>>>>>>>>>> way
>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>>>> "choose
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> last
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> leader" is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simple, and will help
>>>> us
>>>>>>>> deliver
>>>>>>>>> an
>>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>>>>>>>> quickly,
>>>>>>>>>>>>>>>>>>>>>>>>> while
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> URM
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is preferable in the
>>>> long
>>>>>>>> term. My
>>>>>>>>>>>>>> suggestion
>>>>>>>>>>>>>>>>>> here is
>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>> separate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> decision of HOW to do
>>>>> unclean
>>>>>>>>> leader
>>>>>>>>>>>>>> election
>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> decision
>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> WHEN
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> So in other words, have
>>>>>>>>>>>>>>>>>>>> "unclean.leader.election.enable"
>>>>>>>>>>>>>>>>>>>>>>>> specify
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do unclean leader
>>>> election,
>>>>>> and
>>>>>>>>>> have a
>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>> "unclean.recovery.manager.enable"
>>>>>>>>> to
>>>>>>>>>>>>>> determine
>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>> use
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> URM.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Presumably the URM will
>>>>> take
>>>>>>>> some
>>>>>>>>>> time
>>>>>>>>>>>> to
>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>> fully
>>>>>>>>>>>>>>>>>>>>> stable,
>>>>>>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> default to false for a
>>>>> while,
>>>>>>>> and
>>>>>>>>> we
>>>>>>>>>>> can
>>>>>>>>>>>>>> flip
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> default to
>>>>>>>>>>>>>>>>>>>>>>>>> true
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> feel ready.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The URM is somewhat
>>>>>>>>> under-described
>>>>>>>>>>>> here.
>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>> need a
>>>>>>>>>>>>>>>>>>>>>>>> few
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> configurations here
>>>> for it.
>>>>>> For
>>>>>>>>>>> example,
>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>> need a
>>>>>>>>>>>>>>>>>>>>>>>>> configuration to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> specify
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> how long it should wait
>>>>> for a
>>>>>>>>> broker
>>>>>>>>>>> to
>>>>>>>>>>>>>>> respond
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> its
>>>>>>>>>>>>>>>>>>>>> RPCs
>>>>>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> moving
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on. We also need to
>>>>>> understand
>>>>>>>> how
>>>>>>>>>> the
>>>>>>>>>>>> URM
>>>>>>>>>>>>>>>>>> interacts
>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>> unclean.leader.election.enable=always. I
>>>>>>>>>>>>>>> assume
>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>> "always"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just unconditionally
>>>> use
>>>>> the
>>>>>>>> URM
>>>>>>>>>>> rather
>>>>>>>>>>>>> than
>>>>>>>>>>>>>>>>>> choosing
>>>>>>>>>>>>>>>>>>>>>>>> randomly.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> But this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should be spelled out
>>>> in
>>>>> the
>>>>>>>> KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTopicRequest
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>   1.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>   Yes, the plan is
>>>> to
>>>>>>>> replace
>>>>>>>>> the
>>>>>>>>>>>>>>>>>> MetadataRequest
>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>> DescribeTopicRequest
>>>>> for
>>>>>>>> the
>>>>>>>>>>> admin
>>>>>>>>>>>>>>> clients.
>>>>>>>>>>>>>>>>>> Will
>>>>>>>>>>>>>>>>>>>>> check
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> details.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Sounds good. But as I
>>>> said,
>>>>>> you
>>>>>>>>> need
>>>>>>>>>>> to
>>>>>>>>>>>>>>> specify
>>>>>>>>>>>>>>>>>> how
>>>>>>>>>>>>>>>>>>>>>>>> AdminClient
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interacts
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the new request.
>>>> This
>>>>>> will
>>>>>>>>>>> involve
>>>>>>>>>>>>>> adding
>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>> fields
>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TopicDescription.java.
>>>> And
>>>>>> you
>>>>>>>>> need
>>>>>>>>>> to
>>>>>>>>>>>>>> specify
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> changes
>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> kafka-topics.sh command
>>>>> line
>>>>>>>> tool.
>>>>>>>>>>>>> Otherwise
>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>> cannot
>>>>>>>>>>>>>>>>>>>>> use
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> tool to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the new information.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The new requests,
>>>>>>>>>> DescribeTopicRequest
>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> GetReplicaLogInfoRequest,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to have limits placed
>>>> on
>>>>> them
>>>>>>>> so
>>>>>>>>>> that
>>>>>>>>>>>>> their
>>>>>>>>>>>>>>> size
>>>>>>>>>>>>>>>>>>> can't
>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> infinite. We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't want to
>>>> propagate the
>>>>>>>>> current
>>>>>>>>>>>>> problems
>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>> MetadataRequest,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients can request
>>>> massive
>>>>>>>>>> responses
>>>>>>>>>>>> that
>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>> mess
>>>>>>>>>>>>>>>>>>> up
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> JVM
>>>>>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handled.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Adding limits is
>>>> simple for
>>>>>>>>>>>>>>>>>> GetReplicaLogInfoRequest
>>>>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just say
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that only 2000
>>>> partitions
>>>>> at
>>>>>> a
>>>>>>>>> time
>>>>>>>>>>> can
>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>> requested.
>>>>>>>>>>>>>>>>>>>> For
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTopicRequest
>>>> we can
>>>>>>>>> probably
>>>>>>>>>>>> just
>>>>>>>>>>>>>>> limit
>>>>>>>>>>>>>>>>>> to 20
>>>>>>>>>>>>>>>>>>>>> topics
>>>>>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like that, to avoid the
>>>>>>>> complexity
>>>>>>>>>> of
>>>>>>>>>>>>> doing
>>>>>>>>>>>>>>>>>>> pagination
>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>   2.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>   I can let the
>>>> broker
>>>>>> load
>>>>>>>> the
>>>>>>>>>> ELR
>>>>>>>>>>>>> info
>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>> they
>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>> serve
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>> DescribeTopicRequest
>>>>> as
>>>>>>>> well.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yes, it's fine to add
>>>> to
>>>>>>>>>>> MetadataCache.
>>>>>>>>>>>> In
>>>>>>>>>>>>>>> fact,
>>>>>>>>>>>>>>>>>>> you'll
>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>> loading
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> anyway once it's added
>>>> to
>>>>>>>>>>>> PartitionImage.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>   3.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>   Yeah, it does not
>>>> make
>>>>>>>> sense
>>>>>>>>> to
>>>>>>>>>>>> have
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> topic
>>>>>>>>>>>>>>>>>>> id
>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>> DescribeTopicRequest
>>>>> is
>>>>>>>> only
>>>>>>>>>> used
>>>>>>>>>>>> by
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> admin
>>>>>>>>>>>>>>>>>>>>> client.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> OK. That makes things
>>>>>> simpler.
>>>>>>>> We
>>>>>>>>>> can
>>>>>>>>>>>>> always
>>>>>>>>>>>>>>>>>> create a
>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>> API
>>>>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (hopefully not in this
>>>>> KIP!)
>>>>>> to
>>>>>>>>>> query
>>>>>>>>>>> by
>>>>>>>>>>>>>> topic
>>>>>>>>>>>>>>>> ID.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Metrics
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> As for overall
>>>> cluster
>>>>>> health
>>>>>>>>>>>> metrics, I
>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>> under-min-ISR
>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> still
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> useful one. ELR is
>>>> more
>>>>>> like
>>>>>>>> a
>>>>>>>>>>> safety
>>>>>>>>>>>>>> belt.
>>>>>>>>>>>>>>>> When
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> ELR
>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> used, the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cluster availability
>>>> has
>>>>>>>> already
>>>>>>>>>>> been
>>>>>>>>>>>>>>>> impacted.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe we can have a
>>>>> metric
>>>>>> to
>>>>>>>>>> count
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> partitions
>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>> sum(ISR,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ELR)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> min
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ISR. What do you
>>>> think?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> How about:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> A.  a metric for the
>>>> totoal
>>>>>>>> number
>>>>>>>>>> of
>>>>>>>>>>>>>>>>>> under-min-isr
>>>>>>>>>>>>>>>>>>>>>>>> partitions?
>>>>>>>>>>>>>>>>>>>>>>>>> We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have that in Apache
>>>> Kafka
>>>>> at
>>>>>>>> the
>>>>>>>>>>> moment.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> B. a metric for the
>>>> number
>>>>> of
>>>>>>>>>> unclean
>>>>>>>>>>>>> leader
>>>>>>>>>>>>>>>>>>> elections
>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>> did
>>>>>>>>>>>>>>>>>>>>>>>>> (for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simplicity, it can
>>>> reset
>>>>> to 0
>>>>>>>> on
>>>>>>>>>>>>> controller
>>>>>>>>>>>>>>>>>> restart:
>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>> expect
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> people to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> monitor the change over
>>>>> time
>>>>>>>>> anyway)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> best,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Colin
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yeah, for the ongoing
>>>>>> unclean
>>>>>>>>>>>>> recoveries,
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> controller
>>>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> keep an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accurate count
>>>> through
>>>>>>>> failover
>>>>>>>>>>>> because
>>>>>>>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>>>>>>>>>> registration
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> indicate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> whether a recovery is
>>>>>> needed.
>>>>>>>>>>> However,
>>>>>>>>>>>>> for
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> happened
>>>>>>>>>>>>>>>>>>>>>>>> ones,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unless
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want to persist the
>>>>> number
>>>>>>>>>>> somewhere,
>>>>>>>>>>>> we
>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>>> figure
>>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>> out
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Sep 12, 2023
>>>> at
>>>>>>>> 3:16 PM
>>>>>>>>>>> Colin
>>>>>>>>>>>>>>> McCabe <
>>>>>>>>>>>>>>>>>>>>>>>>> cmccabe@apache.org
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Also, we should have
>>>>>> metrics
>>>>>>>>> that
>>>>>>>>>>>> show
>>>>>>>>>>>>>> what
>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>> going
>>>>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> regard
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> eligible replica
>>>> set.
>>>>> I'm
>>>>>>>> not
>>>>>>>>>> sure
>>>>>>>>>>>>>> exactly
>>>>>>>>>>>>>>>>>> what to
>>>>>>>>>>>>>>>>>>>>>>> suggest,
>>>>>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that could identify
>>>> when
>>>>>>>> things
>>>>>>>>>> are
>>>>>>>>>>>>> going
>>>>>>>>>>>>>>>>>> wrong in
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> clsuter.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For example, maybe a
>>>>>> metric
>>>>>>>> for
>>>>>>>>>>>>>> partitions
>>>>>>>>>>>>>>>>>>>> containing
>>>>>>>>>>>>>>>>>>>>>>>>> replicas
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ineligible to be
>>>> leader?
>>>>>>>> That
>>>>>>>>>> would
>>>>>>>>>>>>> show
>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>> spike
>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>> broker
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> had an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unclean restart.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ideally, we'd also
>>>> have
>>>>> a
>>>>>>>>> metric
>>>>>>>>>>> that
>>>>>>>>>>>>>>>> indicates
>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>>>> unclear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> leader
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> election or a
>>>> recovery
>>>>>>>>> happened.
>>>>>>>>>>>> It's a
>>>>>>>>>>>>>> bit
>>>>>>>>>>>>>>>>>> tricky
>>>>>>>>>>>>>>>>>>>>>>> because
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simple
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thing, of tracking
>>>> it
>>>>> per
>>>>>>>>>>> controller,
>>>>>>>>>>>>> may
>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>> bit
>>>>>>>>>>>>>>>>>>>>>>>> confusing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> during
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failovers.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> best,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Colin
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Sep 12,
>>>> 2023, at
>>>>>>>> 14:25,
>>>>>>>>>>> Colin
>>>>>>>>>>>>>>> McCabe
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Calvin,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the
>>>> KIP. I
>>>>>>>> think
>>>>>>>>>> this
>>>>>>>>>>>> is a
>>>>>>>>>>>>>>> great
>>>>>>>>>>>>>>>>>>>>>>> improvement.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Additional High
>>>>>> Watermark
>>>>>>>>>>> advance
>>>>>>>>>>>>>>>>>> requirement
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Typo: change
>>>> "advance"
>>>>>> to
>>>>>>>>>>>>> "advancement"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> A bit recap of
>>>> some
>>>>> key
>>>>>>>>>>> concepts.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Typo: change
>>>> "bit" to
>>>>>>>> "quick"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ack=1/all produce
>>>>>>>> request.
>>>>>>>>> It
>>>>>>>>>>>>> defines
>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> Kafka
>>>>>>>>>>>>>>>>>>>>>>>>> server
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> respond to the
>>>> produce
>>>>>>>> request
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think this
>>>> section
>>>>>>>> would be
>>>>>>>>>>>> clearer
>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>> talked
>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> high
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> watermark
>>>> advancement
>>>>>>>>>> requirement
>>>>>>>>>>>>>> first,
>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>> THEN
>>>>>>>>>>>>>>>>>>>>>>> talked
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> impact on acks=0,
>>>>>> acks=1,
>>>>>>>> and
>>>>>>>>>>>>>>> acks=all.
>>>>>>>>>>>>>>>>>>>>> acks=all
>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> course
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> main case we care
>>>>> about
>>>>>>>> here,
>>>>>>>>>> so
>>>>>>>>>>> it
>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>> good
>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>> lead
>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rather than
>>>> delving
>>>>> into
>>>>>>>> the
>>>>>>>>>>>>>>> technicalities
>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>> acks=0/1
>>>>>>>>>>>>>>>>>>>>>>>>> first.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Unclean recovery
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> So, here you are
>>>>>>>> introducing
>>>>>>>>> a
>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>> configuration,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>> unclean.recovery.strategy.
>>>>>>>>> The
>>>>>>>>>>>>>> difficult
>>>>>>>>>>>>>>>>>> thing
>>>>>>>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lot of overlap
>>>> with
>>>>>>>>>>>>>>>>>>>> unclean.leader.election.enable.
>>>>>>>>>>>>>>>>>>>>> So
>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have 3
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> different
>>>> settings for
>>>>>>>>>>>>>>>>>>> unclean.recovery.strategy,
>>>>>>>>>>>>>>>>>>>>> plus
>>>>>>>>>>>>>>>>>>>>>>> 2
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> settings for
>>>>>>>>>>>>>>>> unclean.leader.election.enable,
>>>>>>>>>>>>>>>>>>>> giving
>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>> cross
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> product of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 6 different
>>>> options.
>>>>> The
>>>>>>>>>>> following
>>>>>>>>>>>>>>> "unclean
>>>>>>>>>>>>>>>>>>>> recovery
>>>>>>>>>>>>>>>>>>>>>>>>> manager"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> section
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> --
>>>> David Arthur
>>>> 
>>> 
> 
> -- 
> -David


Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by David Arthur <da...@confluent.io.INVALID>.
Thanks for raising this here, Calvin. Since this is the first "streaming
results" type API in KafkaAdminClient (as far as I know), we're treading
new ground here.

As you mentioned, we can either accept a consumer or return some iterable
result. Returning a java.util.Stream is also an option, and a bit more
modern/convenient than java.util.Iterator. Personally, I like the consumer
approach, but I'm interested in hearing other's opinions.

This actually brings up another question: Do we think it's safe to assume
that one topic's description can fit into memory? The RPC supports paging
across partitions within a single topic, so maybe the admin API should as
well?

-David

On Fri, Feb 23, 2024 at 12:22 PM Calvin Liu <ca...@confluent.io> wrote:

> Hey,
> As we agreed to implement the pagination for the new API
> DescribeTopicPartitions, the client side must also add a proper interface
> to handle the pagination.
> The current KafkaAdminClient.describeTopics returns
> the DescribeTopicsResult which is the future for querying all the topics.
> It is awkward to fit the pagination into it because
>
>    1. Each future corresponds to a topic. We also want to have the
>    pagination on huge topics for their partitions.
>    2. To avoid OOM, we should only fetch the new topics when we need them
>    and release the used topics. Especially the main use case of looping the
>    topic list is when the client prints all the topics.
>
> So, to better serve the pagination, @David Arthur
> <da...@confluent.io> suggested to add a new interface in the Admin
> client between the following 2.
>
> describeTopics(TopicCollection topics, DescribeTopicsOptions options, Consumer<TopicDescription>);
>
> Iterator<TopicDescription> describeTopics(TopicCollection topics, DescribeTopicsOptions options);
>
> David and I would prefer the first Consumer version which works better as a stream purposes.
>
>
> On Wed, Oct 11, 2023 at 4:28 PM Calvin Liu <ca...@confluent.io> wrote:
>
>> Hi David,
>> Thanks for the comment.
>> Yes, we can separate the ELR enablement from the metadata version. It is
>> also helpful to avoid blocking the following MV releases if the user is not
>> ready for ELR.
>> One thing to correct is that, the Unclean recovery is controlled
>> by unclean.recovery.manager.enabled, a separate config
>> from unclean.recovery.strategy. It determines whether unclean recovery will
>> be used in an unclean leader election.
>> Thanks
>>
>> On Wed, Oct 11, 2023 at 4:11 PM David Arthur <mu...@gmail.com> wrote:
>>
>>> One thing we should consider is a static config to totally enable/disable
>>> the ELR feature. If I understand the KIP correctly, we can effectively
>>> disable the unclean recovery by setting the recovery strategy config to
>>> "none".
>>>
>>> This would make development and rollout of this feature a bit smoother.
>>> Consider the case that we find bugs in ELR after a cluster has updated to
>>> its MetadataVersion. It's simpler to disable the feature through config
>>> rather than going through a MetadataVersion downgrade (once that's
>>> supported).
>>>
>>> Does that make sense?
>>>
>>> -David
>>>
>>> On Wed, Oct 11, 2023 at 1:40 PM Calvin Liu <ca...@confluent.io.invalid>
>>> wrote:
>>>
>>> > Hi Jun
>>> > -Good catch, yes, we don't need the -1 in the DescribeTopicRequest.
>>> > -No new value is added. The LeaderRecoveryState will still be set to 1
>>> if
>>> > we have an unclean leader election. The unclean leader election
>>> includes
>>> > the old random way and the unclean recovery. During the unclean
>>> recovery,
>>> > the LeaderRecoveryState will not change until the controller decides to
>>> > update the records with the new leader.
>>> > Thanks
>>> >
>>> > On Wed, Oct 11, 2023 at 9:02 AM Jun Rao <ju...@confluent.io.invalid>
>>> wrote:
>>> >
>>> > > Hi, Calvin,
>>> > >
>>> > > Another thing. Currently, when there is an unclean leader election,
>>> we
>>> > set
>>> > > the LeaderRecoveryState in PartitionRecord and PartitionChangeRecord
>>> to
>>> > 1.
>>> > > With the KIP, will there be new values for LeaderRecoveryState? If
>>> not,
>>> > > when will LeaderRecoveryState be set to 1?
>>> > >
>>> > > Thanks,
>>> > >
>>> > > Jun
>>> > >
>>> > > On Tue, Oct 10, 2023 at 4:24 PM Jun Rao <ju...@confluent.io> wrote:
>>> > >
>>> > > > Hi, Calvin,
>>> > > >
>>> > > > One more comment.
>>> > > >
>>> > > > "The first partition to fetch details for. -1 means to fetch all
>>> > > > partitions." It seems that FirstPartitionId of 0 naturally means
>>> > fetching
>>> > > > all partitions?
>>> > > >
>>> > > > Thanks,
>>> > > >
>>> > > > Jun
>>> > > >
>>> > > > On Tue, Oct 10, 2023 at 12:40 PM Calvin Liu
>>> <caliu@confluent.io.invalid
>>> > >
>>> > > > wrote:
>>> > > >
>>> > > >> Hi Jun,
>>> > > >> Yeah, with the current Metadata request handling, we only return
>>> > errors
>>> > > on
>>> > > >> the Topic level, like topic not found. It seems that querying a
>>> > specific
>>> > > >> partition is not a valid use case. Will update.
>>> > > >> Thanks
>>> > > >>
>>> > > >> On Tue, Oct 10, 2023 at 11:55 AM Jun Rao <jun@confluent.io.invalid
>>> >
>>> > > >> wrote:
>>> > > >>
>>> > > >> > Hi, Calvin,
>>> > > >> >
>>> > > >> > 60.  If the range query has errors for some of the partitions,
>>> do we
>>> > > >> expect
>>> > > >> > different responses when querying particular partitions?
>>> > > >> >
>>> > > >> > Thanks,
>>> > > >> >
>>> > > >> > Jun
>>> > > >> >
>>> > > >> > On Tue, Oct 10, 2023 at 10:50 AM Calvin Liu
>>> > > <caliu@confluent.io.invalid
>>> > > >> >
>>> > > >> > wrote:
>>> > > >> >
>>> > > >> > > Hi Jun
>>> > > >> > > 60. Yes, it is a good question. I was thinking the API could
>>> be
>>> > > >> flexible
>>> > > >> > to
>>> > > >> > > query the particular partitions if the range query has errors
>>> for
>>> > > >> some of
>>> > > >> > > the partitions. Not sure whether it is a valid assumption,
>>> what do
>>> > > you
>>> > > >> > > think?
>>> > > >> > >
>>> > > >> > > 61. Good point, I will update them to partition level with the
>>> > same
>>> > > >> > limit.
>>> > > >> > >
>>> > > >> > > 62. Sure, will do.
>>> > > >> > >
>>> > > >> > > Thanks
>>> > > >> > >
>>> > > >> > > On Tue, Oct 10, 2023 at 10:12 AM Jun Rao
>>> <jun@confluent.io.invalid
>>> > >
>>> > > >> > wrote:
>>> > > >> > >
>>> > > >> > > > Hi, Calvin,
>>> > > >> > > >
>>> > > >> > > > A few more minor comments on your latest update.
>>> > > >> > > >
>>> > > >> > > > 60. DescribeTopicRequest: When will the Partitions field be
>>> > used?
>>> > > It
>>> > > >> > > seems
>>> > > >> > > > that the FirstPartitionId field is enough for AdminClient
>>> usage.
>>> > > >> > > >
>>> > > >> > > > 61. Could we make the limit for DescribeTopicRequest,
>>> > > >> > > ElectLeadersRequest,
>>> > > >> > > > GetReplicaLogInfo consistent? Currently,
>>> ElectLeadersRequest's
>>> > > >> limit is
>>> > > >> > > at
>>> > > >> > > > topic level and GetReplicaLogInfo has a different partition
>>> > level
>>> > > >> limit
>>> > > >> > > > from DescribeTopicRequest.
>>> > > >> > > >
>>> > > >> > > > 62. Should ElectLeadersRequest.DesiredLeaders be at the same
>>> > level
>>> > > >> as
>>> > > >> > > > ElectLeadersRequest.TopicPartitions.Partitions? In the KIP,
>>> it
>>> > > looks
>>> > > >> > like
>>> > > >> > > > it's at the same level as
>>> ElectLeadersRequest.TopicPartitions.
>>> > > >> > > >
>>> > > >> > > > Thanks,
>>> > > >> > > >
>>> > > >> > > > Jun
>>> > > >> > > >
>>> > > >> > > > On Wed, Oct 4, 2023 at 3:55 PM Calvin Liu
>>> > > >> <ca...@confluent.io.invalid>
>>> > > >> > > > wrote:
>>> > > >> > > >
>>> > > >> > > > > Hi David,
>>> > > >> > > > > Thanks for the comments.
>>> > > >> > > > > ----
>>> > > >> > > > > I thought that a new snapshot with the downgraded MV is
>>> > created
>>> > > in
>>> > > >> > this
>>> > > >> > > > > case. Isn’t it the case?
>>> > > >> > > > > Yes, you are right, a metadata delta will be generated
>>> after
>>> > the
>>> > > >> MV
>>> > > >> > > > > downgrade. Then the user can start the software downgrade.
>>> > > >> > > > > -----
>>> > > >> > > > > Could you also elaborate a bit more on the reasoning
>>> behind
>>> > > adding
>>> > > >> > the
>>> > > >> > > > > limits to the admin RPCs? This is a new pattern in Kafka
>>> so it
>>> > > >> would
>>> > > >> > be
>>> > > >> > > > > good to clear on the motivation.
>>> > > >> > > > > Thanks to Colin for bringing it up. The current
>>> > MetadataRequest
>>> > > >> does
>>> > > >> > > not
>>> > > >> > > > > have a limit on the number of topics to query in a single
>>> > > request.
>>> > > >> > > > Massive
>>> > > >> > > > > requests can mess up the JVM. We want to have some sort of
>>> > > >> throttle
>>> > > >> > on
>>> > > >> > > > the
>>> > > >> > > > > new APIs.
>>> > > >> > > > > -----
>>> > > >> > > > > Could you also explain how the client is supposed to
>>> handle
>>> > the
>>> > > >> > > > > topics/partitions above the limit? I suppose that it will
>>> have
>>> > > to
>>> > > >> > retry
>>> > > >> > > > > those, correct?
>>> > > >> > > > > Corrent. For the official admin clients, it will split the
>>> > large
>>> > > >> > > request
>>> > > >> > > > > into proper pieces and query one after another.
>>> > > >> > > > > -----
>>> > > >> > > > > My understanding is that the topics/partitions above the
>>> limit
>>> > > >> will
>>> > > >> > be
>>> > > >> > > > > failed with an invalid exception error. I wonder if this
>>> > choice
>>> > > is
>>> > > >> > > > > judicious because the invalide request exception is
>>> usually
>>> > > >> fatal. It
>>> > > >> > > may
>>> > > >> > > > > be better to use an new and explicit error for this case.
>>> > > >> > > > >
>>> > > >> > > > > Thanks for bringing this up. How about
>>> > "REQUEST_LIMIT_REACHED"?
>>> > > >> > > > > --------
>>> > > >> > > > > It seems that we still need to specify the changes to the
>>> > admin
>>> > > >> api
>>> > > >> > to
>>> > > >> > > > > accommodate the new or updated apis. Do you plan to add
>>> them?
>>> > > >> > > > > Try to cover the following
>>> > > >> > > > > 1. The admin client will use the new DescribeTopicRequest
>>> to
>>> > > query
>>> > > >> > the
>>> > > >> > > > > topics
>>> > > >> > > > > 2. Mention the API limit and the new retriable error.
>>> > > >> > > > > 3. Output changes for the admin client when describing a
>>> topic
>>> > > >> (new
>>> > > >> > > > fields
>>> > > >> > > > > of ELR...)
>>> > > >> > > > > 4. Changes to data structures like TopicPartitionInfo to
>>> > include
>>> > > >> the
>>> > > >> > > ELR.
>>> > > >> > > > > Anything else I missed?
>>> > > >> > > > >
>>> > > >> > > > > Thanks!
>>> > > >> > > > >
>>> > > >> > > > >
>>> > > >> > > > >
>>> > > >> > > > >
>>> > > >> > > > >
>>> > > >> > > > > On Wed, Oct 4, 2023 at 12:27 PM David Jacot <
>>> > > >> david.jacot@gmail.com>
>>> > > >> > > > wrote:
>>> > > >> > > > >
>>> > > >> > > > > > Hi Calvin,
>>> > > >> > > > > >
>>> > > >> > > > > > I thought that a new snapshot with the downgraded MV is
>>> > > created
>>> > > >> in
>>> > > >> > > this
>>> > > >> > > > > > case. Isn’t it the case?
>>> > > >> > > > > >
>>> > > >> > > > > > Could you also elaborate a bit more on the reasoning
>>> behind
>>> > > >> adding
>>> > > >> > > the
>>> > > >> > > > > > limits to the admin RPCs? This is a new pattern in
>>> Kafka so
>>> > it
>>> > > >> > would
>>> > > >> > > be
>>> > > >> > > > > > good to clear on the motivation.
>>> > > >> > > > > >
>>> > > >> > > > > > Could you also explain how the client is supposed to
>>> handle
>>> > > the
>>> > > >> > > > > > topics/partitions above the limit? I suppose that it
>>> will
>>> > have
>>> > > >> to
>>> > > >> > > retry
>>> > > >> > > > > > those, correct?
>>> > > >> > > > > >
>>> > > >> > > > > > My understanding is that the topics/partitions above the
>>> > limit
>>> > > >> will
>>> > > >> > > be
>>> > > >> > > > > > failed with an invalid exception error. I wonder if this
>>> > > choice
>>> > > >> is
>>> > > >> > > > > > judicious because the invalide request exception is
>>> usually
>>> > > >> fatal.
>>> > > >> > It
>>> > > >> > > > may
>>> > > >> > > > > > be better to use an new and explicit error for this
>>> case.
>>> > > >> > > > > >
>>> > > >> > > > > > It seems that we still need to specify the changes to
>>> the
>>> > > admin
>>> > > >> api
>>> > > >> > > to
>>> > > >> > > > > > accommodate the new or updated apis. Do you plan to add
>>> > them?
>>> > > >> > > > > >
>>> > > >> > > > > > Best,
>>> > > >> > > > > > David
>>> > > >> > > > > >
>>> > > >> > > > > > Le mer. 4 oct. 2023 à 20:39, Calvin Liu
>>> > > >> <caliu@confluent.io.invalid
>>> > > >> > >
>>> > > >> > > a
>>> > > >> > > > > > écrit :
>>> > > >> > > > > >
>>> > > >> > > > > > > Hi Jun,
>>> > > >> > > > > > > After the MV downgrade, the controller will write in
>>> the
>>> > old
>>> > > >> > > version
>>> > > >> > > > of
>>> > > >> > > > > > the
>>> > > >> > > > > > > PartitionRecord/PartitionChangeRecord. If I understand
>>> > > >> correctly,
>>> > > >> > > it
>>> > > >> > > > is
>>> > > >> > > > > > > possible to downgrade the software version if the
>>> > controller
>>> > > >> only
>>> > > >> > > has
>>> > > >> > > > > to
>>> > > >> > > > > > > handle old version records.
>>> > > >> > > > > > > However, the controller will not automatically
>>> rewrite the
>>> > > >> > > > > > PartitionRecord
>>> > > >> > > > > > > with the old version unless there is a partition
>>> update.
>>> > > Then,
>>> > > >> > the
>>> > > >> > > > user
>>> > > >> > > > > > may
>>> > > >> > > > > > > have to wait an unknown amount of time before the
>>> software
>>> > > >> > > downgrades
>>> > > >> > > > > > > unless they do a roll to force update every
>>> partition. If
>>> > it
>>> > > >> > makes
>>> > > >> > > > > > sense, I
>>> > > >> > > > > > > can mention these steps to do a software downgrade.
>>> > > >> > > > > > > Thanks
>>> > > >> > > > > > >
>>> > > >> > > > > > > On Wed, Oct 4, 2023 at 11:20 AM Jun Rao
>>> > > >> <jun@confluent.io.invalid
>>> > > >> > >
>>> > > >> > > > > > wrote:
>>> > > >> > > > > > >
>>> > > >> > > > > > > > Hi, Calvin and Justine,
>>> > > >> > > > > > > >
>>> > > >> > > > > > > > Historically, when we change the record format in
>>> the
>>> > log,
>>> > > >> we
>>> > > >> > > don't
>>> > > >> > > > > > > support
>>> > > >> > > > > > > > software version downgrading.
>>> > > >> > > > > > > >
>>> > > >> > > > > > > > For the record format change in the metadata log,
>>> have
>>> > we
>>> > > >> > thought
>>> > > >> > > > > about
>>> > > >> > > > > > > > forcing the write of the latest metadata records
>>> with
>>> > the
>>> > > >> old
>>> > > >> > > > version
>>> > > >> > > > > > > > during MV downgrading? This will in theory allow
>>> the old
>>> > > >> > version
>>> > > >> > > of
>>> > > >> > > > > the
>>> > > >> > > > > > > > software to obtain the latest metadata.
>>> > > >> > > > > > > >
>>> > > >> > > > > > > > Thanks,
>>> > > >> > > > > > > >
>>> > > >> > > > > > > > Jun
>>> > > >> > > > > > > >
>>> > > >> > > > > > > > On Wed, Oct 4, 2023 at 9:53 AM Justine Olshan
>>> > > >> > > > > > > <jolshan@confluent.io.invalid
>>> > > >> > > > > > > > >
>>> > > >> > > > > > > > wrote:
>>> > > >> > > > > > > >
>>> > > >> > > > > > > > > Sorry -- not MV but software version.
>>> > > >> > > > > > > > >
>>> > > >> > > > > > > > > On Wed, Oct 4, 2023 at 9:51 AM Justine Olshan <
>>> > > >> > > > > jolshan@confluent.io>
>>> > > >> > > > > > > > > wrote:
>>> > > >> > > > > > > > >
>>> > > >> > > > > > > > > > Catching up with this discussion.
>>> > > >> > > > > > > > > >
>>> > > >> > > > > > > > > > I was just curious -- have we had other
>>> instances
>>> > > where
>>> > > >> > > > > downgrading
>>> > > >> > > > > > > MV
>>> > > >> > > > > > > > is
>>> > > >> > > > > > > > > > not supported? I think Kafka typically tries to
>>> > > support
>>> > > >> > > > > downgrades,
>>> > > >> > > > > > > > and I
>>> > > >> > > > > > > > > > couldn't think of other examples.
>>> > > >> > > > > > > > > >
>>> > > >> > > > > > > > > > Thanks,
>>> > > >> > > > > > > > > > Justine
>>> > > >> > > > > > > > > >
>>> > > >> > > > > > > > > > On Wed, Oct 4, 2023 at 9:40 AM Calvin Liu
>>> > > >> > > > > > <caliu@confluent.io.invalid
>>> > > >> > > > > > > >
>>> > > >> > > > > > > > > > wrote:
>>> > > >> > > > > > > > > >
>>> > > >> > > > > > > > > >> Hi Jun,
>>> > > >> > > > > > > > > >> 54. Marked the software downgrading is not
>>> > supported.
>>> > > >> As
>>> > > >> > the
>>> > > >> > > > old
>>> > > >> > > > > > > > > >> controller
>>> > > >> > > > > > > > > >> will not understand the new PartitionRecord and
>>> > > >> > > > > > > PartitionChangeRecord.
>>> > > >> > > > > > > > > >> Thanks!
>>> > > >> > > > > > > > > >>
>>> > > >> > > > > > > > > >> On Wed, Oct 4, 2023 at 9:12 AM Jun Rao
>>> > > >> > > > <jun@confluent.io.invalid
>>> > > >> > > > > >
>>> > > >> > > > > > > > > wrote:
>>> > > >> > > > > > > > > >>
>>> > > >> > > > > > > > > >> > Hi, Calvin,
>>> > > >> > > > > > > > > >> >
>>> > > >> > > > > > > > > >> > Thanks for the reply. Just one more comment.
>>> > > >> > > > > > > > > >> >
>>> > > >> > > > > > > > > >> > 54. It seems that downgrading MV is
>>> supported. Is
>>> > > >> > > > downgrading
>>> > > >> > > > > > the
>>> > > >> > > > > > > > > >> software
>>> > > >> > > > > > > > > >> > version supported? It would be useful to
>>> document
>>> > > >> that.
>>> > > >> > > > > > > > > >> >
>>> > > >> > > > > > > > > >> > Thanks,
>>> > > >> > > > > > > > > >> >
>>> > > >> > > > > > > > > >> > Jun
>>> > > >> > > > > > > > > >> >
>>> > > >> > > > > > > > > >> > On Tue, Oct 3, 2023 at 4:55 PM Artem Livshits
>>> > > >> > > > > > > > > >> > <al...@confluent.io.invalid> wrote:
>>> > > >> > > > > > > > > >> >
>>> > > >> > > > > > > > > >> > > Hi Colin,
>>> > > >> > > > > > > > > >> > >
>>> > > >> > > > > > > > > >> > > I think in your example
>>> "do_unclean_recovery"
>>> > > would
>>> > > >> > need
>>> > > >> > > > to
>>> > > >> > > > > do
>>> > > >> > > > > > > > > >> different
>>> > > >> > > > > > > > > >> > > things depending on the strategy.
>>> > > >> > > > > > > > > >> > >
>>> > > >> > > > > > > > > >> > > do_unclean_recovery() {
>>> > > >> > > > > > > > > >> > >    if (unclean.recovery.manager.enabled) {
>>> > > >> > > > > > > > > >> > >     if (strategy == Aggressive)
>>> > > >> > > > > > > > > >> > >       use
>>> > > >> > UncleanRecoveryManager(waitLastKnownERL=false)
>>> > > >> > > > //
>>> > > >> > > > > > > just
>>> > > >> > > > > > > > > >> inspect
>>> > > >> > > > > > > > > >> > > logs from whoever is available
>>> > > >> > > > > > > > > >> > >     else
>>> > > >> > > > > > > > > >> > >       use
>>> > > >> > UncleanRecoveryManager(waitLastKnownERL=true)
>>> > > >> > > > //
>>> > > >> > > > > > > must
>>> > > >> > > > > > > > > wait
>>> > > >> > > > > > > > > >> > for
>>> > > >> > > > > > > > > >> > > at least last known ELR
>>> > > >> > > > > > > > > >> > >   } else {
>>> > > >> > > > > > > > > >> > >     if (strategy == Aggressive)
>>> > > >> > > > > > > > > >> > >       choose the last known leader if that
>>> is
>>> > > >> > available,
>>> > > >> > > > or
>>> > > >> > > > > a
>>> > > >> > > > > > > > random
>>> > > >> > > > > > > > > >> > leader
>>> > > >> > > > > > > > > >> > > if not)
>>> > > >> > > > > > > > > >> > >     else
>>> > > >> > > > > > > > > >> > >       wait for last known leader to get
>>> back
>>> > > >> > > > > > > > > >> > >   }
>>> > > >> > > > > > > > > >> > > }
>>> > > >> > > > > > > > > >> > >
>>> > > >> > > > > > > > > >> > > The idea is that the Aggressive strategy
>>> would
>>> > > >> kick in
>>> > > >> > > as
>>> > > >> > > > > soon
>>> > > >> > > > > > > as
>>> > > >> > > > > > > > we
>>> > > >> > > > > > > > > >> lost
>>> > > >> > > > > > > > > >> > > the leader and would pick a leader from
>>> whoever
>>> > > is
>>> > > >> > > > > available;
>>> > > >> > > > > > > but
>>> > > >> > > > > > > > > the
>>> > > >> > > > > > > > > >> > > Balanced will only kick in when ELR is
>>> empty
>>> > and
>>> > > >> will
>>> > > >> > > wait
>>> > > >> > > > > for
>>> > > >> > > > > > > the
>>> > > >> > > > > > > > > >> > brokers
>>> > > >> > > > > > > > > >> > > that likely have most data to be available.
>>> > > >> > > > > > > > > >> > >
>>> > > >> > > > > > > > > >> > > On Tue, Oct 3, 2023 at 3:04 PM Colin
>>> McCabe <
>>> > > >> > > > > > cmccabe@apache.org
>>> > > >> > > > > > > >
>>> > > >> > > > > > > > > >> wrote:
>>> > > >> > > > > > > > > >> > >
>>> > > >> > > > > > > > > >> > > > On Tue, Oct 3, 2023, at 10:49, Jun Rao
>>> wrote:
>>> > > >> > > > > > > > > >> > > > > Hi, Calvin,
>>> > > >> > > > > > > > > >> > > > >
>>> > > >> > > > > > > > > >> > > > > Thanks for the update KIP. A few more
>>> > > comments.
>>> > > >> > > > > > > > > >> > > > >
>>> > > >> > > > > > > > > >> > > > > 41. Why would a user choose the option
>>> to
>>> > > >> select a
>>> > > >> > > > > random
>>> > > >> > > > > > > > > replica
>>> > > >> > > > > > > > > >> as
>>> > > >> > > > > > > > > >> > > the
>>> > > >> > > > > > > > > >> > > > > leader instead of using
>>> > > >> > > > > > unclean.recovery.strateg=Aggressive?
>>> > > >> > > > > > > > It
>>> > > >> > > > > > > > > >> seems
>>> > > >> > > > > > > > > >> > > > that
>>> > > >> > > > > > > > > >> > > > > the latter is strictly better? If
>>> that's
>>> > not
>>> > > >> the
>>> > > >> > > case,
>>> > > >> > > > > > could
>>> > > >> > > > > > > > we
>>> > > >> > > > > > > > > >> fold
>>> > > >> > > > > > > > > >> > > this
>>> > > >> > > > > > > > > >> > > > > option under unclean.recovery.strategy
>>> > > instead
>>> > > >> of
>>> > > >> > > > > > > introducing
>>> > > >> > > > > > > > a
>>> > > >> > > > > > > > > >> > > separate
>>> > > >> > > > > > > > > >> > > > > config?
>>> > > >> > > > > > > > > >> > > >
>>> > > >> > > > > > > > > >> > > > Hi Jun,
>>> > > >> > > > > > > > > >> > > >
>>> > > >> > > > > > > > > >> > > > I thought the flow of control was:
>>> > > >> > > > > > > > > >> > > >
>>> > > >> > > > > > > > > >> > > > If there is no leader for the partition {
>>> > > >> > > > > > > > > >> > > >   If (there are unfenced ELR members) {
>>> > > >> > > > > > > > > >> > > >     choose_an_unfenced_ELR_member
>>> > > >> > > > > > > > > >> > > >   } else if (there are fenced ELR
>>> members AND
>>> > > >> > > > > > > > > strategy=Aggressive) {
>>> > > >> > > > > > > > > >> > > >     do_unclean_recovery
>>> > > >> > > > > > > > > >> > > >   } else if (there are no ELR members AND
>>> > > >> strategy
>>> > > >> > !=
>>> > > >> > > > > None)
>>> > > >> > > > > > {
>>> > > >> > > > > > > > > >> > > >     do_unclean_recovery
>>> > > >> > > > > > > > > >> > > >   } else {
>>> > > >> > > > > > > > > >> > > >     do nothing about the missing leader
>>> > > >> > > > > > > > > >> > > >   }
>>> > > >> > > > > > > > > >> > > > }
>>> > > >> > > > > > > > > >> > > >
>>> > > >> > > > > > > > > >> > > > do_unclean_recovery() {
>>> > > >> > > > > > > > > >> > > >    if (unclean.recovery.manager.enabled)
>>> {
>>> > > >> > > > > > > > > >> > > >     use UncleanRecoveryManager
>>> > > >> > > > > > > > > >> > > >   } else {
>>> > > >> > > > > > > > > >> > > >     choose the last known leader if that
>>> is
>>> > > >> > available,
>>> > > >> > > > or
>>> > > >> > > > > a
>>> > > >> > > > > > > > random
>>> > > >> > > > > > > > > >> > leader
>>> > > >> > > > > > > > > >> > > > if not)
>>> > > >> > > > > > > > > >> > > >   }
>>> > > >> > > > > > > > > >> > > > }
>>> > > >> > > > > > > > > >> > > >
>>> > > >> > > > > > > > > >> > > > However, I think this could be clarified,
>>> > > >> especially
>>> > > >> > > the
>>> > > >> > > > > > > > behavior
>>> > > >> > > > > > > > > >> when
>>> > > >> > > > > > > > > >> > > > unclean.recovery.manager.enabled=false.
>>> > > >> Inuitively
>>> > > >> > the
>>> > > >> > > > > goal
>>> > > >> > > > > > > for
>>> > > >> > > > > > > > > >> > > > unclean.recovery.manager.enabled=false
>>> is to
>>> > be
>>> > > >> "the
>>> > > >> > > > same
>>> > > >> > > > > as
>>> > > >> > > > > > > > now,
>>> > > >> > > > > > > > > >> > mostly"
>>> > > >> > > > > > > > > >> > > > but it's very underspecified in the KIP,
>>> I
>>> > > agree.
>>> > > >> > > > > > > > > >> > > >
>>> > > >> > > > > > > > > >> > > > >
>>> > > >> > > > > > > > > >> > > > > 50. ElectLeadersRequest: "If more than
>>> 20
>>> > > >> topics
>>> > > >> > are
>>> > > >> > > > > > > included,
>>> > > >> > > > > > > > > >> only
>>> > > >> > > > > > > > > >> > the
>>> > > >> > > > > > > > > >> > > > > first 20 will be served. Others will be
>>> > > >> returned
>>> > > >> > > with
>>> > > >> > > > > > > > > >> > DesiredLeaders."
>>> > > >> > > > > > > > > >> > > > Hmm,
>>> > > >> > > > > > > > > >> > > > > not sure that I understand this.
>>> > > >> > > ElectLeadersResponse
>>> > > >> > > > > > > doesn't
>>> > > >> > > > > > > > > >> have a
>>> > > >> > > > > > > > > >> > > > > DesiredLeaders field.
>>> > > >> > > > > > > > > >> > > > >
>>> > > >> > > > > > > > > >> > > > > 51. GetReplicaLogInfo: "If more than
>>> 2000
>>> > > >> > partitions
>>> > > >> > > > are
>>> > > >> > > > > > > > > included,
>>> > > >> > > > > > > > > >> > only
>>> > > >> > > > > > > > > >> > > > the
>>> > > >> > > > > > > > > >> > > > > first 2000 will be served" Do we
>>> return an
>>> > > >> error
>>> > > >> > for
>>> > > >> > > > the
>>> > > >> > > > > > > > > remaining
>>> > > >> > > > > > > > > >> > > > > partitions? Actually, should we
>>> include an
>>> > > >> > errorCode
>>> > > >> > > > > field
>>> > > >> > > > > > > at
>>> > > >> > > > > > > > > the
>>> > > >> > > > > > > > > >> > > > partition
>>> > > >> > > > > > > > > >> > > > > level in GetReplicaLogInfoResponse to
>>> cover
>>> > > >> > > > non-existing
>>> > > >> > > > > > > > > >> partitions
>>> > > >> > > > > > > > > >> > and
>>> > > >> > > > > > > > > >> > > > no
>>> > > >> > > > > > > > > >> > > > > authorization, etc?
>>> > > >> > > > > > > > > >> > > > >
>>> > > >> > > > > > > > > >> > > > > 52. The entry should matches => The
>>> entry
>>> > > >> should
>>> > > >> > > match
>>> > > >> > > > > > > > > >> > > > >
>>> > > >> > > > > > > > > >> > > > > 53. ElectLeadersRequest.DesiredLeaders:
>>> > > Should
>>> > > >> it
>>> > > >> > be
>>> > > >> > > > > > > nullable
>>> > > >> > > > > > > > > >> since a
>>> > > >> > > > > > > > > >> > > > user
>>> > > >> > > > > > > > > >> > > > > may not specify DesiredLeaders?
>>> > > >> > > > > > > > > >> > > > >
>>> > > >> > > > > > > > > >> > > > > 54. Downgrade: Is that indeed
>>> possible? I
>>> > > >> thought
>>> > > >> > > > > earlier
>>> > > >> > > > > > > you
>>> > > >> > > > > > > > > said
>>> > > >> > > > > > > > > >> > that
>>> > > >> > > > > > > > > >> > > > > once the new version of the records
>>> are in
>>> > > the
>>> > > >> > > > metadata
>>> > > >> > > > > > log,
>>> > > >> > > > > > > > one
>>> > > >> > > > > > > > > >> > can't
>>> > > >> > > > > > > > > >> > > > > downgrade since the old broker doesn't
>>> know
>>> > > >> how to
>>> > > >> > > > parse
>>> > > >> > > > > > the
>>> > > >> > > > > > > > new
>>> > > >> > > > > > > > > >> > > version
>>> > > >> > > > > > > > > >> > > > of
>>> > > >> > > > > > > > > >> > > > > the metadata records?
>>> > > >> > > > > > > > > >> > > > >
>>> > > >> > > > > > > > > >> > > >
>>> > > >> > > > > > > > > >> > > > MetadataVersion downgrade is currently
>>> broken
>>> > > >> but we
>>> > > >> > > > have
>>> > > >> > > > > > > fixing
>>> > > >> > > > > > > > > it
>>> > > >> > > > > > > > > >> on
>>> > > >> > > > > > > > > >> > > our
>>> > > >> > > > > > > > > >> > > > plate for Kafka 3.7.
>>> > > >> > > > > > > > > >> > > >
>>> > > >> > > > > > > > > >> > > > The way downgrade works is that "new
>>> > features"
>>> > > >> are
>>> > > >> > > > > dropped,
>>> > > >> > > > > > > > > leaving
>>> > > >> > > > > > > > > >> > only
>>> > > >> > > > > > > > > >> > > > the old ones.
>>> > > >> > > > > > > > > >> > > >
>>> > > >> > > > > > > > > >> > > > > 55. CleanShutdownFile: Should we add a
>>> > > version
>>> > > >> > field
>>> > > >> > > > for
>>> > > >> > > > > > > > future
>>> > > >> > > > > > > > > >> > > > extension?
>>> > > >> > > > > > > > > >> > > > >
>>> > > >> > > > > > > > > >> > > > > 56. Config changes are public facing.
>>> Could
>>> > > we
>>> > > >> > have
>>> > > >> > > a
>>> > > >> > > > > > > separate
>>> > > >> > > > > > > > > >> > section
>>> > > >> > > > > > > > > >> > > to
>>> > > >> > > > > > > > > >> > > > > document all the config changes?
>>> > > >> > > > > > > > > >> > > >
>>> > > >> > > > > > > > > >> > > > +1. A separate section for this would be
>>> > good.
>>> > > >> > > > > > > > > >> > > >
>>> > > >> > > > > > > > > >> > > > best,
>>> > > >> > > > > > > > > >> > > > Colin
>>> > > >> > > > > > > > > >> > > >
>>> > > >> > > > > > > > > >> > > > >
>>> > > >> > > > > > > > > >> > > > > Thanks,
>>> > > >> > > > > > > > > >> > > > >
>>> > > >> > > > > > > > > >> > > > > Jun
>>> > > >> > > > > > > > > >> > > > >
>>> > > >> > > > > > > > > >> > > > > On Mon, Sep 25, 2023 at 4:29 PM Calvin
>>> Liu
>>> > > >> > > > > > > > > >> > <caliu@confluent.io.invalid
>>> > > >> > > > > > > > > >> > > >
>>> > > >> > > > > > > > > >> > > > > wrote:
>>> > > >> > > > > > > > > >> > > > >
>>> > > >> > > > > > > > > >> > > > >> Hi Jun
>>> > > >> > > > > > > > > >> > > > >> Thanks for the comments.
>>> > > >> > > > > > > > > >> > > > >>
>>> > > >> > > > > > > > > >> > > > >> 40. If we change to None, it is not
>>> > > guaranteed
>>> > > >> > for
>>> > > >> > > no
>>> > > >> > > > > > data
>>> > > >> > > > > > > > > loss.
>>> > > >> > > > > > > > > >> For
>>> > > >> > > > > > > > > >> > > > users
>>> > > >> > > > > > > > > >> > > > >> who are not able to validate the data
>>> with
>>> > > >> > external
>>> > > >> > > > > > > > resources,
>>> > > >> > > > > > > > > >> > manual
>>> > > >> > > > > > > > > >> > > > >> intervention does not give a better
>>> result
>>> > > >> but a
>>> > > >> > > loss
>>> > > >> > > > > of
>>> > > >> > > > > > > > > >> > availability.
>>> > > >> > > > > > > > > >> > > > So
>>> > > >> > > > > > > > > >> > > > >> practically speaking, the Balance mode
>>> > would
>>> > > >> be a
>>> > > >> > > > > better
>>> > > >> > > > > > > > > default
>>> > > >> > > > > > > > > >> > > value.
>>> > > >> > > > > > > > > >> > > > >>
>>> > > >> > > > > > > > > >> > > > >> 41. No, it represents how we want to
>>> do
>>> > the
>>> > > >> > unclean
>>> > > >> > > > > > leader
>>> > > >> > > > > > > > > >> election.
>>> > > >> > > > > > > > > >> > > If
>>> > > >> > > > > > > > > >> > > > it
>>> > > >> > > > > > > > > >> > > > >> is false, the unclean leader election
>>> will
>>> > > be
>>> > > >> the
>>> > > >> > > old
>>> > > >> > > > > > > random
>>> > > >> > > > > > > > > way.
>>> > > >> > > > > > > > > >> > > > >> Otherwise, the unclean recovery will
>>> be
>>> > > used.
>>> > > >> > > > > > > > > >> > > > >>
>>> > > >> > > > > > > > > >> > > > >> 42. Good catch. Updated.
>>> > > >> > > > > > > > > >> > > > >>
>>> > > >> > > > > > > > > >> > > > >> 43. Only the first 20 topics will be
>>> > served.
>>> > > >> > Others
>>> > > >> > > > > will
>>> > > >> > > > > > be
>>> > > >> > > > > > > > > >> returned
>>> > > >> > > > > > > > > >> > > > with
>>> > > >> > > > > > > > > >> > > > >> InvalidRequestError
>>> > > >> > > > > > > > > >> > > > >>
>>> > > >> > > > > > > > > >> > > > >> 44. The order matters. The desired
>>> leader
>>> > > >> entries
>>> > > >> > > > match
>>> > > >> > > > > > > with
>>> > > >> > > > > > > > > the
>>> > > >> > > > > > > > > >> > topic
>>> > > >> > > > > > > > > >> > > > >> partition list by the index.
>>> > > >> > > > > > > > > >> > > > >>
>>> > > >> > > > > > > > > >> > > > >> 45. Thanks! Updated.
>>> > > >> > > > > > > > > >> > > > >>
>>> > > >> > > > > > > > > >> > > > >> 46. Good advice! Updated.
>>> > > >> > > > > > > > > >> > > > >>
>>> > > >> > > > > > > > > >> > > > >> 47.1, updated the comment. Basically
>>> it
>>> > will
>>> > > >> > elect
>>> > > >> > > > the
>>> > > >> > > > > > > > replica
>>> > > >> > > > > > > > > in
>>> > > >> > > > > > > > > >> > the
>>> > > >> > > > > > > > > >> > > > >> desiredLeader field to be the leader
>>> > > >> > > > > > > > > >> > > > >>
>>> > > >> > > > > > > > > >> > > > >> 47.2 We can let the admin client do
>>> the
>>> > > >> > conversion.
>>> > > >> > > > > Using
>>> > > >> > > > > > > the
>>> > > >> > > > > > > > > >> > > > desiredLeader
>>> > > >> > > > > > > > > >> > > > >> field in the json format seems easier
>>> for
>>> > > >> users.
>>> > > >> > > > > > > > > >> > > > >>
>>> > > >> > > > > > > > > >> > > > >> 48. Once the MV version is
>>> downgraded, all
>>> > > the
>>> > > >> > ELR
>>> > > >> > > > > > related
>>> > > >> > > > > > > > > fields
>>> > > >> > > > > > > > > >> > will
>>> > > >> > > > > > > > > >> > > > be
>>> > > >> > > > > > > > > >> > > > >> removed on the next partition change.
>>> The
>>> > > >> > > controller
>>> > > >> > > > > will
>>> > > >> > > > > > > > also
>>> > > >> > > > > > > > > >> > ignore
>>> > > >> > > > > > > > > >> > > > the
>>> > > >> > > > > > > > > >> > > > >> ELR fields. Updated the KIP.
>>> > > >> > > > > > > > > >> > > > >>
>>> > > >> > > > > > > > > >> > > > >> 49. Yes, it would be
>>> deprecated/removed.
>>> > > >> > > > > > > > > >> > > > >>
>>> > > >> > > > > > > > > >> > > > >>
>>> > > >> > > > > > > > > >> > > > >> On Mon, Sep 25, 2023 at 3:49 PM Jun
>>> Rao
>>> > > >> > > > > > > > > <jun@confluent.io.invalid
>>> > > >> > > > > > > > > >> >
>>> > > >> > > > > > > > > >> > > > wrote:
>>> > > >> > > > > > > > > >> > > > >>
>>> > > >> > > > > > > > > >> > > > >> > Hi, Calvin,
>>> > > >> > > > > > > > > >> > > > >> >
>>> > > >> > > > > > > > > >> > > > >> > Thanks for the updated KIP. Made
>>> another
>>> > > >> pass.
>>> > > >> > A
>>> > > >> > > > few
>>> > > >> > > > > > more
>>> > > >> > > > > > > > > >> comments
>>> > > >> > > > > > > > > >> > > > below.
>>> > > >> > > > > > > > > >> > > > >> >
>>> > > >> > > > > > > > > >> > > > >> > 40.
>>> unclean.leader.election.enable.false
>>> > > ->
>>> > > >> > > > > > > > > >> > > > >> > unclean.recovery.strategy.Balanced:
>>> The
>>> > > >> > Balanced
>>> > > >> > > > mode
>>> > > >> > > > > > > could
>>> > > >> > > > > > > > > >> still
>>> > > >> > > > > > > > > >> > > > lead to
>>> > > >> > > > > > > > > >> > > > >> > data loss. So, I am wondering if
>>> > > >> > > > > > > > > >> > > unclean.leader.election.enable.false
>>> > > >> > > > > > > > > >> > > > >> > should map to None?
>>> > > >> > > > > > > > > >> > > > >> >
>>> > > >> > > > > > > > > >> > > > >> > 41.
>>> unclean.recovery.manager.enabled: I
>>> > am
>>> > > >> not
>>> > > >> > > sure
>>> > > >> > > > > why
>>> > > >> > > > > > > we
>>> > > >> > > > > > > > > >> > introduce
>>> > > >> > > > > > > > > >> > > > this
>>> > > >> > > > > > > > > >> > > > >> > additional config. Is it the same as
>>> > > >> > > > > > > > > >> > unclean.recovery.strategy=None?
>>> > > >> > > > > > > > > >> > > > >> >
>>> > > >> > > > > > > > > >> > > > >> > 42.
>>> > > >> > > > DescribeTopicResponse.TopicAuthorizedOperations:
>>> > > >> > > > > > > Should
>>> > > >> > > > > > > > > >> this
>>> > > >> > > > > > > > > >> > be
>>> > > >> > > > > > > > > >> > > at
>>> > > >> > > > > > > > > >> > > > >> the
>>> > > >> > > > > > > > > >> > > > >> > topic level?
>>> > > >> > > > > > > > > >> > > > >> >
>>> > > >> > > > > > > > > >> > > > >> > 43. "Limit: 20 topics max per
>>> request":
>>> > > >> Could
>>> > > >> > we
>>> > > >> > > > > > describe
>>> > > >> > > > > > > > > what
>>> > > >> > > > > > > > > >> > > > happens if
>>> > > >> > > > > > > > > >> > > > >> > the request includes more than 20
>>> > topics?
>>> > > >> > > > > > > > > >> > > > >> >
>>> > > >> > > > > > > > > >> > > > >> > 44.
>>> ElectLeadersRequest.DesiredLeaders:
>>> > > >> Could
>>> > > >> > we
>>> > > >> > > > > > describe
>>> > > >> > > > > > > > > >> whether
>>> > > >> > > > > > > > > >> > > the
>>> > > >> > > > > > > > > >> > > > >> > ordering matters?
>>> > > >> > > > > > > > > >> > > > >> >
>>> > > >> > > > > > > > > >> > > > >> > 45.
>>> GetReplicaLogInfo.TopicPartitions:
>>> > > >> "about":
>>> > > >> > > > "The
>>> > > >> > > > > > > topic
>>> > > >> > > > > > > > > >> > > partitions
>>> > > >> > > > > > > > > >> > > > to
>>> > > >> > > > > > > > > >> > > > >> > elect leaders.": The description in
>>> > > "about"
>>> > > >> is
>>> > > >> > > > > > incorrect.
>>> > > >> > > > > > > > > >> > > > >> >
>>> > > >> > > > > > > > > >> > > > >> > 46. GetReplicaLogInfoResponse:
>>> Should we
>>> > > >> nest
>>> > > >> > > > > > partitions
>>> > > >> > > > > > > > > under
>>> > > >> > > > > > > > > >> > > > topicId to
>>> > > >> > > > > > > > > >> > > > >> > be consistent with other types of
>>> > > responses?
>>> > > >> > > > > > > > > >> > > > >> >
>>> > > >> > > > > > > > > >> > > > >> > 47. kafka-leader-election.sh:
>>> > > >> > > > > > > > > >> > > > >> > 47.1 Could we explain DESIGNATION?
>>> > > >> > > > > > > > > >> > > > >> > 47.2 desiredLeader: Should it be a
>>> list
>>> > to
>>> > > >> > match
>>> > > >> > > > the
>>> > > >> > > > > > > field
>>> > > >> > > > > > > > in
>>> > > >> > > > > > > > > >> > > > >> > ElectLeadersRequest?
>>> > > >> > > > > > > > > >> > > > >> >
>>> > > >> > > > > > > > > >> > > > >> > 48. We could add a section on
>>> downgrade?
>>> > > >> > > > > > > > > >> > > > >> >
>>> > > >> > > > > > > > > >> > > > >> > 49. LastKnownLeader: This seems only
>>> > > needed
>>> > > >> in
>>> > > >> > > the
>>> > > >> > > > > > first
>>> > > >> > > > > > > > > phase
>>> > > >> > > > > > > > > >> of
>>> > > >> > > > > > > > > >> > > > >> > delivering ELR. Will it be removed
>>> when
>>> > > the
>>> > > >> > > > complete
>>> > > >> > > > > > KIP
>>> > > >> > > > > > > is
>>> > > >> > > > > > > > > >> > > delivered?
>>> > > >> > > > > > > > > >> > > > >> >
>>> > > >> > > > > > > > > >> > > > >> > Thanks,
>>> > > >> > > > > > > > > >> > > > >> >
>>> > > >> > > > > > > > > >> > > > >> > Jun
>>> > > >> > > > > > > > > >> > > > >> >
>>> > > >> > > > > > > > > >> > > > >> > On Tue, Sep 19, 2023 at 1:30 PM
>>> Colin
>>> > > >> McCabe <
>>> > > >> > > > > > > > > >> cmccabe@apache.org>
>>> > > >> > > > > > > > > >> > > > wrote:
>>> > > >> > > > > > > > > >> > > > >> >
>>> > > >> > > > > > > > > >> > > > >> > > Hi Calvin,
>>> > > >> > > > > > > > > >> > > > >> > >
>>> > > >> > > > > > > > > >> > > > >> > > Thanks for the explanations. I
>>> like
>>> > the
>>> > > >> idea
>>> > > >> > of
>>> > > >> > > > > using
>>> > > >> > > > > > > > none,
>>> > > >> > > > > > > > > >> > > > balanced,
>>> > > >> > > > > > > > > >> > > > >> > > aggressive. We also had an offline
>>> > > >> discussion
>>> > > >> > > > about
>>> > > >> > > > > > why
>>> > > >> > > > > > > > it
>>> > > >> > > > > > > > > is
>>> > > >> > > > > > > > > >> > good
>>> > > >> > > > > > > > > >> > > > to
>>> > > >> > > > > > > > > >> > > > >> > use a
>>> > > >> > > > > > > > > >> > > > >> > > new config key (basically, so
>>> that we
>>> > > can
>>> > > >> > > > deprecate
>>> > > >> > > > > > the
>>> > > >> > > > > > > > old
>>> > > >> > > > > > > > > >> one
>>> > > >> > > > > > > > > >> > > > which
>>> > > >> > > > > > > > > >> > > > >> had
>>> > > >> > > > > > > > > >> > > > >> > > only false/true values in 4.0)
>>> With
>>> > > these
>>> > > >> > > > changes,
>>> > > >> > > > > I
>>> > > >> > > > > > am
>>> > > >> > > > > > > > +1.
>>> > > >> > > > > > > > > >> > > > >> > >
>>> > > >> > > > > > > > > >> > > > >> > > best,
>>> > > >> > > > > > > > > >> > > > >> > > Colin
>>> > > >> > > > > > > > > >> > > > >> > >
>>> > > >> > > > > > > > > >> > > > >> > > On Mon, Sep 18, 2023, at 15:54,
>>> Calvin
>>> > > Liu
>>> > > >> > > wrote:
>>> > > >> > > > > > > > > >> > > > >> > > > Hi Colin,
>>> > > >> > > > > > > > > >> > > > >> > > > Also, can we deprecate
>>> > > >> > > > > > unclean.leader.election.enable
>>> > > >> > > > > > > > in
>>> > > >> > > > > > > > > >> 4.0?
>>> > > >> > > > > > > > > >> > > > Before
>>> > > >> > > > > > > > > >> > > > >> > > that,
>>> > > >> > > > > > > > > >> > > > >> > > > we can have both the config
>>> > > >> > > > > > unclean.recovery.strategy
>>> > > >> > > > > > > > and
>>> > > >> > > > > > > > > >> > > > >> > > > unclean.leader.election.enable
>>> > > >> > > > > > > > > >> > > > >> > > > and using the
>>> > unclean.recovery.Enabled
>>> > > >> to
>>> > > >> > > > > determine
>>> > > >> > > > > > > > which
>>> > > >> > > > > > > > > >> > config
>>> > > >> > > > > > > > > >> > > > to
>>> > > >> > > > > > > > > >> > > > >> use
>>> > > >> > > > > > > > > >> > > > >> > > > during the unclean leader
>>> election.
>>> > > >> > > > > > > > > >> > > > >> > > >
>>> > > >> > > > > > > > > >> > > > >> > > > On Mon, Sep 18, 2023 at 3:51 PM
>>> > Calvin
>>> > > >> Liu
>>> > > >> > <
>>> > > >> > > > > > > > > >> > caliu@confluent.io>
>>> > > >> > > > > > > > > >> > > > >> wrote:
>>> > > >> > > > > > > > > >> > > > >> > > >
>>> > > >> > > > > > > > > >> > > > >> > > >> Hi Colin,
>>> > > >> > > > > > > > > >> > > > >> > > >> For the
>>> unclean.recovery.strategy
>>> > > >> config
>>> > > >> > > name,
>>> > > >> > > > > how
>>> > > >> > > > > > > > about
>>> > > >> > > > > > > > > >> we
>>> > > >> > > > > > > > > >> > use
>>> > > >> > > > > > > > > >> > > > the
>>> > > >> > > > > > > > > >> > > > >> > > >> following
>>> > > >> > > > > > > > > >> > > > >> > > >> None. It basically means no
>>> unclean
>>> > > >> > recovery
>>> > > >> > > > > will
>>> > > >> > > > > > be
>>> > > >> > > > > > > > > >> > performed.
>>> > > >> > > > > > > > > >> > > > >> > > >> Aggressive. It means
>>> availability
>>> > > goes
>>> > > >> > > first.
>>> > > >> > > > > > > Whenever
>>> > > >> > > > > > > > > the
>>> > > >> > > > > > > > > >> > > > partition
>>> > > >> > > > > > > > > >> > > > >> > > can't
>>> > > >> > > > > > > > > >> > > > >> > > >> elect a durable replica, the
>>> > > controller
>>> > > >> > will
>>> > > >> > > > try
>>> > > >> > > > > > the
>>> > > >> > > > > > > > > >> unclean
>>> > > >> > > > > > > > > >> > > > >> recovery.
>>> > > >> > > > > > > > > >> > > > >> > > >> Balanced. It is the balance
>>> point
>>> > of
>>> > > >> the
>>> > > >> > > > > > > availability
>>> > > >> > > > > > > > > >> > > > >> > first(Aggressive)
>>> > > >> > > > > > > > > >> > > > >> > > >> and least availability(None).
>>> The
>>> > > >> > controller
>>> > > >> > > > > > > performs
>>> > > >> > > > > > > > > >> unclean
>>> > > >> > > > > > > > > >> > > > >> recovery
>>> > > >> > > > > > > > > >> > > > >> > > when
>>> > > >> > > > > > > > > >> > > > >> > > >> both ISR and ELR are empty.
>>> > > >> > > > > > > > > >> > > > >> > > >>
>>> > > >> > > > > > > > > >> > > > >> > > >>
>>> > > >> > > > > > > > > >> > > > >> > > >> On Fri, Sep 15, 2023 at
>>> 11:42 AM
>>> > > Calvin
>>> > > >> > Liu
>>> > > >> > > <
>>> > > >> > > > > > > > > >> > > caliu@confluent.io>
>>> > > >> > > > > > > > > >> > > > >> > wrote:
>>> > > >> > > > > > > > > >> > > > >> > > >>
>>> > > >> > > > > > > > > >> > > > >> > > >>> Hi Colin,
>>> > > >> > > > > > > > > >> > > > >> > > >>>
>>> > > >> > > > > > > > > >> > > > >> > > >>> > So, the proposal is that if
>>> > > someone
>>> > > >> > sets
>>> > > >> > > > > > > > > >> > > > >> > > "unclean.leader.election.enable
>>> > > >> > > > > > > > > >> > > > >> > > >>> = true"...
>>> > > >> > > > > > > > > >> > > > >> > > >>>
>>> > > >> > > > > > > > > >> > > > >> > > >>>
>>> > > >> > > > > > > > > >> > > > >> > > >>> The idea is to use one of the
>>> > > >> > > > > > > > > >> unclean.leader.election.enable
>>> > > >> > > > > > > > > >> > > and
>>> > > >> > > > > > > > > >> > > > >> > > >>> unclean.recovery.strategy
>>> based on
>>> > > the
>>> > > >> > > > > > > > > >> > > > unclean.recovery.Enabled. A
>>> > > >> > > > > > > > > >> > > > >> > > possible
>>> > > >> > > > > > > > > >> > > > >> > > >>> version can be
>>> > > >> > > > > > > > > >> > > > >> > > >>>
>>> > > >> > > > > > > > > >> > > > >> > > >>> If unclean.recovery.Enabled:
>>> > > >> > > > > > > > > >> > > > >> > > >>>
>>> > > >> > > > > > > > > >> > > > >> > > >>> {
>>> > > >> > > > > > > > > >> > > > >> > > >>>
>>> > > >> > > > > > > > > >> > > > >> > > >>> Check
>>> unclean.recovery.strategy.
>>> > If
>>> > > >> set,
>>> > > >> > > use
>>> > > >> > > > > it.
>>> > > >> > > > > > > > > >> Otherwise,
>>> > > >> > > > > > > > > >> > > > check
>>> > > >> > > > > > > > > >> > > > >> > > >>>
>>> unclean.leader.election.enable and
>>> > > >> > > translate
>>> > > >> > > > it
>>> > > >> > > > > > to
>>> > > >> > > > > > > > > >> > > > >> > > >>> unclean.recovery.strategy.
>>> > > >> > > > > > > > > >> > > > >> > > >>>
>>> > > >> > > > > > > > > >> > > > >> > > >>> } else {
>>> > > >> > > > > > > > > >> > > > >> > > >>>
>>> > > >> > > > > > > > > >> > > > >> > > >>> Use
>>> unclean.leader.election.enable
>>> > > >> > > > > > > > > >> > > > >> > > >>>
>>> > > >> > > > > > > > > >> > > > >> > > >>> }
>>> > > >> > > > > > > > > >> > > > >> > > >>>
>>> > > >> > > > > > > > > >> > > > >> > > >>>
>>> > > >> > > > > > > > > >> > > > >> > > >>> —--------
>>> > > >> > > > > > > > > >> > > > >> > > >>>
>>> > > >> > > > > > > > > >> > > > >> > > >>> >The configuration key should
>>> be
>>> > > >> > > > > > > > > >> > > > >> "unclean.recovery.manager.enabled",
>>> > > >> > > > > > > > > >> > > > >> > > >>> right?
>>> > > >> > > > > > > > > >> > > > >> > > >>>
>>> > > >> > > > > > > > > >> > > > >> > > >>>
>>> > > >> > > > > > > > > >> > > > >> > > >>> I think we have two ways of
>>> > > choosing a
>>> > > >> > > leader
>>> > > >> > > > > > > > > uncleanly,
>>> > > >> > > > > > > > > >> > > unclean
>>> > > >> > > > > > > > > >> > > > >> > leader
>>> > > >> > > > > > > > > >> > > > >> > > >>> election and unclean
>>> recovery(log
>>> > > >> > > inspection)
>>> > > >> > > > > and
>>> > > >> > > > > > > we
>>> > > >> > > > > > > > > try
>>> > > >> > > > > > > > > >> to
>>> > > >> > > > > > > > > >> > > > switch
>>> > > >> > > > > > > > > >> > > > >> > > between
>>> > > >> > > > > > > > > >> > > > >> > > >>> them.
>>> > > >> > > > > > > > > >> > > > >> > > >>>
>>> > > >> > > > > > > > > >> > > > >> > > >>> Do you mean we want to
>>> develop two
>>> > > >> ways
>>> > > >> > of
>>> > > >> > > > > > > performing
>>> > > >> > > > > > > > > the
>>> > > >> > > > > > > > > >> > > > unclean
>>> > > >> > > > > > > > > >> > > > >> > > >>> recovery and one of them is
>>> using
>>> > > >> > “unclean
>>> > > >> > > > > > recovery
>>> > > >> > > > > > > > > >> > manager”?
>>> > > >> > > > > > > > > >> > > I
>>> > > >> > > > > > > > > >> > > > >> guess
>>> > > >> > > > > > > > > >> > > > >> > > we
>>> > > >> > > > > > > > > >> > > > >> > > >>> haven’t discussed the second
>>> way.
>>> > > >> > > > > > > > > >> > > > >> > > >>>
>>> > > >> > > > > > > > > >> > > > >> > > >>>
>>> > > >> > > > > > > > > >> > > > >> > > >>> —-------
>>> > > >> > > > > > > > > >> > > > >> > > >>>
>>> > > >> > > > > > > > > >> > > > >> > > >>> >How do these 4 levels of
>>> > overrides
>>> > > >> > > interact
>>> > > >> > > > > with
>>> > > >> > > > > > > > your
>>> > > >> > > > > > > > > >> new
>>> > > >> > > > > > > > > >> > > > >> > > >>> configurations?
>>> > > >> > > > > > > > > >> > > > >> > > >>>
>>> > > >> > > > > > > > > >> > > > >> > > >>>
>>> > > >> > > > > > > > > >> > > > >> > > >>> I do notice in the Kraft
>>> > controller
>>> > > >> code,
>>> > > >> > > the
>>> > > >> > > > > > > method
>>> > > >> > > > > > > > to
>>> > > >> > > > > > > > > >> > check
>>> > > >> > > > > > > > > >> > > > >> whether
>>> > > >> > > > > > > > > >> > > > >> > > >>> perform unclean leader
>>> election is
>>> > > >> hard
>>> > > >> > > coded
>>> > > >> > > > > to
>>> > > >> > > > > > > > false
>>> > > >> > > > > > > > > >> since
>>> > > >> > > > > > > > > >> > > > >> > > >>>
>>> > > >> > 2021(uncleanLeaderElectionEnabledForTopic).
>>> > > >> > > > > Isn’t
>>> > > >> > > > > > > it
>>> > > >> > > > > > > > a
>>> > > >> > > > > > > > > >> good
>>> > > >> > > > > > > > > >> > > > chance
>>> > > >> > > > > > > > > >> > > > >> to
>>> > > >> > > > > > > > > >> > > > >> > > >>> completely deprecate the
>>> > > >> > > > > > > > > unclean.leader.election.enable?
>>> > > >> > > > > > > > > >> We
>>> > > >> > > > > > > > > >> > > > don’t
>>> > > >> > > > > > > > > >> > > > >> > even
>>> > > >> > > > > > > > > >> > > > >> > > have
>>> > > >> > > > > > > > > >> > > > >> > > >>> to worry about the config
>>> > > conversion.
>>> > > >> > > > > > > > > >> > > > >> > > >>>
>>> > > >> > > > > > > > > >> > > > >> > > >>> On the other hand, whatever
>>> the
>>> > > >> override
>>> > > >> > > is,
>>> > > >> > > > as
>>> > > >> > > > > > > long
>>> > > >> > > > > > > > as
>>> > > >> > > > > > > > > >> the
>>> > > >> > > > > > > > > >> > > > >> > controller
>>> > > >> > > > > > > > > >> > > > >> > > >>> can have the final effective
>>> > > >> > > > > > > > > >> unclean.leader.election.enable,
>>> > > >> > > > > > > > > >> > > the
>>> > > >> > > > > > > > > >> > > > >> > topic
>>> > > >> > > > > > > > > >> > > > >> > > >>> level config
>>> > > >> unclean.recovery.strategy,
>>> > > >> > the
>>> > > >> > > > > > cluster
>>> > > >> > > > > > > > > level
>>> > > >> > > > > > > > > >> > > config
>>> > > >> > > > > > > > > >> > > > >> > > >>> unclean.recovery.Enabled, the
>>> > > >> controller
>>> > > >> > > can
>>> > > >> > > > > > > > calculate
>>> > > >> > > > > > > > > >> the
>>> > > >> > > > > > > > > >> > > > correct
>>> > > >> > > > > > > > > >> > > > >> > > methods
>>> > > >> > > > > > > > > >> > > > >> > > >>> to use right?
>>> > > >> > > > > > > > > >> > > > >> > > >>>
>>> > > >> > > > > > > > > >> > > > >> > > >>>
>>> > > >> > > > > > > > > >> > > > >> > > >>> On Fri, Sep 15, 2023 at
>>> 10:02 AM
>>> > > Colin
>>> > > >> > > > McCabe <
>>> > > >> > > > > > > > > >> > > > cmccabe@apache.org>
>>> > > >> > > > > > > > > >> > > > >> > > wrote:
>>> > > >> > > > > > > > > >> > > > >> > > >>>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> On Thu, Sep 14, 2023, at
>>> 22:23,
>>> > > >> Calvin
>>> > > >> > Liu
>>> > > >> > > > > > wrote:
>>> > > >> > > > > > > > > >> > > > >> > > >>>> > Hi Colin
>>> > > >> > > > > > > > > >> > > > >> > > >>>> > 1. I think using the new
>>> config
>>> > > >> name
>>> > > >> > is
>>> > > >> > > > more
>>> > > >> > > > > > > > clear.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >        a. The unclean
>>> leader
>>> > > >> election
>>> > > >> > is
>>> > > >> > > > > > > actually
>>> > > >> > > > > > > > > >> removed
>>> > > >> > > > > > > > > >> > > if
>>> > > >> > > > > > > > > >> > > > >> > unclean
>>> > > >> > > > > > > > > >> > > > >> > > >>>> > recovery is in use.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >        b. Using multiple
>>> values
>>> > > in
>>> > > >> > > > > > > > > >> > > > >> unclean.leader.election.enable
>>> > > >> > > > > > > > > >> > > > >> > is
>>> > > >> > > > > > > > > >> > > > >> > > >>>> > confusing and it will be
>>> more
>>> > > >> > confusing
>>> > > >> > > > > after
>>> > > >> > > > > > > > people
>>> > > >> > > > > > > > > >> > forget
>>> > > >> > > > > > > > > >> > > > >> about
>>> > > >> > > > > > > > > >> > > > >> > > this
>>> > > >> > > > > > > > > >> > > > >> > > >>>> > discussion.
>>> > > >> > > > > > > > > >> > > > >> > > >>>>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> Hi Calvin,
>>> > > >> > > > > > > > > >> > > > >> > > >>>>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> So, the proposal is that if
>>> > someone
>>> > > >> sets
>>> > > >> > > > > > > > > >> > > > >> > > "unclean.leader.election.enable
>>> > > >> > > > > > > > > >> > > > >> > > >>>> = true" but then sets one of
>>> your
>>> > > new
>>> > > >> > > > > > > > configurations,
>>> > > >> > > > > > > > > >> the
>>> > > >> > > > > > > > > >> > > > value of
>>> > > >> > > > > > > > > >> > > > >> > > >>>>
>>> unclean.leader.election.enable is
>>> > > >> > ignored?
>>> > > >> > > > > That
>>> > > >> > > > > > > > seems
>>> > > >> > > > > > > > > >> less
>>> > > >> > > > > > > > > >> > > > clear
>>> > > >> > > > > > > > > >> > > > >> to
>>> > > >> > > > > > > > > >> > > > >> > > me, not
>>> > > >> > > > > > > > > >> > > > >> > > >>>> more. Just in general, having
>>> > > >> multiple
>>> > > >> > > > > > > configuration
>>> > > >> > > > > > > > > >> keys
>>> > > >> > > > > > > > > >> > to
>>> > > >> > > > > > > > > >> > > > >> control
>>> > > >> > > > > > > > > >> > > > >> > > the
>>> > > >> > > > > > > > > >> > > > >> > > >>>> same thing confuses users.
>>> > > Basically,
>>> > > >> > they
>>> > > >> > > > are
>>> > > >> > > > > > > > sitting
>>> > > >> > > > > > > > > >> at a
>>> > > >> > > > > > > > > >> > > > giant
>>> > > >> > > > > > > > > >> > > > >> > > control
>>> > > >> > > > > > > > > >> > > > >> > > >>>> panel, and some of the
>>> levers do
>>> > > >> > nothing.
>>> > > >> > > > > > > > > >> > > > >> > > >>>>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> > 2. Sorry I forgot to
>>> mention in
>>> > > the
>>> > > >> > > > response
>>> > > >> > > > > > > that
>>> > > >> > > > > > > > I
>>> > > >> > > > > > > > > >> did
>>> > > >> > > > > > > > > >> > add
>>> > > >> > > > > > > > > >> > > > the
>>> > > >> > > > > > > > > >> > > > >> > > >>>> > unclean.recovery.Enabled
>>> flag.
>>> > > >> > > > > > > > > >> > > > >> > > >>>>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> The configuration key should
>>> be
>>> > > >> > > > > > > > > >> > > > >> "unclean.recovery.manager.enabled",
>>> > > >> > > > > > > > > >> > > > >> > > >>>> right? Becuase we can do
>>> "unclean
>>> > > >> > > recovery"
>>> > > >> > > > > > > without
>>> > > >> > > > > > > > > the
>>> > > >> > > > > > > > > >> > > > manager.
>>> > > >> > > > > > > > > >> > > > >> > > Disabling
>>> > > >> > > > > > > > > >> > > > >> > > >>>> the manager just means we
>>> use a
>>> > > >> > different
>>> > > >> > > > > > > mechanism
>>> > > >> > > > > > > > > for
>>> > > >> > > > > > > > > >> > > > recovery.
>>> > > >> > > > > > > > > >> > > > >> > > >>>>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >        c. Maybe I
>>> > underestimated
>>> > > >> the
>>> > > >> > > > > challenge
>>> > > >> > > > > > > of
>>> > > >> > > > > > > > > >> > replacing
>>> > > >> > > > > > > > > >> > > > the
>>> > > >> > > > > > > > > >> > > > >> > > >>>> config. Any
>>> > > >> > > > > > > > > >> > > > >> > > >>>> > implementation problems
>>> ahead?
>>> > > >> > > > > > > > > >> > > > >> > > >>>>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> There are four levels of
>>> > overrides
>>> > > >> for
>>> > > >> > > > > > > > > >> > > > >> > unclean.leader.election.enable.
>>> > > >> > > > > > > > > >> > > > >> > > >>>>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> 1. static configuration for
>>> node.
>>> > > >> > > > > > > > > >> > > > >> > > >>>>     This goes in the
>>> > configuration
>>> > > >> file,
>>> > > >> > > > > > typically
>>> > > >> > > > > > > > > named
>>> > > >> > > > > > > > > >> > > > >> > > >>>> server.properties
>>> > > >> > > > > > > > > >> > > > >> > > >>>>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> 2. dynamic configuration for
>>> node
>>> > > >> > default
>>> > > >> > > > > > > > > >> > > > >> > > >>>>   ConfigResource(type=BROKER,
>>> > > >> name="")
>>> > > >> > > > > > > > > >> > > > >> > > >>>>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> 3. dynamic configuration for
>>> node
>>> > > >> > > > > > > > > >> > > > >> > > >>>>   ConfigResource(type=BROKER,
>>> > > >> > > > name=<controller
>>> > > >> > > > > > > id>)
>>> > > >> > > > > > > > > >> > > > >> > > >>>>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> 4. dynamic configuration for
>>> > topic
>>> > > >> > > > > > > > > >> > > > >> > > >>>>   ConfigResource(type=TOPIC,
>>> > > >> > > > > name=<topic-name>)
>>> > > >> > > > > > > > > >> > > > >> > > >>>>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> How do these 4 levels of
>>> > overrides
>>> > > >> > > interact
>>> > > >> > > > > with
>>> > > >> > > > > > > > your
>>> > > >> > > > > > > > > >> new
>>> > > >> > > > > > > > > >> > > > >> > > >>>> configurations? If the new
>>> > > >> > configurations
>>> > > >> > > > > > dominate
>>> > > >> > > > > > > > > over
>>> > > >> > > > > > > > > >> the
>>> > > >> > > > > > > > > >> > > old
>>> > > >> > > > > > > > > >> > > > >> > ones,
>>> > > >> > > > > > > > > >> > > > >> > > it
>>> > > >> > > > > > > > > >> > > > >> > > >>>> seems like this will get a
>>> lot
>>> > more
>>> > > >> > > > confusing
>>> > > >> > > > > to
>>> > > >> > > > > > > > > >> implement
>>> > > >> > > > > > > > > >> > > (and
>>> > > >> > > > > > > > > >> > > > >> also
>>> > > >> > > > > > > > > >> > > > >> > > to
>>> > > >> > > > > > > > > >> > > > >> > > >>>> use.)
>>> > > >> > > > > > > > > >> > > > >> > > >>>>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> Again, I'd recommend just
>>> adding
>>> > > some
>>> > > >> > new
>>> > > >> > > > > values
>>> > > >> > > > > > > to
>>> > > >> > > > > > > > > >> > > > >> > > >>>>
>>> unclean.leader.election.enable.
>>> > > It's
>>> > > >> > > simple
>>> > > >> > > > > and
>>> > > >> > > > > > > will
>>> > > >> > > > > > > > > >> > prevent
>>> > > >> > > > > > > > > >> > > > user
>>> > > >> > > > > > > > > >> > > > >> > > confusion
>>> > > >> > > > > > > > > >> > > > >> > > >>>> (as well as developer
>>> confusion.)
>>> > > >> > > > > > > > > >> > > > >> > > >>>>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> best,
>>> > > >> > > > > > > > > >> > > > >> > > >>>> Colin
>>> > > >> > > > > > > > > >> > > > >> > > >>>>
>>> > > >> > > > > > > > > >> > > > >> > > >>>>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> > 3. About the admin client,
>>> I
>>> > > >> > mentioned 3
>>> > > >> > > > > > changes
>>> > > >> > > > > > > > in
>>> > > >> > > > > > > > > >> the
>>> > > >> > > > > > > > > >> > > > client.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> Anything
>>> > > >> > > > > > > > > >> > > > >> > > >>>> > else I missed in the KIP?
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >       a. The client will
>>> switch
>>> > > to
>>> > > >> > using
>>> > > >> > > > the
>>> > > >> > > > > > new
>>> > > >> > > > > > > > RPC
>>> > > >> > > > > > > > > >> > > instead
>>> > > >> > > > > > > > > >> > > > of
>>> > > >> > > > > > > > > >> > > > >> > > >>>> > MetadataRequest for the
>>> topics.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >       b. The
>>> TopicPartitionInfo
>>> > > >> used
>>> > > >> > in
>>> > > >> > > > > > > > > >> TopicDescription
>>> > > >> > > > > > > > > >> > > > needs
>>> > > >> > > > > > > > > >> > > > >> to
>>> > > >> > > > > > > > > >> > > > >> > > add
>>> > > >> > > > > > > > > >> > > > >> > > >>>> new
>>> > > >> > > > > > > > > >> > > > >> > > >>>> > fields related to the ELR.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >       c. The outputs will
>>> add
>>> > the
>>> > > >> ELR
>>> > > >> > > > > related
>>> > > >> > > > > > > > > fields.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> > On Thu, Sep 14, 2023 at
>>> 9:19 PM
>>> > > >> Colin
>>> > > >> > > > > McCabe <
>>> > > >> > > > > > > > > >> > > > >> cmccabe@apache.org>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> wrote:
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> Hi Calvin,
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> Thanks for the changes.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> 1. Earlier I commented
>>> that
>>> > > >> creating
>>> > > >> > > > > > > > > >> > > > >> "unclean.recovery.strategy "
>>> > > >> > > > > > > > > >> > > > >> > > is
>>> > > >> > > > > > > > > >> > > > >> > > >>>> not
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> necessary, and we can just
>>> > reuse
>>> > > >> the
>>> > > >> > > > > existing
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >>
>>> > "unclean.leader.election.enable"
>>> > > >> > > > > > configuration
>>> > > >> > > > > > > > key.
>>> > > >> > > > > > > > > >> > Let's
>>> > > >> > > > > > > > > >> > > > >> discuss
>>> > > >> > > > > > > > > >> > > > >> > > >>>> that.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> 2.I also don't understand
>>> why
>>> > > you
>>> > > >> > > didn't
>>> > > >> > > > > add
>>> > > >> > > > > > a
>>> > > >> > > > > > > > > >> > > > configuration to
>>> > > >> > > > > > > > > >> > > > >> > > >>>> enable or
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> disable the Unclean
>>> Recovery
>>> > > >> Manager.
>>> > > >> > > > This
>>> > > >> > > > > > > seems
>>> > > >> > > > > > > > > >> like a
>>> > > >> > > > > > > > > >> > > very
>>> > > >> > > > > > > > > >> > > > >> > simple
>>> > > >> > > > > > > > > >> > > > >> > > >>>> way to
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> handle the staging issue
>>> which
>>> > > we
>>> > > >> > > > > discussed.
>>> > > >> > > > > > > The
>>> > > >> > > > > > > > > URM
>>> > > >> > > > > > > > > >> can
>>> > > >> > > > > > > > > >> > > > just
>>> > > >> > > > > > > > > >> > > > >> be
>>> > > >> > > > > > > > > >> > > > >> > > >>>> turned off
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> until it is production
>>> ready.
>>> > > >> Let's
>>> > > >> > > > discuss
>>> > > >> > > > > > > this.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> 3. You still need to
>>> describe
>>> > > the
>>> > > >> > > changes
>>> > > >> > > > > to
>>> > > >> > > > > > > > > >> AdminClient
>>> > > >> > > > > > > > > >> > > > that
>>> > > >> > > > > > > > > >> > > > >> are
>>> > > >> > > > > > > > > >> > > > >> > > >>>> needed
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> to use
>>> DescribeTopicRequest.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> Keep at it. It's looking
>>> > better.
>>> > > >> :)
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> best,
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> Colin
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> On Thu, Sep 14, 2023, at
>>> > 11:03,
>>> > > >> > Calvin
>>> > > >> > > > Liu
>>> > > >> > > > > > > wrote:
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> > Hi Colin
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> > Thanks for the comments!
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> > I did the following
>>> changes
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    1.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    Simplified the API
>>> spec
>>> > > >> section
>>> > > >> > to
>>> > > >> > > > > only
>>> > > >> > > > > > > > > include
>>> > > >> > > > > > > > > >> the
>>> > > >> > > > > > > > > >> > > > diff.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    2.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    Reordered the HWM
>>> > > requirement
>>> > > >> > > > section.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    3.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    Removed the URM
>>> > > >> implementation
>>> > > >> > > > details
>>> > > >> > > > > > to
>>> > > >> > > > > > > > keep
>>> > > >> > > > > > > > > >> the
>>> > > >> > > > > > > > > >> > > > >> necessary
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    characteristics to
>>> > perform
>>> > > >> the
>>> > > >> > > > unclean
>>> > > >> > > > > > > > > recovery.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    1.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >       When to perform
>>> the
>>> > > >> unclean
>>> > > >> > > > > recovery
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >       2.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >       Under different
>>> > config,
>>> > > >> how
>>> > > >> > the
>>> > > >> > > > > > unclean
>>> > > >> > > > > > > > > >> recovery
>>> > > >> > > > > > > > > >> > > > finds
>>> > > >> > > > > > > > > >> > > > >> > the
>>> > > >> > > > > > > > > >> > > > >> > > >>>> leader.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >       3.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >       How the config
>>> > > >> > > > > > > > > unclean.leader.election.enable
>>> > > >> > > > > > > > > >> > and
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>>> > >  unclean.recovery.strategy
>>> > > >> are
>>> > > >> > > > > > converted
>>> > > >> > > > > > > > > when
>>> > > >> > > > > > > > > >> > users
>>> > > >> > > > > > > > > >> > > > >> > > >>>> enable/disable
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> the
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >       unclean recovery.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >       4.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    More details about
>>> how we
>>> > > >> change
>>> > > >> > > > admin
>>> > > >> > > > > > > > client.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    5.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    API limits on the
>>> > > >> > > > > > GetReplicaLogInfoRequest
>>> > > >> > > > > > > > and
>>> > > >> > > > > > > > > >> > > > >> > > >>>> DescribeTopicRequest.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    6.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    Two metrics added
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    1.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>>> > > >> > > > > > > > > >> > > >
>>> > > >> > Kafka.controller.global_under_min_isr_partition_count
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >       2.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>>> > > >> > > > > > > > > >>
>>> kafka.controller.unclean_recovery_finished_count
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> > On Wed, Sep 13, 2023 at
>>> > > 10:46 AM
>>> > > >> > > Colin
>>> > > >> > > > > > > McCabe <
>>> > > >> > > > > > > > > >> > > > >> > > cmccabe@apache.org>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> wrote:
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> On Tue, Sep 12, 2023,
>>> at
>>> > > 17:21,
>>> > > >> > > Calvin
>>> > > >> > > > > Liu
>>> > > >> > > > > > > > > wrote:
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Hi Colin
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Thanks for the
>>> comments!
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> Hi Calvin,
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> Thanks again for the
>>> KIP.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> One meta-comment: it's
>>> > > usually
>>> > > >> > > better
>>> > > >> > > > to
>>> > > >> > > > > > > just
>>> > > >> > > > > > > > > do a
>>> > > >> > > > > > > > > >> > diff
>>> > > >> > > > > > > > > >> > > > on a
>>> > > >> > > > > > > > > >> > > > >> > > >>>> message
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> spec
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> file or java file if
>>> you're
>>> > > >> > > including
>>> > > >> > > > > > > changes
>>> > > >> > > > > > > > to
>>> > > >> > > > > > > > > >> it
>>> > > >> > > > > > > > > >> > in
>>> > > >> > > > > > > > > >> > > > the
>>> > > >> > > > > > > > > >> > > > >> > KIP.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> This is
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> easier to read than
>>> looking
>>> > > for
>>> > > >> > "new
>>> > > >> > > > > > fields
>>> > > >> > > > > > > > > begin"
>>> > > >> > > > > > > > > >> > etc.
>>> > > >> > > > > > > > > >> > > > in
>>> > > >> > > > > > > > > >> > > > >> the
>>> > > >> > > > > > > > > >> > > > >> > > >>>> text, and
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> gracefully handles the
>>> case
>>> > > >> where
>>> > > >> > > > > existing
>>> > > >> > > > > > > > > fields
>>> > > >> > > > > > > > > >> > were
>>> > > >> > > > > > > > > >> > > > >> > changed.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Rewrite the
>>> Additional
>>> > High
>>> > > >> > > > Watermark
>>> > > >> > > > > > > > > >> advancement
>>> > > >> > > > > > > > > >> > > > >> > requirement
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > There was feedback on
>>> > this
>>> > > >> > section
>>> > > >> > > > > that
>>> > > >> > > > > > > some
>>> > > >> > > > > > > > > >> > readers
>>> > > >> > > > > > > > > >> > > > may
>>> > > >> > > > > > > > > >> > > > >> not
>>> > > >> > > > > > > > > >> > > > >> > > be
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> familiar
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > with HWM and
>>> Ack=0,1,all
>>> > > >> > requests.
>>> > > >> > > > > This
>>> > > >> > > > > > > can
>>> > > >> > > > > > > > > help
>>> > > >> > > > > > > > > >> > them
>>> > > >> > > > > > > > > >> > > > >> > > understand
>>> > > >> > > > > > > > > >> > > > >> > > >>>> the
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > proposal. I will
>>> rewrite
>>> > > this
>>> > > >> > part
>>> > > >> > > > for
>>> > > >> > > > > > > more
>>> > > >> > > > > > > > > >> > > > readability.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> To be clear, I wasn't
>>> > > >> suggesting
>>> > > >> > > > > dropping
>>> > > >> > > > > > > > either
>>> > > >> > > > > > > > > >> > > > section. I
>>> > > >> > > > > > > > > >> > > > >> > > agree
>>> > > >> > > > > > > > > >> > > > >> > > >>>> that
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> they add useful
>>> > background. I
>>> > > >> was
>>> > > >> > > just
>>> > > >> > > > > > > > > suggesting
>>> > > >> > > > > > > > > >> > that
>>> > > >> > > > > > > > > >> > > we
>>> > > >> > > > > > > > > >> > > > >> > should
>>> > > >> > > > > > > > > >> > > > >> > > >>>> discuss
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> the "acks" setting
>>> AFTER
>>> > > >> > discussing
>>> > > >> > > > the
>>> > > >> > > > > > new
>>> > > >> > > > > > > > high
>>> > > >> > > > > > > > > >> > > > watermark
>>> > > >> > > > > > > > > >> > > > >> > > >>>> advancement
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> conditions. We also
>>> should
>>> > > >> discuss
>>> > > >> > > > > acks=0.
>>> > > >> > > > > > > > While
>>> > > >> > > > > > > > > >> it
>>> > > >> > > > > > > > > >> > > isn't
>>> > > >> > > > > > > > > >> > > > >> > > >>>> conceptually
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> much
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> different than acks=1
>>> here,
>>> > > its
>>> > > >> > > > omission
>>> > > >> > > > > > > from
>>> > > >> > > > > > > > > this
>>> > > >> > > > > > > > > >> > > > section
>>> > > >> > > > > > > > > >> > > > >> is
>>> > > >> > > > > > > > > >> > > > >> > > >>>> confusing.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Unclean recovery
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > The plan is to
>>> replace
>>> > the
>>> > > >> > > > > > > > > >> > > > unclean.leader.election.enable
>>> > > >> > > > > > > > > >> > > > >> > with
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>>> > unclean.recovery.strategy.
>>> > > If
>>> > > >> > the
>>> > > >> > > > > > Unclean
>>> > > >> > > > > > > > > >> Recovery
>>> > > >> > > > > > > > > >> > is
>>> > > >> > > > > > > > > >> > > > >> > enabled
>>> > > >> > > > > > > > > >> > > > >> > > >>>> then it
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> deals
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > with the three
>>> options in
>>> > > the
>>> > > >> > > > > > > > > >> > > > unclean.recovery.strategy.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Let’s refine the
>>> Unclean
>>> > > >> > Recovery.
>>> > > >> > > > We
>>> > > >> > > > > > have
>>> > > >> > > > > > > > > >> already
>>> > > >> > > > > > > > > >> > > > taken a
>>> > > >> > > > > > > > > >> > > > >> > > lot of
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > suggestions and I
>>> hope to
>>> > > >> > enhance
>>> > > >> > > > the
>>> > > >> > > > > > > > > >> durability of
>>> > > >> > > > > > > > > >> > > > Kafka
>>> > > >> > > > > > > > > >> > > > >> to
>>> > > >> > > > > > > > > >> > > > >> > > the
>>> > > >> > > > > > > > > >> > > > >> > > >>>> next
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> level
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > with this KIP.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> I am OK with doing the
>>> > > unclean
>>> > > >> > > leader
>>> > > >> > > > > > > recovery
>>> > > >> > > > > > > > > >> > > > improvements
>>> > > >> > > > > > > > > >> > > > >> in
>>> > > >> > > > > > > > > >> > > > >> > > >>>> this KIP.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> However, I think we
>>> need to
>>> > > >> really
>>> > > >> > > > work
>>> > > >> > > > > on
>>> > > >> > > > > > > the
>>> > > >> > > > > > > > > >> > > > configuration
>>> > > >> > > > > > > > > >> > > > >> > > >>>> settings.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> Configuration
>>> overrides are
>>> > > >> often
>>> > > >> > > > quite
>>> > > >> > > > > > > messy.
>>> > > >> > > > > > > > > For
>>> > > >> > > > > > > > > >> > > > example,
>>> > > >> > > > > > > > > >> > > > >> > the
>>> > > >> > > > > > > > > >> > > > >> > > >>>> cases
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> where we have
>>> > log.roll.hours
>>> > > >> and
>>> > > >> > > > > > > > > >> log.roll.segment.ms
>>> > > >> > > > > > > > > >> > ,
>>> > > >> > > > > > > > > >> > > > the
>>> > > >> > > > > > > > > >> > > > >> > user
>>> > > >> > > > > > > > > >> > > > >> > > >>>> has to
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> remember which one
>>> takes
>>> > > >> > precedence,
>>> > > >> > > > and
>>> > > >> > > > > > it
>>> > > >> > > > > > > is
>>> > > >> > > > > > > > > not
>>> > > >> > > > > > > > > >> > > > obvious.
>>> > > >> > > > > > > > > >> > > > >> > So,
>>> > > >> > > > > > > > > >> > > > >> > > >>>> rather
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> than
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> creating a new
>>> > configuration,
>>> > > >> why
>>> > > >> > > not
>>> > > >> > > > > add
>>> > > >> > > > > > > > > >> additional
>>> > > >> > > > > > > > > >> > > > values
>>> > > >> > > > > > > > > >> > > > >> to
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> "unclean.leader.election.enable"?
>>> > > >> > I
>>> > > >> > > > > think
>>> > > >> > > > > > > this
>>> > > >> > > > > > > > > >> will
>>> > > >> > > > > > > > > >> > be
>>> > > >> > > > > > > > > >> > > > >> simpler
>>> > > >> > > > > > > > > >> > > > >> > > for
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> people
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> to understand, and
>>> simpler
>>> > in
>>> > > >> the
>>> > > >> > > code
>>> > > >> > > > > as
>>> > > >> > > > > > > > well.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> What if we continued
>>> to use
>>> > > >> > > > > > > > > >> > > > "unclean.leader.election.enable"
>>> > > >> > > > > > > > > >> > > > >> > but
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> extended
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> it so that it took a
>>> > string?
>>> > > >> Then
>>> > > >> > > the
>>> > > >> > > > > > string
>>> > > >> > > > > > > > > could
>>> > > >> > > > > > > > > >> > have
>>> > > >> > > > > > > > > >> > > > >> these
>>> > > >> > > > > > > > > >> > > > >> > > >>>> values:
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> never
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>     never
>>> automatically do
>>> > an
>>> > > >> > > unclean
>>> > > >> > > > > > leader
>>> > > >> > > > > > > > > >> election
>>> > > >> > > > > > > > > >> > > > under
>>> > > >> > > > > > > > > >> > > > >> > any
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> conditions
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> false / default
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>     only do an unclean
>>> > leader
>>> > > >> > > election
>>> > > >> > > > > if
>>> > > >> > > > > > > > there
>>> > > >> > > > > > > > > >> may
>>> > > >> > > > > > > > > >> > be
>>> > > >> > > > > > > > > >> > > > >> > possible
>>> > > >> > > > > > > > > >> > > > >> > > >>>> data
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> loss
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> true / always
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>     always do an
>>> unclean
>>> > > leader
>>> > > >> > > > election
>>> > > >> > > > > > if
>>> > > >> > > > > > > we
>>> > > >> > > > > > > > > >> can't
>>> > > >> > > > > > > > > >> > > > >> > immediately
>>> > > >> > > > > > > > > >> > > > >> > > >>>> elect a
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> leader
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> It's a bit awkward that
>>> > false
>>> > > >> maps
>>> > > >> > > to
>>> > > >> > > > > > > default
>>> > > >> > > > > > > > > >> rather
>>> > > >> > > > > > > > > >> > > > than to
>>> > > >> > > > > > > > > >> > > > >> > > >>>> never. But
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> this awkwardness
>>> exists if
>>> > we
>>> > > >> use
>>> > > >> > > two
>>> > > >> > > > > > > > different
>>> > > >> > > > > > > > > >> > > > >> configuration
>>> > > >> > > > > > > > > >> > > > >> > > keys
>>> > > >> > > > > > > > > >> > > > >> > > >>>> as
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> well.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> The reason for the
>>> > > awkwardness
>>> > > >> is
>>> > > >> > > that
>>> > > >> > > > > we
>>> > > >> > > > > > > > simply
>>> > > >> > > > > > > > > >> > don't
>>> > > >> > > > > > > > > >> > > > want
>>> > > >> > > > > > > > > >> > > > >> > most
>>> > > >> > > > > > > > > >> > > > >> > > >>>> of the
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> people currently
>>> setting
>>> > > >> > > > > > > > > >> > > > >> unclean.leader.election.enable=false
>>> > > >> > > > > > > > > >> > > > >> > to
>>> > > >> > > > > > > > > >> > > > >> > > >>>> get the
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> "never" behavior. We
>>> have
>>> > to
>>> > > >> bite
>>> > > >> > > that
>>> > > >> > > > > > > bullet.
>>> > > >> > > > > > > > > >> Better
>>> > > >> > > > > > > > > >> > > to
>>> > > >> > > > > > > > > >> > > > be
>>> > > >> > > > > > > > > >> > > > >> > > clear
>>> > > >> > > > > > > > > >> > > > >> > > >>>> and
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> explicit than hide it.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> Another thing that's a
>>> bit
>>> > > >> awkward
>>> > > >> > > is
>>> > > >> > > > > > having
>>> > > >> > > > > > > > two
>>> > > >> > > > > > > > > >> > > > different
>>> > > >> > > > > > > > > >> > > > >> > ways
>>> > > >> > > > > > > > > >> > > > >> > > to
>>> > > >> > > > > > > > > >> > > > >> > > >>>> do
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> unclean leader election
>>> > > >> specified
>>> > > >> > in
>>> > > >> > > > the
>>> > > >> > > > > > > KIP.
>>> > > >> > > > > > > > > You
>>> > > >> > > > > > > > > >> > > > descirbe
>>> > > >> > > > > > > > > >> > > > >> two
>>> > > >> > > > > > > > > >> > > > >> > > >>>> methods:
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> the
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> simple "choose the last
>>> > > leader"
>>> > > >> > > > method,
>>> > > >> > > > > > and
>>> > > >> > > > > > > > the
>>> > > >> > > > > > > > > >> > > "unclean
>>> > > >> > > > > > > > > >> > > > >> > > recovery
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> manager"
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> method. I understand
>>> why
>>> > you
>>> > > >> did
>>> > > >> > it
>>> > > >> > > > this
>>> > > >> > > > > > way
>>> > > >> > > > > > > > --
>>> > > >> > > > > > > > > >> > "choose
>>> > > >> > > > > > > > > >> > > > the
>>> > > >> > > > > > > > > >> > > > >> > last
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> leader" is
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> simple, and will help
>>> us
>>> > > >> deliver
>>> > > >> > an
>>> > > >> > > > > > > > > implementation
>>> > > >> > > > > > > > > >> > > > quickly,
>>> > > >> > > > > > > > > >> > > > >> > > while
>>> > > >> > > > > > > > > >> > > > >> > > >>>> the
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> URM
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> is preferable in the
>>> long
>>> > > >> term. My
>>> > > >> > > > > > > suggestion
>>> > > >> > > > > > > > > >> here is
>>> > > >> > > > > > > > > >> > > to
>>> > > >> > > > > > > > > >> > > > >> > > separate
>>> > > >> > > > > > > > > >> > > > >> > > >>>> the
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> decision of HOW to do
>>> > unclean
>>> > > >> > leader
>>> > > >> > > > > > > election
>>> > > >> > > > > > > > > from
>>> > > >> > > > > > > > > >> > the
>>> > > >> > > > > > > > > >> > > > >> > decision
>>> > > >> > > > > > > > > >> > > > >> > > of
>>> > > >> > > > > > > > > >> > > > >> > > >>>> WHEN
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> to
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> do it.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> So in other words, have
>>> > > >> > > > > > > > > >> > > "unclean.leader.election.enable"
>>> > > >> > > > > > > > > >> > > > >> > specify
>>> > > >> > > > > > > > > >> > > > >> > > >>>> when we
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> do unclean leader
>>> election,
>>> > > and
>>> > > >> > > have a
>>> > > >> > > > > new
>>> > > >> > > > > > > > > >> > > configuration
>>> > > >> > > > > > > > > >> > > > >> like
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> "unclean.recovery.manager.enable"
>>> > > >> > to
>>> > > >> > > > > > > determine
>>> > > >> > > > > > > > > if
>>> > > >> > > > > > > > > >> we
>>> > > >> > > > > > > > > >> > > use
>>> > > >> > > > > > > > > >> > > > the
>>> > > >> > > > > > > > > >> > > > >> > > URM.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> Presumably the URM will
>>> > take
>>> > > >> some
>>> > > >> > > time
>>> > > >> > > > > to
>>> > > >> > > > > > > get
>>> > > >> > > > > > > > > >> fully
>>> > > >> > > > > > > > > >> > > > stable,
>>> > > >> > > > > > > > > >> > > > >> so
>>> > > >> > > > > > > > > >> > > > >> > > >>>> this can
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> default to false for a
>>> > while,
>>> > > >> and
>>> > > >> > we
>>> > > >> > > > can
>>> > > >> > > > > > > flip
>>> > > >> > > > > > > > > the
>>> > > >> > > > > > > > > >> > > > default to
>>> > > >> > > > > > > > > >> > > > >> > > true
>>> > > >> > > > > > > > > >> > > > >> > > >>>> when
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> we
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> feel ready.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> The URM is somewhat
>>> > > >> > under-described
>>> > > >> > > > > here.
>>> > > >> > > > > > I
>>> > > >> > > > > > > > > think
>>> > > >> > > > > > > > > >> we
>>> > > >> > > > > > > > > >> > > > need a
>>> > > >> > > > > > > > > >> > > > >> > few
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> configurations here
>>> for it.
>>> > > For
>>> > > >> > > > example,
>>> > > >> > > > > > we
>>> > > >> > > > > > > > > need a
>>> > > >> > > > > > > > > >> > > > >> > > configuration to
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> specify
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> how long it should wait
>>> > for a
>>> > > >> > broker
>>> > > >> > > > to
>>> > > >> > > > > > > > respond
>>> > > >> > > > > > > > > to
>>> > > >> > > > > > > > > >> > its
>>> > > >> > > > > > > > > >> > > > RPCs
>>> > > >> > > > > > > > > >> > > > >> > > before
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> moving
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> on. We also need to
>>> > > understand
>>> > > >> how
>>> > > >> > > the
>>> > > >> > > > > URM
>>> > > >> > > > > > > > > >> interacts
>>> > > >> > > > > > > > > >> > > with
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > unclean.leader.election.enable=always. I
>>> > > >> > > > > > > > assume
>>> > > >> > > > > > > > > >> that
>>> > > >> > > > > > > > > >> > > with
>>> > > >> > > > > > > > > >> > > > >> > > "always"
>>> > > >> > > > > > > > > >> > > > >> > > >>>> we
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> will
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> just unconditionally
>>> use
>>> > the
>>> > > >> URM
>>> > > >> > > > rather
>>> > > >> > > > > > than
>>> > > >> > > > > > > > > >> choosing
>>> > > >> > > > > > > > > >> > > > >> > randomly.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> But this
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> should be spelled out
>>> in
>>> > the
>>> > > >> KIP.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > DescribeTopicRequest
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >    1.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >    Yes, the plan is
>>> to
>>> > > >> replace
>>> > > >> > the
>>> > > >> > > > > > > > > >> MetadataRequest
>>> > > >> > > > > > > > > >> > > with
>>> > > >> > > > > > > > > >> > > > >> the
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>>> DescribeTopicRequest
>>> > for
>>> > > >> the
>>> > > >> > > > admin
>>> > > >> > > > > > > > clients.
>>> > > >> > > > > > > > > >> Will
>>> > > >> > > > > > > > > >> > > > check
>>> > > >> > > > > > > > > >> > > > >> > the
>>> > > >> > > > > > > > > >> > > > >> > > >>>> details.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> Sounds good. But as I
>>> said,
>>> > > you
>>> > > >> > need
>>> > > >> > > > to
>>> > > >> > > > > > > > specify
>>> > > >> > > > > > > > > >> how
>>> > > >> > > > > > > > > >> > > > >> > AdminClient
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> interacts
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> with the new request.
>>> This
>>> > > will
>>> > > >> > > > involve
>>> > > >> > > > > > > adding
>>> > > >> > > > > > > > > >> some
>>> > > >> > > > > > > > > >> > > > fields
>>> > > >> > > > > > > > > >> > > > >> to
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> TopicDescription.java.
>>> And
>>> > > you
>>> > > >> > need
>>> > > >> > > to
>>> > > >> > > > > > > specify
>>> > > >> > > > > > > > > the
>>> > > >> > > > > > > > > >> > > > changes
>>> > > >> > > > > > > > > >> > > > >> to
>>> > > >> > > > > > > > > >> > > > >> > > the
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> kafka-topics.sh command
>>> > line
>>> > > >> tool.
>>> > > >> > > > > > Otherwise
>>> > > >> > > > > > > > we
>>> > > >> > > > > > > > > >> > cannot
>>> > > >> > > > > > > > > >> > > > use
>>> > > >> > > > > > > > > >> > > > >> the
>>> > > >> > > > > > > > > >> > > > >> > > >>>> tool to
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> see
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> the new information.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> The new requests,
>>> > > >> > > DescribeTopicRequest
>>> > > >> > > > > and
>>> > > >> > > > > > > > > >> > > > >> > > >>>> GetReplicaLogInfoRequest,
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> need
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> to have limits placed
>>> on
>>> > them
>>> > > >> so
>>> > > >> > > that
>>> > > >> > > > > > their
>>> > > >> > > > > > > > size
>>> > > >> > > > > > > > > >> > can't
>>> > > >> > > > > > > > > >> > > be
>>> > > >> > > > > > > > > >> > > > >> > > >>>> infinite. We
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> don't want to
>>> propagate the
>>> > > >> > current
>>> > > >> > > > > > problems
>>> > > >> > > > > > > > of
>>> > > >> > > > > > > > > >> > > > >> > MetadataRequest,
>>> > > >> > > > > > > > > >> > > > >> > > >>>> where
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> clients can request
>>> massive
>>> > > >> > > responses
>>> > > >> > > > > that
>>> > > >> > > > > > > can
>>> > > >> > > > > > > > > >> mess
>>> > > >> > > > > > > > > >> > up
>>> > > >> > > > > > > > > >> > > > the
>>> > > >> > > > > > > > > >> > > > >> JVM
>>> > > >> > > > > > > > > >> > > > >> > > when
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> handled.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> Adding limits is
>>> simple for
>>> > > >> > > > > > > > > >> GetReplicaLogInfoRequest
>>> > > >> > > > > > > > > >> > --
>>> > > >> > > > > > > > > >> > > > we
>>> > > >> > > > > > > > > >> > > > >> can
>>> > > >> > > > > > > > > >> > > > >> > > >>>> just say
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> that only 2000
>>> partitions
>>> > at
>>> > > a
>>> > > >> > time
>>> > > >> > > > can
>>> > > >> > > > > be
>>> > > >> > > > > > > > > >> requested.
>>> > > >> > > > > > > > > >> > > For
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> DescribeTopicRequest
>>> we can
>>> > > >> > probably
>>> > > >> > > > > just
>>> > > >> > > > > > > > limit
>>> > > >> > > > > > > > > >> to 20
>>> > > >> > > > > > > > > >> > > > topics
>>> > > >> > > > > > > > > >> > > > >> > or
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> something
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> like that, to avoid the
>>> > > >> complexity
>>> > > >> > > of
>>> > > >> > > > > > doing
>>> > > >> > > > > > > > > >> > pagination
>>> > > >> > > > > > > > > >> > > in
>>> > > >> > > > > > > > > >> > > > >> this
>>> > > >> > > > > > > > > >> > > > >> > > KIP.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >    2.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >    I can let the
>>> broker
>>> > > load
>>> > > >> the
>>> > > >> > > ELR
>>> > > >> > > > > > info
>>> > > >> > > > > > > so
>>> > > >> > > > > > > > > >> that
>>> > > >> > > > > > > > > >> > > they
>>> > > >> > > > > > > > > >> > > > can
>>> > > >> > > > > > > > > >> > > > >> > > serve
>>> > > >> > > > > > > > > >> > > > >> > > >>>> the
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>>> DescribeTopicRequest
>>> > as
>>> > > >> well.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> Yes, it's fine to add
>>> to
>>> > > >> > > > MetadataCache.
>>> > > >> > > > > In
>>> > > >> > > > > > > > fact,
>>> > > >> > > > > > > > > >> > you'll
>>> > > >> > > > > > > > > >> > > > be
>>> > > >> > > > > > > > > >> > > > >> > > loading
>>> > > >> > > > > > > > > >> > > > >> > > >>>> it
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> anyway once it's added
>>> to
>>> > > >> > > > > PartitionImage.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >    3.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >    Yeah, it does not
>>> make
>>> > > >> sense
>>> > > >> > to
>>> > > >> > > > > have
>>> > > >> > > > > > > the
>>> > > >> > > > > > > > > >> topic
>>> > > >> > > > > > > > > >> > id
>>> > > >> > > > > > > > > >> > > if
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>>> DescribeTopicRequest
>>> > is
>>> > > >> only
>>> > > >> > > used
>>> > > >> > > > > by
>>> > > >> > > > > > > the
>>> > > >> > > > > > > > > >> admin
>>> > > >> > > > > > > > > >> > > > client.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> OK. That makes things
>>> > > simpler.
>>> > > >> We
>>> > > >> > > can
>>> > > >> > > > > > always
>>> > > >> > > > > > > > > >> create a
>>> > > >> > > > > > > > > >> > > new
>>> > > >> > > > > > > > > >> > > > >> API
>>> > > >> > > > > > > > > >> > > > >> > > later
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> (hopefully not in this
>>> > KIP!)
>>> > > to
>>> > > >> > > query
>>> > > >> > > > by
>>> > > >> > > > > > > topic
>>> > > >> > > > > > > > > ID.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Metrics
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > As for overall
>>> cluster
>>> > > health
>>> > > >> > > > > metrics, I
>>> > > >> > > > > > > > think
>>> > > >> > > > > > > > > >> > > > >> under-min-ISR
>>> > > >> > > > > > > > > >> > > > >> > > is
>>> > > >> > > > > > > > > >> > > > >> > > >>>> still
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> a
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > useful one. ELR is
>>> more
>>> > > like
>>> > > >> a
>>> > > >> > > > safety
>>> > > >> > > > > > > belt.
>>> > > >> > > > > > > > > When
>>> > > >> > > > > > > > > >> > the
>>> > > >> > > > > > > > > >> > > > ELR
>>> > > >> > > > > > > > > >> > > > >> is
>>> > > >> > > > > > > > > >> > > > >> > > >>>> used, the
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > cluster availability
>>> has
>>> > > >> already
>>> > > >> > > > been
>>> > > >> > > > > > > > > impacted.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Maybe we can have a
>>> > metric
>>> > > to
>>> > > >> > > count
>>> > > >> > > > > the
>>> > > >> > > > > > > > > >> partitions
>>> > > >> > > > > > > > > >> > > that
>>> > > >> > > > > > > > > >> > > > >> > > sum(ISR,
>>> > > >> > > > > > > > > >> > > > >> > > >>>> ELR)
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> <
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> min
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > ISR. What do you
>>> think?
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> How about:
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> A.  a metric for the
>>> totoal
>>> > > >> number
>>> > > >> > > of
>>> > > >> > > > > > > > > >> under-min-isr
>>> > > >> > > > > > > > > >> > > > >> > partitions?
>>> > > >> > > > > > > > > >> > > > >> > > We
>>> > > >> > > > > > > > > >> > > > >> > > >>>> don't
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> have that in Apache
>>> Kafka
>>> > at
>>> > > >> the
>>> > > >> > > > moment.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> B. a metric for the
>>> number
>>> > of
>>> > > >> > > unclean
>>> > > >> > > > > > leader
>>> > > >> > > > > > > > > >> > elections
>>> > > >> > > > > > > > > >> > > we
>>> > > >> > > > > > > > > >> > > > >> did
>>> > > >> > > > > > > > > >> > > > >> > > (for
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> simplicity, it can
>>> reset
>>> > to 0
>>> > > >> on
>>> > > >> > > > > > controller
>>> > > >> > > > > > > > > >> restart:
>>> > > >> > > > > > > > > >> > we
>>> > > >> > > > > > > > > >> > > > >> expect
>>> > > >> > > > > > > > > >> > > > >> > > >>>> people to
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> monitor the change over
>>> > time
>>> > > >> > anyway)
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> best,
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> Colin
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Yeah, for the ongoing
>>> > > unclean
>>> > > >> > > > > > recoveries,
>>> > > >> > > > > > > > the
>>> > > >> > > > > > > > > >> > > > controller
>>> > > >> > > > > > > > > >> > > > >> can
>>> > > >> > > > > > > > > >> > > > >> > > >>>> keep an
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > accurate count
>>> through
>>> > > >> failover
>>> > > >> > > > > because
>>> > > >> > > > > > > > > >> partition
>>> > > >> > > > > > > > > >> > > > >> > registration
>>> > > >> > > > > > > > > >> > > > >> > > >>>> can
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> indicate
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > whether a recovery is
>>> > > needed.
>>> > > >> > > > However,
>>> > > >> > > > > > for
>>> > > >> > > > > > > > the
>>> > > >> > > > > > > > > >> > > happened
>>> > > >> > > > > > > > > >> > > > >> > ones,
>>> > > >> > > > > > > > > >> > > > >> > > >>>> unless
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> we
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > want to persist the
>>> > number
>>> > > >> > > > somewhere,
>>> > > >> > > > > we
>>> > > >> > > > > > > can
>>> > > >> > > > > > > > > >> only
>>> > > >> > > > > > > > > >> > > > figure
>>> > > >> > > > > > > > > >> > > > >> it
>>> > > >> > > > > > > > > >> > > > >> > > out
>>> > > >> > > > > > > > > >> > > > >> > > >>>> from
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> the
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > log.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > On Tue, Sep 12, 2023
>>> at
>>> > > >> 3:16 PM
>>> > > >> > > > Colin
>>> > > >> > > > > > > > McCabe <
>>> > > >> > > > > > > > > >> > > > >> > > cmccabe@apache.org
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> wrote:
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> Also, we should have
>>> > > metrics
>>> > > >> > that
>>> > > >> > > > > show
>>> > > >> > > > > > > what
>>> > > >> > > > > > > > > is
>>> > > >> > > > > > > > > >> > going
>>> > > >> > > > > > > > > >> > > > on
>>> > > >> > > > > > > > > >> > > > >> > with
>>> > > >> > > > > > > > > >> > > > >> > > >>>> regard
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> to
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> the
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> eligible replica
>>> set.
>>> > I'm
>>> > > >> not
>>> > > >> > > sure
>>> > > >> > > > > > > exactly
>>> > > >> > > > > > > > > >> what to
>>> > > >> > > > > > > > > >> > > > >> suggest,
>>> > > >> > > > > > > > > >> > > > >> > > but
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> something
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> that could identify
>>> when
>>> > > >> things
>>> > > >> > > are
>>> > > >> > > > > > going
>>> > > >> > > > > > > > > >> wrong in
>>> > > >> > > > > > > > > >> > > the
>>> > > >> > > > > > > > > >> > > > >> > > clsuter.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> For example, maybe a
>>> > > metric
>>> > > >> for
>>> > > >> > > > > > > partitions
>>> > > >> > > > > > > > > >> > > containing
>>> > > >> > > > > > > > > >> > > > >> > > replicas
>>> > > >> > > > > > > > > >> > > > >> > > >>>> that
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> are
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> ineligible to be
>>> leader?
>>> > > >> That
>>> > > >> > > would
>>> > > >> > > > > > show
>>> > > >> > > > > > > a
>>> > > >> > > > > > > > > >> spike
>>> > > >> > > > > > > > > >> > > when
>>> > > >> > > > > > > > > >> > > > a
>>> > > >> > > > > > > > > >> > > > >> > > broker
>>> > > >> > > > > > > > > >> > > > >> > > >>>> had an
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> unclean restart.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> Ideally, we'd also
>>> have
>>> > a
>>> > > >> > metric
>>> > > >> > > > that
>>> > > >> > > > > > > > > indicates
>>> > > >> > > > > > > > > >> > when
>>> > > >> > > > > > > > > >> > > > an
>>> > > >> > > > > > > > > >> > > > >> > > unclear
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> leader
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> election or a
>>> recovery
>>> > > >> > happened.
>>> > > >> > > > > It's a
>>> > > >> > > > > > > bit
>>> > > >> > > > > > > > > >> tricky
>>> > > >> > > > > > > > > >> > > > >> because
>>> > > >> > > > > > > > > >> > > > >> > > the
>>> > > >> > > > > > > > > >> > > > >> > > >>>> simple
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> thing, of tracking
>>> it
>>> > per
>>> > > >> > > > controller,
>>> > > >> > > > > > may
>>> > > >> > > > > > > > be
>>> > > >> > > > > > > > > a
>>> > > >> > > > > > > > > >> bit
>>> > > >> > > > > > > > > >> > > > >> > confusing
>>> > > >> > > > > > > > > >> > > > >> > > >>>> during
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> failovers.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> best,
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> Colin
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >>
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> On Tue, Sep 12,
>>> 2023, at
>>> > > >> 14:25,
>>> > > >> > > > Colin
>>> > > >> > > > > > > > McCabe
>>> > > >> > > > > > > > > >> > wrote:
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > Hi Calvin,
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > Thanks for the
>>> KIP. I
>>> > > >> think
>>> > > >> > > this
>>> > > >> > > > > is a
>>> > > >> > > > > > > > great
>>> > > >> > > > > > > > > >> > > > >> improvement.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >> Additional High
>>> > > Watermark
>>> > > >> > > > advance
>>> > > >> > > > > > > > > >> requirement
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > Typo: change
>>> "advance"
>>> > > to
>>> > > >> > > > > > "advancement"
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >> A bit recap of
>>> some
>>> > key
>>> > > >> > > > concepts.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > Typo: change
>>> "bit" to
>>> > > >> "quick"
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >> Ack=1/all produce
>>> > > >> request.
>>> > > >> > It
>>> > > >> > > > > > defines
>>> > > >> > > > > > > > when
>>> > > >> > > > > > > > > >> the
>>> > > >> > > > > > > > > >> > > > Kafka
>>> > > >> > > > > > > > > >> > > > >> > > server
>>> > > >> > > > > > > > > >> > > > >> > > >>>> should
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> respond to the
>>> produce
>>> > > >> request
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > I think this
>>> section
>>> > > >> would be
>>> > > >> > > > > clearer
>>> > > >> > > > > > > if
>>> > > >> > > > > > > > we
>>> > > >> > > > > > > > > >> > talked
>>> > > >> > > > > > > > > >> > > > >> about
>>> > > >> > > > > > > > > >> > > > >> > > the
>>> > > >> > > > > > > > > >> > > > >> > > >>>> new
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> high
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > watermark
>>> advancement
>>> > > >> > > requirement
>>> > > >> > > > > > > first,
>>> > > >> > > > > > > > > and
>>> > > >> > > > > > > > > >> > THEN
>>> > > >> > > > > > > > > >> > > > >> talked
>>> > > >> > > > > > > > > >> > > > >> > > >>>> about its
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > impact on acks=0,
>>> > > acks=1,
>>> > > >> and
>>> > > >> > > > > > > >  acks=all.
>>> > > >> > > > > > > > > >> > > > acks=all
>>> > > >> > > > > > > > > >> > > > >> is
>>> > > >> > > > > > > > > >> > > > >> > of
>>> > > >> > > > > > > > > >> > > > >> > > >>>> course
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> the
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > main case we care
>>> > about
>>> > > >> here,
>>> > > >> > > so
>>> > > >> > > > it
>>> > > >> > > > > > > would
>>> > > >> > > > > > > > > be
>>> > > >> > > > > > > > > >> > good
>>> > > >> > > > > > > > > >> > > to
>>> > > >> > > > > > > > > >> > > > >> lead
>>> > > >> > > > > > > > > >> > > > >> > > with
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> that,
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > rather than
>>> delving
>>> > into
>>> > > >> the
>>> > > >> > > > > > > > technicalities
>>> > > >> > > > > > > > > >> of
>>> > > >> > > > > > > > > >> > > > acks=0/1
>>> > > >> > > > > > > > > >> > > > >> > > first.
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >> Unclean recovery
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > So, here you are
>>> > > >> introducing
>>> > > >> > a
>>> > > >> > > > new
>>> > > >> > > > > > > > > >> > configuration,
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
>>> > > unclean.recovery.strategy.
>>> > > >> > The
>>> > > >> > > > > > > difficult
>>> > > >> > > > > > > > > >> thing
>>> > > >> > > > > > > > > >> > > here
>>> > > >> > > > > > > > > >> > > > is
>>> > > >> > > > > > > > > >> > > > >> > that
>>> > > >> > > > > > > > > >> > > > >> > > >>>> there
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> is a
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > lot of overlap
>>> with
>>> > > >> > > > > > > > > >> > > unclean.leader.election.enable.
>>> > > >> > > > > > > > > >> > > > So
>>> > > >> > > > > > > > > >> > > > >> we
>>> > > >> > > > > > > > > >> > > > >> > > >>>> have 3
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > different
>>> settings for
>>> > > >> > > > > > > > > >> > unclean.recovery.strategy,
>>> > > >> > > > > > > > > >> > > > plus
>>> > > >> > > > > > > > > >> > > > >> 2
>>> > > >> > > > > > > > > >> > > > >> > > >>>> different
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > settings for
>>> > > >> > > > > > > > > unclean.leader.election.enable,
>>> > > >> > > > > > > > > >> > > giving
>>> > > >> > > > > > > > > >> > > > a
>>> > > >> > > > > > > > > >> > > > >> > cross
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> product of
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > 6 different
>>> options.
>>> > The
>>> > > >> > > > following
>>> > > >> > > > > > > > "unclean
>>> > > >> > > > > > > > > >> > > recovery
>>> > > >> > > > > > > > > >> > > > >> > > manager"
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> section
>>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > on
>>> > > >> > > > > >
>>> > > >> > > > >
>>> > > >> > > >
>>> > > >> > >
>>> > > >> >
>>> > > >>
>>> > > >
>>> > >
>>> >
>>>
>>>
>>> --
>>> David Arthur
>>>
>>

-- 
-David

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Calvin Liu <ca...@confluent.io.INVALID>.
Hey,
As we agreed to implement the pagination for the new API
DescribeTopicPartitions, the client side must also add a proper interface
to handle the pagination.
The current KafkaAdminClient.describeTopics returns
the DescribeTopicsResult which is the future for querying all the topics.
It is awkward to fit the pagination into it because

   1. Each future corresponds to a topic. We also want to have the
   pagination on huge topics for their partitions.
   2. To avoid OOM, we should only fetch the new topics when we need them
   and release the used topics. Especially the main use case of looping the
   topic list is when the client prints all the topics.

So, to better serve the pagination, @David Arthur
<da...@confluent.io> suggested to add a new interface in the Admin
client between the following 2.

describeTopics(TopicCollection topics, DescribeTopicsOptions options,
Consumer<TopicDescription>);

Iterator<TopicDescription> describeTopics(TopicCollection topics,
DescribeTopicsOptions options);

David and I would prefer the first Consumer version which works better
as a stream purposes.


On Wed, Oct 11, 2023 at 4:28 PM Calvin Liu <ca...@confluent.io> wrote:

> Hi David,
> Thanks for the comment.
> Yes, we can separate the ELR enablement from the metadata version. It is
> also helpful to avoid blocking the following MV releases if the user is not
> ready for ELR.
> One thing to correct is that, the Unclean recovery is controlled
> by unclean.recovery.manager.enabled, a separate config
> from unclean.recovery.strategy. It determines whether unclean recovery will
> be used in an unclean leader election.
> Thanks
>
> On Wed, Oct 11, 2023 at 4:11 PM David Arthur <mu...@gmail.com> wrote:
>
>> One thing we should consider is a static config to totally enable/disable
>> the ELR feature. If I understand the KIP correctly, we can effectively
>> disable the unclean recovery by setting the recovery strategy config to
>> "none".
>>
>> This would make development and rollout of this feature a bit smoother.
>> Consider the case that we find bugs in ELR after a cluster has updated to
>> its MetadataVersion. It's simpler to disable the feature through config
>> rather than going through a MetadataVersion downgrade (once that's
>> supported).
>>
>> Does that make sense?
>>
>> -David
>>
>> On Wed, Oct 11, 2023 at 1:40 PM Calvin Liu <ca...@confluent.io.invalid>
>> wrote:
>>
>> > Hi Jun
>> > -Good catch, yes, we don't need the -1 in the DescribeTopicRequest.
>> > -No new value is added. The LeaderRecoveryState will still be set to 1
>> if
>> > we have an unclean leader election. The unclean leader election includes
>> > the old random way and the unclean recovery. During the unclean
>> recovery,
>> > the LeaderRecoveryState will not change until the controller decides to
>> > update the records with the new leader.
>> > Thanks
>> >
>> > On Wed, Oct 11, 2023 at 9:02 AM Jun Rao <ju...@confluent.io.invalid>
>> wrote:
>> >
>> > > Hi, Calvin,
>> > >
>> > > Another thing. Currently, when there is an unclean leader election, we
>> > set
>> > > the LeaderRecoveryState in PartitionRecord and PartitionChangeRecord
>> to
>> > 1.
>> > > With the KIP, will there be new values for LeaderRecoveryState? If
>> not,
>> > > when will LeaderRecoveryState be set to 1?
>> > >
>> > > Thanks,
>> > >
>> > > Jun
>> > >
>> > > On Tue, Oct 10, 2023 at 4:24 PM Jun Rao <ju...@confluent.io> wrote:
>> > >
>> > > > Hi, Calvin,
>> > > >
>> > > > One more comment.
>> > > >
>> > > > "The first partition to fetch details for. -1 means to fetch all
>> > > > partitions." It seems that FirstPartitionId of 0 naturally means
>> > fetching
>> > > > all partitions?
>> > > >
>> > > > Thanks,
>> > > >
>> > > > Jun
>> > > >
>> > > > On Tue, Oct 10, 2023 at 12:40 PM Calvin Liu
>> <caliu@confluent.io.invalid
>> > >
>> > > > wrote:
>> > > >
>> > > >> Hi Jun,
>> > > >> Yeah, with the current Metadata request handling, we only return
>> > errors
>> > > on
>> > > >> the Topic level, like topic not found. It seems that querying a
>> > specific
>> > > >> partition is not a valid use case. Will update.
>> > > >> Thanks
>> > > >>
>> > > >> On Tue, Oct 10, 2023 at 11:55 AM Jun Rao <jun@confluent.io.invalid
>> >
>> > > >> wrote:
>> > > >>
>> > > >> > Hi, Calvin,
>> > > >> >
>> > > >> > 60.  If the range query has errors for some of the partitions,
>> do we
>> > > >> expect
>> > > >> > different responses when querying particular partitions?
>> > > >> >
>> > > >> > Thanks,
>> > > >> >
>> > > >> > Jun
>> > > >> >
>> > > >> > On Tue, Oct 10, 2023 at 10:50 AM Calvin Liu
>> > > <caliu@confluent.io.invalid
>> > > >> >
>> > > >> > wrote:
>> > > >> >
>> > > >> > > Hi Jun
>> > > >> > > 60. Yes, it is a good question. I was thinking the API could be
>> > > >> flexible
>> > > >> > to
>> > > >> > > query the particular partitions if the range query has errors
>> for
>> > > >> some of
>> > > >> > > the partitions. Not sure whether it is a valid assumption,
>> what do
>> > > you
>> > > >> > > think?
>> > > >> > >
>> > > >> > > 61. Good point, I will update them to partition level with the
>> > same
>> > > >> > limit.
>> > > >> > >
>> > > >> > > 62. Sure, will do.
>> > > >> > >
>> > > >> > > Thanks
>> > > >> > >
>> > > >> > > On Tue, Oct 10, 2023 at 10:12 AM Jun Rao
>> <jun@confluent.io.invalid
>> > >
>> > > >> > wrote:
>> > > >> > >
>> > > >> > > > Hi, Calvin,
>> > > >> > > >
>> > > >> > > > A few more minor comments on your latest update.
>> > > >> > > >
>> > > >> > > > 60. DescribeTopicRequest: When will the Partitions field be
>> > used?
>> > > It
>> > > >> > > seems
>> > > >> > > > that the FirstPartitionId field is enough for AdminClient
>> usage.
>> > > >> > > >
>> > > >> > > > 61. Could we make the limit for DescribeTopicRequest,
>> > > >> > > ElectLeadersRequest,
>> > > >> > > > GetReplicaLogInfo consistent? Currently,
>> ElectLeadersRequest's
>> > > >> limit is
>> > > >> > > at
>> > > >> > > > topic level and GetReplicaLogInfo has a different partition
>> > level
>> > > >> limit
>> > > >> > > > from DescribeTopicRequest.
>> > > >> > > >
>> > > >> > > > 62. Should ElectLeadersRequest.DesiredLeaders be at the same
>> > level
>> > > >> as
>> > > >> > > > ElectLeadersRequest.TopicPartitions.Partitions? In the KIP,
>> it
>> > > looks
>> > > >> > like
>> > > >> > > > it's at the same level as
>> ElectLeadersRequest.TopicPartitions.
>> > > >> > > >
>> > > >> > > > Thanks,
>> > > >> > > >
>> > > >> > > > Jun
>> > > >> > > >
>> > > >> > > > On Wed, Oct 4, 2023 at 3:55 PM Calvin Liu
>> > > >> <ca...@confluent.io.invalid>
>> > > >> > > > wrote:
>> > > >> > > >
>> > > >> > > > > Hi David,
>> > > >> > > > > Thanks for the comments.
>> > > >> > > > > ----
>> > > >> > > > > I thought that a new snapshot with the downgraded MV is
>> > created
>> > > in
>> > > >> > this
>> > > >> > > > > case. Isn’t it the case?
>> > > >> > > > > Yes, you are right, a metadata delta will be generated
>> after
>> > the
>> > > >> MV
>> > > >> > > > > downgrade. Then the user can start the software downgrade.
>> > > >> > > > > -----
>> > > >> > > > > Could you also elaborate a bit more on the reasoning behind
>> > > adding
>> > > >> > the
>> > > >> > > > > limits to the admin RPCs? This is a new pattern in Kafka
>> so it
>> > > >> would
>> > > >> > be
>> > > >> > > > > good to clear on the motivation.
>> > > >> > > > > Thanks to Colin for bringing it up. The current
>> > MetadataRequest
>> > > >> does
>> > > >> > > not
>> > > >> > > > > have a limit on the number of topics to query in a single
>> > > request.
>> > > >> > > > Massive
>> > > >> > > > > requests can mess up the JVM. We want to have some sort of
>> > > >> throttle
>> > > >> > on
>> > > >> > > > the
>> > > >> > > > > new APIs.
>> > > >> > > > > -----
>> > > >> > > > > Could you also explain how the client is supposed to handle
>> > the
>> > > >> > > > > topics/partitions above the limit? I suppose that it will
>> have
>> > > to
>> > > >> > retry
>> > > >> > > > > those, correct?
>> > > >> > > > > Corrent. For the official admin clients, it will split the
>> > large
>> > > >> > > request
>> > > >> > > > > into proper pieces and query one after another.
>> > > >> > > > > -----
>> > > >> > > > > My understanding is that the topics/partitions above the
>> limit
>> > > >> will
>> > > >> > be
>> > > >> > > > > failed with an invalid exception error. I wonder if this
>> > choice
>> > > is
>> > > >> > > > > judicious because the invalide request exception is usually
>> > > >> fatal. It
>> > > >> > > may
>> > > >> > > > > be better to use an new and explicit error for this case.
>> > > >> > > > >
>> > > >> > > > > Thanks for bringing this up. How about
>> > "REQUEST_LIMIT_REACHED"?
>> > > >> > > > > --------
>> > > >> > > > > It seems that we still need to specify the changes to the
>> > admin
>> > > >> api
>> > > >> > to
>> > > >> > > > > accommodate the new or updated apis. Do you plan to add
>> them?
>> > > >> > > > > Try to cover the following
>> > > >> > > > > 1. The admin client will use the new DescribeTopicRequest
>> to
>> > > query
>> > > >> > the
>> > > >> > > > > topics
>> > > >> > > > > 2. Mention the API limit and the new retriable error.
>> > > >> > > > > 3. Output changes for the admin client when describing a
>> topic
>> > > >> (new
>> > > >> > > > fields
>> > > >> > > > > of ELR...)
>> > > >> > > > > 4. Changes to data structures like TopicPartitionInfo to
>> > include
>> > > >> the
>> > > >> > > ELR.
>> > > >> > > > > Anything else I missed?
>> > > >> > > > >
>> > > >> > > > > Thanks!
>> > > >> > > > >
>> > > >> > > > >
>> > > >> > > > >
>> > > >> > > > >
>> > > >> > > > >
>> > > >> > > > > On Wed, Oct 4, 2023 at 12:27 PM David Jacot <
>> > > >> david.jacot@gmail.com>
>> > > >> > > > wrote:
>> > > >> > > > >
>> > > >> > > > > > Hi Calvin,
>> > > >> > > > > >
>> > > >> > > > > > I thought that a new snapshot with the downgraded MV is
>> > > created
>> > > >> in
>> > > >> > > this
>> > > >> > > > > > case. Isn’t it the case?
>> > > >> > > > > >
>> > > >> > > > > > Could you also elaborate a bit more on the reasoning
>> behind
>> > > >> adding
>> > > >> > > the
>> > > >> > > > > > limits to the admin RPCs? This is a new pattern in Kafka
>> so
>> > it
>> > > >> > would
>> > > >> > > be
>> > > >> > > > > > good to clear on the motivation.
>> > > >> > > > > >
>> > > >> > > > > > Could you also explain how the client is supposed to
>> handle
>> > > the
>> > > >> > > > > > topics/partitions above the limit? I suppose that it will
>> > have
>> > > >> to
>> > > >> > > retry
>> > > >> > > > > > those, correct?
>> > > >> > > > > >
>> > > >> > > > > > My understanding is that the topics/partitions above the
>> > limit
>> > > >> will
>> > > >> > > be
>> > > >> > > > > > failed with an invalid exception error. I wonder if this
>> > > choice
>> > > >> is
>> > > >> > > > > > judicious because the invalide request exception is
>> usually
>> > > >> fatal.
>> > > >> > It
>> > > >> > > > may
>> > > >> > > > > > be better to use an new and explicit error for this case.
>> > > >> > > > > >
>> > > >> > > > > > It seems that we still need to specify the changes to the
>> > > admin
>> > > >> api
>> > > >> > > to
>> > > >> > > > > > accommodate the new or updated apis. Do you plan to add
>> > them?
>> > > >> > > > > >
>> > > >> > > > > > Best,
>> > > >> > > > > > David
>> > > >> > > > > >
>> > > >> > > > > > Le mer. 4 oct. 2023 à 20:39, Calvin Liu
>> > > >> <caliu@confluent.io.invalid
>> > > >> > >
>> > > >> > > a
>> > > >> > > > > > écrit :
>> > > >> > > > > >
>> > > >> > > > > > > Hi Jun,
>> > > >> > > > > > > After the MV downgrade, the controller will write in
>> the
>> > old
>> > > >> > > version
>> > > >> > > > of
>> > > >> > > > > > the
>> > > >> > > > > > > PartitionRecord/PartitionChangeRecord. If I understand
>> > > >> correctly,
>> > > >> > > it
>> > > >> > > > is
>> > > >> > > > > > > possible to downgrade the software version if the
>> > controller
>> > > >> only
>> > > >> > > has
>> > > >> > > > > to
>> > > >> > > > > > > handle old version records.
>> > > >> > > > > > > However, the controller will not automatically rewrite
>> the
>> > > >> > > > > > PartitionRecord
>> > > >> > > > > > > with the old version unless there is a partition
>> update.
>> > > Then,
>> > > >> > the
>> > > >> > > > user
>> > > >> > > > > > may
>> > > >> > > > > > > have to wait an unknown amount of time before the
>> software
>> > > >> > > downgrades
>> > > >> > > > > > > unless they do a roll to force update every partition.
>> If
>> > it
>> > > >> > makes
>> > > >> > > > > > sense, I
>> > > >> > > > > > > can mention these steps to do a software downgrade.
>> > > >> > > > > > > Thanks
>> > > >> > > > > > >
>> > > >> > > > > > > On Wed, Oct 4, 2023 at 11:20 AM Jun Rao
>> > > >> <jun@confluent.io.invalid
>> > > >> > >
>> > > >> > > > > > wrote:
>> > > >> > > > > > >
>> > > >> > > > > > > > Hi, Calvin and Justine,
>> > > >> > > > > > > >
>> > > >> > > > > > > > Historically, when we change the record format in the
>> > log,
>> > > >> we
>> > > >> > > don't
>> > > >> > > > > > > support
>> > > >> > > > > > > > software version downgrading.
>> > > >> > > > > > > >
>> > > >> > > > > > > > For the record format change in the metadata log,
>> have
>> > we
>> > > >> > thought
>> > > >> > > > > about
>> > > >> > > > > > > > forcing the write of the latest metadata records with
>> > the
>> > > >> old
>> > > >> > > > version
>> > > >> > > > > > > > during MV downgrading? This will in theory allow the
>> old
>> > > >> > version
>> > > >> > > of
>> > > >> > > > > the
>> > > >> > > > > > > > software to obtain the latest metadata.
>> > > >> > > > > > > >
>> > > >> > > > > > > > Thanks,
>> > > >> > > > > > > >
>> > > >> > > > > > > > Jun
>> > > >> > > > > > > >
>> > > >> > > > > > > > On Wed, Oct 4, 2023 at 9:53 AM Justine Olshan
>> > > >> > > > > > > <jolshan@confluent.io.invalid
>> > > >> > > > > > > > >
>> > > >> > > > > > > > wrote:
>> > > >> > > > > > > >
>> > > >> > > > > > > > > Sorry -- not MV but software version.
>> > > >> > > > > > > > >
>> > > >> > > > > > > > > On Wed, Oct 4, 2023 at 9:51 AM Justine Olshan <
>> > > >> > > > > jolshan@confluent.io>
>> > > >> > > > > > > > > wrote:
>> > > >> > > > > > > > >
>> > > >> > > > > > > > > > Catching up with this discussion.
>> > > >> > > > > > > > > >
>> > > >> > > > > > > > > > I was just curious -- have we had other instances
>> > > where
>> > > >> > > > > downgrading
>> > > >> > > > > > > MV
>> > > >> > > > > > > > is
>> > > >> > > > > > > > > > not supported? I think Kafka typically tries to
>> > > support
>> > > >> > > > > downgrades,
>> > > >> > > > > > > > and I
>> > > >> > > > > > > > > > couldn't think of other examples.
>> > > >> > > > > > > > > >
>> > > >> > > > > > > > > > Thanks,
>> > > >> > > > > > > > > > Justine
>> > > >> > > > > > > > > >
>> > > >> > > > > > > > > > On Wed, Oct 4, 2023 at 9:40 AM Calvin Liu
>> > > >> > > > > > <caliu@confluent.io.invalid
>> > > >> > > > > > > >
>> > > >> > > > > > > > > > wrote:
>> > > >> > > > > > > > > >
>> > > >> > > > > > > > > >> Hi Jun,
>> > > >> > > > > > > > > >> 54. Marked the software downgrading is not
>> > supported.
>> > > >> As
>> > > >> > the
>> > > >> > > > old
>> > > >> > > > > > > > > >> controller
>> > > >> > > > > > > > > >> will not understand the new PartitionRecord and
>> > > >> > > > > > > PartitionChangeRecord.
>> > > >> > > > > > > > > >> Thanks!
>> > > >> > > > > > > > > >>
>> > > >> > > > > > > > > >> On Wed, Oct 4, 2023 at 9:12 AM Jun Rao
>> > > >> > > > <jun@confluent.io.invalid
>> > > >> > > > > >
>> > > >> > > > > > > > > wrote:
>> > > >> > > > > > > > > >>
>> > > >> > > > > > > > > >> > Hi, Calvin,
>> > > >> > > > > > > > > >> >
>> > > >> > > > > > > > > >> > Thanks for the reply. Just one more comment.
>> > > >> > > > > > > > > >> >
>> > > >> > > > > > > > > >> > 54. It seems that downgrading MV is
>> supported. Is
>> > > >> > > > downgrading
>> > > >> > > > > > the
>> > > >> > > > > > > > > >> software
>> > > >> > > > > > > > > >> > version supported? It would be useful to
>> document
>> > > >> that.
>> > > >> > > > > > > > > >> >
>> > > >> > > > > > > > > >> > Thanks,
>> > > >> > > > > > > > > >> >
>> > > >> > > > > > > > > >> > Jun
>> > > >> > > > > > > > > >> >
>> > > >> > > > > > > > > >> > On Tue, Oct 3, 2023 at 4:55 PM Artem Livshits
>> > > >> > > > > > > > > >> > <al...@confluent.io.invalid> wrote:
>> > > >> > > > > > > > > >> >
>> > > >> > > > > > > > > >> > > Hi Colin,
>> > > >> > > > > > > > > >> > >
>> > > >> > > > > > > > > >> > > I think in your example
>> "do_unclean_recovery"
>> > > would
>> > > >> > need
>> > > >> > > > to
>> > > >> > > > > do
>> > > >> > > > > > > > > >> different
>> > > >> > > > > > > > > >> > > things depending on the strategy.
>> > > >> > > > > > > > > >> > >
>> > > >> > > > > > > > > >> > > do_unclean_recovery() {
>> > > >> > > > > > > > > >> > >    if (unclean.recovery.manager.enabled) {
>> > > >> > > > > > > > > >> > >     if (strategy == Aggressive)
>> > > >> > > > > > > > > >> > >       use
>> > > >> > UncleanRecoveryManager(waitLastKnownERL=false)
>> > > >> > > > //
>> > > >> > > > > > > just
>> > > >> > > > > > > > > >> inspect
>> > > >> > > > > > > > > >> > > logs from whoever is available
>> > > >> > > > > > > > > >> > >     else
>> > > >> > > > > > > > > >> > >       use
>> > > >> > UncleanRecoveryManager(waitLastKnownERL=true)
>> > > >> > > > //
>> > > >> > > > > > > must
>> > > >> > > > > > > > > wait
>> > > >> > > > > > > > > >> > for
>> > > >> > > > > > > > > >> > > at least last known ELR
>> > > >> > > > > > > > > >> > >   } else {
>> > > >> > > > > > > > > >> > >     if (strategy == Aggressive)
>> > > >> > > > > > > > > >> > >       choose the last known leader if that
>> is
>> > > >> > available,
>> > > >> > > > or
>> > > >> > > > > a
>> > > >> > > > > > > > random
>> > > >> > > > > > > > > >> > leader
>> > > >> > > > > > > > > >> > > if not)
>> > > >> > > > > > > > > >> > >     else
>> > > >> > > > > > > > > >> > >       wait for last known leader to get back
>> > > >> > > > > > > > > >> > >   }
>> > > >> > > > > > > > > >> > > }
>> > > >> > > > > > > > > >> > >
>> > > >> > > > > > > > > >> > > The idea is that the Aggressive strategy
>> would
>> > > >> kick in
>> > > >> > > as
>> > > >> > > > > soon
>> > > >> > > > > > > as
>> > > >> > > > > > > > we
>> > > >> > > > > > > > > >> lost
>> > > >> > > > > > > > > >> > > the leader and would pick a leader from
>> whoever
>> > > is
>> > > >> > > > > available;
>> > > >> > > > > > > but
>> > > >> > > > > > > > > the
>> > > >> > > > > > > > > >> > > Balanced will only kick in when ELR is empty
>> > and
>> > > >> will
>> > > >> > > wait
>> > > >> > > > > for
>> > > >> > > > > > > the
>> > > >> > > > > > > > > >> > brokers
>> > > >> > > > > > > > > >> > > that likely have most data to be available.
>> > > >> > > > > > > > > >> > >
>> > > >> > > > > > > > > >> > > On Tue, Oct 3, 2023 at 3:04 PM Colin McCabe
>> <
>> > > >> > > > > > cmccabe@apache.org
>> > > >> > > > > > > >
>> > > >> > > > > > > > > >> wrote:
>> > > >> > > > > > > > > >> > >
>> > > >> > > > > > > > > >> > > > On Tue, Oct 3, 2023, at 10:49, Jun Rao
>> wrote:
>> > > >> > > > > > > > > >> > > > > Hi, Calvin,
>> > > >> > > > > > > > > >> > > > >
>> > > >> > > > > > > > > >> > > > > Thanks for the update KIP. A few more
>> > > comments.
>> > > >> > > > > > > > > >> > > > >
>> > > >> > > > > > > > > >> > > > > 41. Why would a user choose the option
>> to
>> > > >> select a
>> > > >> > > > > random
>> > > >> > > > > > > > > replica
>> > > >> > > > > > > > > >> as
>> > > >> > > > > > > > > >> > > the
>> > > >> > > > > > > > > >> > > > > leader instead of using
>> > > >> > > > > > unclean.recovery.strateg=Aggressive?
>> > > >> > > > > > > > It
>> > > >> > > > > > > > > >> seems
>> > > >> > > > > > > > > >> > > > that
>> > > >> > > > > > > > > >> > > > > the latter is strictly better? If that's
>> > not
>> > > >> the
>> > > >> > > case,
>> > > >> > > > > > could
>> > > >> > > > > > > > we
>> > > >> > > > > > > > > >> fold
>> > > >> > > > > > > > > >> > > this
>> > > >> > > > > > > > > >> > > > > option under unclean.recovery.strategy
>> > > instead
>> > > >> of
>> > > >> > > > > > > introducing
>> > > >> > > > > > > > a
>> > > >> > > > > > > > > >> > > separate
>> > > >> > > > > > > > > >> > > > > config?
>> > > >> > > > > > > > > >> > > >
>> > > >> > > > > > > > > >> > > > Hi Jun,
>> > > >> > > > > > > > > >> > > >
>> > > >> > > > > > > > > >> > > > I thought the flow of control was:
>> > > >> > > > > > > > > >> > > >
>> > > >> > > > > > > > > >> > > > If there is no leader for the partition {
>> > > >> > > > > > > > > >> > > >   If (there are unfenced ELR members) {
>> > > >> > > > > > > > > >> > > >     choose_an_unfenced_ELR_member
>> > > >> > > > > > > > > >> > > >   } else if (there are fenced ELR members
>> AND
>> > > >> > > > > > > > > strategy=Aggressive) {
>> > > >> > > > > > > > > >> > > >     do_unclean_recovery
>> > > >> > > > > > > > > >> > > >   } else if (there are no ELR members AND
>> > > >> strategy
>> > > >> > !=
>> > > >> > > > > None)
>> > > >> > > > > > {
>> > > >> > > > > > > > > >> > > >     do_unclean_recovery
>> > > >> > > > > > > > > >> > > >   } else {
>> > > >> > > > > > > > > >> > > >     do nothing about the missing leader
>> > > >> > > > > > > > > >> > > >   }
>> > > >> > > > > > > > > >> > > > }
>> > > >> > > > > > > > > >> > > >
>> > > >> > > > > > > > > >> > > > do_unclean_recovery() {
>> > > >> > > > > > > > > >> > > >    if (unclean.recovery.manager.enabled) {
>> > > >> > > > > > > > > >> > > >     use UncleanRecoveryManager
>> > > >> > > > > > > > > >> > > >   } else {
>> > > >> > > > > > > > > >> > > >     choose the last known leader if that
>> is
>> > > >> > available,
>> > > >> > > > or
>> > > >> > > > > a
>> > > >> > > > > > > > random
>> > > >> > > > > > > > > >> > leader
>> > > >> > > > > > > > > >> > > > if not)
>> > > >> > > > > > > > > >> > > >   }
>> > > >> > > > > > > > > >> > > > }
>> > > >> > > > > > > > > >> > > >
>> > > >> > > > > > > > > >> > > > However, I think this could be clarified,
>> > > >> especially
>> > > >> > > the
>> > > >> > > > > > > > behavior
>> > > >> > > > > > > > > >> when
>> > > >> > > > > > > > > >> > > > unclean.recovery.manager.enabled=false.
>> > > >> Inuitively
>> > > >> > the
>> > > >> > > > > goal
>> > > >> > > > > > > for
>> > > >> > > > > > > > > >> > > > unclean.recovery.manager.enabled=false is
>> to
>> > be
>> > > >> "the
>> > > >> > > > same
>> > > >> > > > > as
>> > > >> > > > > > > > now,
>> > > >> > > > > > > > > >> > mostly"
>> > > >> > > > > > > > > >> > > > but it's very underspecified in the KIP, I
>> > > agree.
>> > > >> > > > > > > > > >> > > >
>> > > >> > > > > > > > > >> > > > >
>> > > >> > > > > > > > > >> > > > > 50. ElectLeadersRequest: "If more than
>> 20
>> > > >> topics
>> > > >> > are
>> > > >> > > > > > > included,
>> > > >> > > > > > > > > >> only
>> > > >> > > > > > > > > >> > the
>> > > >> > > > > > > > > >> > > > > first 20 will be served. Others will be
>> > > >> returned
>> > > >> > > with
>> > > >> > > > > > > > > >> > DesiredLeaders."
>> > > >> > > > > > > > > >> > > > Hmm,
>> > > >> > > > > > > > > >> > > > > not sure that I understand this.
>> > > >> > > ElectLeadersResponse
>> > > >> > > > > > > doesn't
>> > > >> > > > > > > > > >> have a
>> > > >> > > > > > > > > >> > > > > DesiredLeaders field.
>> > > >> > > > > > > > > >> > > > >
>> > > >> > > > > > > > > >> > > > > 51. GetReplicaLogInfo: "If more than
>> 2000
>> > > >> > partitions
>> > > >> > > > are
>> > > >> > > > > > > > > included,
>> > > >> > > > > > > > > >> > only
>> > > >> > > > > > > > > >> > > > the
>> > > >> > > > > > > > > >> > > > > first 2000 will be served" Do we return
>> an
>> > > >> error
>> > > >> > for
>> > > >> > > > the
>> > > >> > > > > > > > > remaining
>> > > >> > > > > > > > > >> > > > > partitions? Actually, should we include
>> an
>> > > >> > errorCode
>> > > >> > > > > field
>> > > >> > > > > > > at
>> > > >> > > > > > > > > the
>> > > >> > > > > > > > > >> > > > partition
>> > > >> > > > > > > > > >> > > > > level in GetReplicaLogInfoResponse to
>> cover
>> > > >> > > > non-existing
>> > > >> > > > > > > > > >> partitions
>> > > >> > > > > > > > > >> > and
>> > > >> > > > > > > > > >> > > > no
>> > > >> > > > > > > > > >> > > > > authorization, etc?
>> > > >> > > > > > > > > >> > > > >
>> > > >> > > > > > > > > >> > > > > 52. The entry should matches => The
>> entry
>> > > >> should
>> > > >> > > match
>> > > >> > > > > > > > > >> > > > >
>> > > >> > > > > > > > > >> > > > > 53. ElectLeadersRequest.DesiredLeaders:
>> > > Should
>> > > >> it
>> > > >> > be
>> > > >> > > > > > > nullable
>> > > >> > > > > > > > > >> since a
>> > > >> > > > > > > > > >> > > > user
>> > > >> > > > > > > > > >> > > > > may not specify DesiredLeaders?
>> > > >> > > > > > > > > >> > > > >
>> > > >> > > > > > > > > >> > > > > 54. Downgrade: Is that indeed possible?
>> I
>> > > >> thought
>> > > >> > > > > earlier
>> > > >> > > > > > > you
>> > > >> > > > > > > > > said
>> > > >> > > > > > > > > >> > that
>> > > >> > > > > > > > > >> > > > > once the new version of the records are
>> in
>> > > the
>> > > >> > > > metadata
>> > > >> > > > > > log,
>> > > >> > > > > > > > one
>> > > >> > > > > > > > > >> > can't
>> > > >> > > > > > > > > >> > > > > downgrade since the old broker doesn't
>> know
>> > > >> how to
>> > > >> > > > parse
>> > > >> > > > > > the
>> > > >> > > > > > > > new
>> > > >> > > > > > > > > >> > > version
>> > > >> > > > > > > > > >> > > > of
>> > > >> > > > > > > > > >> > > > > the metadata records?
>> > > >> > > > > > > > > >> > > > >
>> > > >> > > > > > > > > >> > > >
>> > > >> > > > > > > > > >> > > > MetadataVersion downgrade is currently
>> broken
>> > > >> but we
>> > > >> > > > have
>> > > >> > > > > > > fixing
>> > > >> > > > > > > > > it
>> > > >> > > > > > > > > >> on
>> > > >> > > > > > > > > >> > > our
>> > > >> > > > > > > > > >> > > > plate for Kafka 3.7.
>> > > >> > > > > > > > > >> > > >
>> > > >> > > > > > > > > >> > > > The way downgrade works is that "new
>> > features"
>> > > >> are
>> > > >> > > > > dropped,
>> > > >> > > > > > > > > leaving
>> > > >> > > > > > > > > >> > only
>> > > >> > > > > > > > > >> > > > the old ones.
>> > > >> > > > > > > > > >> > > >
>> > > >> > > > > > > > > >> > > > > 55. CleanShutdownFile: Should we add a
>> > > version
>> > > >> > field
>> > > >> > > > for
>> > > >> > > > > > > > future
>> > > >> > > > > > > > > >> > > > extension?
>> > > >> > > > > > > > > >> > > > >
>> > > >> > > > > > > > > >> > > > > 56. Config changes are public facing.
>> Could
>> > > we
>> > > >> > have
>> > > >> > > a
>> > > >> > > > > > > separate
>> > > >> > > > > > > > > >> > section
>> > > >> > > > > > > > > >> > > to
>> > > >> > > > > > > > > >> > > > > document all the config changes?
>> > > >> > > > > > > > > >> > > >
>> > > >> > > > > > > > > >> > > > +1. A separate section for this would be
>> > good.
>> > > >> > > > > > > > > >> > > >
>> > > >> > > > > > > > > >> > > > best,
>> > > >> > > > > > > > > >> > > > Colin
>> > > >> > > > > > > > > >> > > >
>> > > >> > > > > > > > > >> > > > >
>> > > >> > > > > > > > > >> > > > > Thanks,
>> > > >> > > > > > > > > >> > > > >
>> > > >> > > > > > > > > >> > > > > Jun
>> > > >> > > > > > > > > >> > > > >
>> > > >> > > > > > > > > >> > > > > On Mon, Sep 25, 2023 at 4:29 PM Calvin
>> Liu
>> > > >> > > > > > > > > >> > <caliu@confluent.io.invalid
>> > > >> > > > > > > > > >> > > >
>> > > >> > > > > > > > > >> > > > > wrote:
>> > > >> > > > > > > > > >> > > > >
>> > > >> > > > > > > > > >> > > > >> Hi Jun
>> > > >> > > > > > > > > >> > > > >> Thanks for the comments.
>> > > >> > > > > > > > > >> > > > >>
>> > > >> > > > > > > > > >> > > > >> 40. If we change to None, it is not
>> > > guaranteed
>> > > >> > for
>> > > >> > > no
>> > > >> > > > > > data
>> > > >> > > > > > > > > loss.
>> > > >> > > > > > > > > >> For
>> > > >> > > > > > > > > >> > > > users
>> > > >> > > > > > > > > >> > > > >> who are not able to validate the data
>> with
>> > > >> > external
>> > > >> > > > > > > > resources,
>> > > >> > > > > > > > > >> > manual
>> > > >> > > > > > > > > >> > > > >> intervention does not give a better
>> result
>> > > >> but a
>> > > >> > > loss
>> > > >> > > > > of
>> > > >> > > > > > > > > >> > availability.
>> > > >> > > > > > > > > >> > > > So
>> > > >> > > > > > > > > >> > > > >> practically speaking, the Balance mode
>> > would
>> > > >> be a
>> > > >> > > > > better
>> > > >> > > > > > > > > default
>> > > >> > > > > > > > > >> > > value.
>> > > >> > > > > > > > > >> > > > >>
>> > > >> > > > > > > > > >> > > > >> 41. No, it represents how we want to do
>> > the
>> > > >> > unclean
>> > > >> > > > > > leader
>> > > >> > > > > > > > > >> election.
>> > > >> > > > > > > > > >> > > If
>> > > >> > > > > > > > > >> > > > it
>> > > >> > > > > > > > > >> > > > >> is false, the unclean leader election
>> will
>> > > be
>> > > >> the
>> > > >> > > old
>> > > >> > > > > > > random
>> > > >> > > > > > > > > way.
>> > > >> > > > > > > > > >> > > > >> Otherwise, the unclean recovery will be
>> > > used.
>> > > >> > > > > > > > > >> > > > >>
>> > > >> > > > > > > > > >> > > > >> 42. Good catch. Updated.
>> > > >> > > > > > > > > >> > > > >>
>> > > >> > > > > > > > > >> > > > >> 43. Only the first 20 topics will be
>> > served.
>> > > >> > Others
>> > > >> > > > > will
>> > > >> > > > > > be
>> > > >> > > > > > > > > >> returned
>> > > >> > > > > > > > > >> > > > with
>> > > >> > > > > > > > > >> > > > >> InvalidRequestError
>> > > >> > > > > > > > > >> > > > >>
>> > > >> > > > > > > > > >> > > > >> 44. The order matters. The desired
>> leader
>> > > >> entries
>> > > >> > > > match
>> > > >> > > > > > > with
>> > > >> > > > > > > > > the
>> > > >> > > > > > > > > >> > topic
>> > > >> > > > > > > > > >> > > > >> partition list by the index.
>> > > >> > > > > > > > > >> > > > >>
>> > > >> > > > > > > > > >> > > > >> 45. Thanks! Updated.
>> > > >> > > > > > > > > >> > > > >>
>> > > >> > > > > > > > > >> > > > >> 46. Good advice! Updated.
>> > > >> > > > > > > > > >> > > > >>
>> > > >> > > > > > > > > >> > > > >> 47.1, updated the comment. Basically it
>> > will
>> > > >> > elect
>> > > >> > > > the
>> > > >> > > > > > > > replica
>> > > >> > > > > > > > > in
>> > > >> > > > > > > > > >> > the
>> > > >> > > > > > > > > >> > > > >> desiredLeader field to be the leader
>> > > >> > > > > > > > > >> > > > >>
>> > > >> > > > > > > > > >> > > > >> 47.2 We can let the admin client do the
>> > > >> > conversion.
>> > > >> > > > > Using
>> > > >> > > > > > > the
>> > > >> > > > > > > > > >> > > > desiredLeader
>> > > >> > > > > > > > > >> > > > >> field in the json format seems easier
>> for
>> > > >> users.
>> > > >> > > > > > > > > >> > > > >>
>> > > >> > > > > > > > > >> > > > >> 48. Once the MV version is downgraded,
>> all
>> > > the
>> > > >> > ELR
>> > > >> > > > > > related
>> > > >> > > > > > > > > fields
>> > > >> > > > > > > > > >> > will
>> > > >> > > > > > > > > >> > > > be
>> > > >> > > > > > > > > >> > > > >> removed on the next partition change.
>> The
>> > > >> > > controller
>> > > >> > > > > will
>> > > >> > > > > > > > also
>> > > >> > > > > > > > > >> > ignore
>> > > >> > > > > > > > > >> > > > the
>> > > >> > > > > > > > > >> > > > >> ELR fields. Updated the KIP.
>> > > >> > > > > > > > > >> > > > >>
>> > > >> > > > > > > > > >> > > > >> 49. Yes, it would be
>> deprecated/removed.
>> > > >> > > > > > > > > >> > > > >>
>> > > >> > > > > > > > > >> > > > >>
>> > > >> > > > > > > > > >> > > > >> On Mon, Sep 25, 2023 at 3:49 PM Jun Rao
>> > > >> > > > > > > > > <jun@confluent.io.invalid
>> > > >> > > > > > > > > >> >
>> > > >> > > > > > > > > >> > > > wrote:
>> > > >> > > > > > > > > >> > > > >>
>> > > >> > > > > > > > > >> > > > >> > Hi, Calvin,
>> > > >> > > > > > > > > >> > > > >> >
>> > > >> > > > > > > > > >> > > > >> > Thanks for the updated KIP. Made
>> another
>> > > >> pass.
>> > > >> > A
>> > > >> > > > few
>> > > >> > > > > > more
>> > > >> > > > > > > > > >> comments
>> > > >> > > > > > > > > >> > > > below.
>> > > >> > > > > > > > > >> > > > >> >
>> > > >> > > > > > > > > >> > > > >> > 40.
>> unclean.leader.election.enable.false
>> > > ->
>> > > >> > > > > > > > > >> > > > >> > unclean.recovery.strategy.Balanced:
>> The
>> > > >> > Balanced
>> > > >> > > > mode
>> > > >> > > > > > > could
>> > > >> > > > > > > > > >> still
>> > > >> > > > > > > > > >> > > > lead to
>> > > >> > > > > > > > > >> > > > >> > data loss. So, I am wondering if
>> > > >> > > > > > > > > >> > > unclean.leader.election.enable.false
>> > > >> > > > > > > > > >> > > > >> > should map to None?
>> > > >> > > > > > > > > >> > > > >> >
>> > > >> > > > > > > > > >> > > > >> > 41.
>> unclean.recovery.manager.enabled: I
>> > am
>> > > >> not
>> > > >> > > sure
>> > > >> > > > > why
>> > > >> > > > > > > we
>> > > >> > > > > > > > > >> > introduce
>> > > >> > > > > > > > > >> > > > this
>> > > >> > > > > > > > > >> > > > >> > additional config. Is it the same as
>> > > >> > > > > > > > > >> > unclean.recovery.strategy=None?
>> > > >> > > > > > > > > >> > > > >> >
>> > > >> > > > > > > > > >> > > > >> > 42.
>> > > >> > > > DescribeTopicResponse.TopicAuthorizedOperations:
>> > > >> > > > > > > Should
>> > > >> > > > > > > > > >> this
>> > > >> > > > > > > > > >> > be
>> > > >> > > > > > > > > >> > > at
>> > > >> > > > > > > > > >> > > > >> the
>> > > >> > > > > > > > > >> > > > >> > topic level?
>> > > >> > > > > > > > > >> > > > >> >
>> > > >> > > > > > > > > >> > > > >> > 43. "Limit: 20 topics max per
>> request":
>> > > >> Could
>> > > >> > we
>> > > >> > > > > > describe
>> > > >> > > > > > > > > what
>> > > >> > > > > > > > > >> > > > happens if
>> > > >> > > > > > > > > >> > > > >> > the request includes more than 20
>> > topics?
>> > > >> > > > > > > > > >> > > > >> >
>> > > >> > > > > > > > > >> > > > >> > 44.
>> ElectLeadersRequest.DesiredLeaders:
>> > > >> Could
>> > > >> > we
>> > > >> > > > > > describe
>> > > >> > > > > > > > > >> whether
>> > > >> > > > > > > > > >> > > the
>> > > >> > > > > > > > > >> > > > >> > ordering matters?
>> > > >> > > > > > > > > >> > > > >> >
>> > > >> > > > > > > > > >> > > > >> > 45.
>> GetReplicaLogInfo.TopicPartitions:
>> > > >> "about":
>> > > >> > > > "The
>> > > >> > > > > > > topic
>> > > >> > > > > > > > > >> > > partitions
>> > > >> > > > > > > > > >> > > > to
>> > > >> > > > > > > > > >> > > > >> > elect leaders.": The description in
>> > > "about"
>> > > >> is
>> > > >> > > > > > incorrect.
>> > > >> > > > > > > > > >> > > > >> >
>> > > >> > > > > > > > > >> > > > >> > 46. GetReplicaLogInfoResponse:
>> Should we
>> > > >> nest
>> > > >> > > > > > partitions
>> > > >> > > > > > > > > under
>> > > >> > > > > > > > > >> > > > topicId to
>> > > >> > > > > > > > > >> > > > >> > be consistent with other types of
>> > > responses?
>> > > >> > > > > > > > > >> > > > >> >
>> > > >> > > > > > > > > >> > > > >> > 47. kafka-leader-election.sh:
>> > > >> > > > > > > > > >> > > > >> > 47.1 Could we explain DESIGNATION?
>> > > >> > > > > > > > > >> > > > >> > 47.2 desiredLeader: Should it be a
>> list
>> > to
>> > > >> > match
>> > > >> > > > the
>> > > >> > > > > > > field
>> > > >> > > > > > > > in
>> > > >> > > > > > > > > >> > > > >> > ElectLeadersRequest?
>> > > >> > > > > > > > > >> > > > >> >
>> > > >> > > > > > > > > >> > > > >> > 48. We could add a section on
>> downgrade?
>> > > >> > > > > > > > > >> > > > >> >
>> > > >> > > > > > > > > >> > > > >> > 49. LastKnownLeader: This seems only
>> > > needed
>> > > >> in
>> > > >> > > the
>> > > >> > > > > > first
>> > > >> > > > > > > > > phase
>> > > >> > > > > > > > > >> of
>> > > >> > > > > > > > > >> > > > >> > delivering ELR. Will it be removed
>> when
>> > > the
>> > > >> > > > complete
>> > > >> > > > > > KIP
>> > > >> > > > > > > is
>> > > >> > > > > > > > > >> > > delivered?
>> > > >> > > > > > > > > >> > > > >> >
>> > > >> > > > > > > > > >> > > > >> > Thanks,
>> > > >> > > > > > > > > >> > > > >> >
>> > > >> > > > > > > > > >> > > > >> > Jun
>> > > >> > > > > > > > > >> > > > >> >
>> > > >> > > > > > > > > >> > > > >> > On Tue, Sep 19, 2023 at 1:30 PM Colin
>> > > >> McCabe <
>> > > >> > > > > > > > > >> cmccabe@apache.org>
>> > > >> > > > > > > > > >> > > > wrote:
>> > > >> > > > > > > > > >> > > > >> >
>> > > >> > > > > > > > > >> > > > >> > > Hi Calvin,
>> > > >> > > > > > > > > >> > > > >> > >
>> > > >> > > > > > > > > >> > > > >> > > Thanks for the explanations. I like
>> > the
>> > > >> idea
>> > > >> > of
>> > > >> > > > > using
>> > > >> > > > > > > > none,
>> > > >> > > > > > > > > >> > > > balanced,
>> > > >> > > > > > > > > >> > > > >> > > aggressive. We also had an offline
>> > > >> discussion
>> > > >> > > > about
>> > > >> > > > > > why
>> > > >> > > > > > > > it
>> > > >> > > > > > > > > is
>> > > >> > > > > > > > > >> > good
>> > > >> > > > > > > > > >> > > > to
>> > > >> > > > > > > > > >> > > > >> > use a
>> > > >> > > > > > > > > >> > > > >> > > new config key (basically, so that
>> we
>> > > can
>> > > >> > > > deprecate
>> > > >> > > > > > the
>> > > >> > > > > > > > old
>> > > >> > > > > > > > > >> one
>> > > >> > > > > > > > > >> > > > which
>> > > >> > > > > > > > > >> > > > >> had
>> > > >> > > > > > > > > >> > > > >> > > only false/true values in 4.0) With
>> > > these
>> > > >> > > > changes,
>> > > >> > > > > I
>> > > >> > > > > > am
>> > > >> > > > > > > > +1.
>> > > >> > > > > > > > > >> > > > >> > >
>> > > >> > > > > > > > > >> > > > >> > > best,
>> > > >> > > > > > > > > >> > > > >> > > Colin
>> > > >> > > > > > > > > >> > > > >> > >
>> > > >> > > > > > > > > >> > > > >> > > On Mon, Sep 18, 2023, at 15:54,
>> Calvin
>> > > Liu
>> > > >> > > wrote:
>> > > >> > > > > > > > > >> > > > >> > > > Hi Colin,
>> > > >> > > > > > > > > >> > > > >> > > > Also, can we deprecate
>> > > >> > > > > > unclean.leader.election.enable
>> > > >> > > > > > > > in
>> > > >> > > > > > > > > >> 4.0?
>> > > >> > > > > > > > > >> > > > Before
>> > > >> > > > > > > > > >> > > > >> > > that,
>> > > >> > > > > > > > > >> > > > >> > > > we can have both the config
>> > > >> > > > > > unclean.recovery.strategy
>> > > >> > > > > > > > and
>> > > >> > > > > > > > > >> > > > >> > > > unclean.leader.election.enable
>> > > >> > > > > > > > > >> > > > >> > > > and using the
>> > unclean.recovery.Enabled
>> > > >> to
>> > > >> > > > > determine
>> > > >> > > > > > > > which
>> > > >> > > > > > > > > >> > config
>> > > >> > > > > > > > > >> > > > to
>> > > >> > > > > > > > > >> > > > >> use
>> > > >> > > > > > > > > >> > > > >> > > > during the unclean leader
>> election.
>> > > >> > > > > > > > > >> > > > >> > > >
>> > > >> > > > > > > > > >> > > > >> > > > On Mon, Sep 18, 2023 at 3:51 PM
>> > Calvin
>> > > >> Liu
>> > > >> > <
>> > > >> > > > > > > > > >> > caliu@confluent.io>
>> > > >> > > > > > > > > >> > > > >> wrote:
>> > > >> > > > > > > > > >> > > > >> > > >
>> > > >> > > > > > > > > >> > > > >> > > >> Hi Colin,
>> > > >> > > > > > > > > >> > > > >> > > >> For the
>> unclean.recovery.strategy
>> > > >> config
>> > > >> > > name,
>> > > >> > > > > how
>> > > >> > > > > > > > about
>> > > >> > > > > > > > > >> we
>> > > >> > > > > > > > > >> > use
>> > > >> > > > > > > > > >> > > > the
>> > > >> > > > > > > > > >> > > > >> > > >> following
>> > > >> > > > > > > > > >> > > > >> > > >> None. It basically means no
>> unclean
>> > > >> > recovery
>> > > >> > > > > will
>> > > >> > > > > > be
>> > > >> > > > > > > > > >> > performed.
>> > > >> > > > > > > > > >> > > > >> > > >> Aggressive. It means
>> availability
>> > > goes
>> > > >> > > first.
>> > > >> > > > > > > Whenever
>> > > >> > > > > > > > > the
>> > > >> > > > > > > > > >> > > > partition
>> > > >> > > > > > > > > >> > > > >> > > can't
>> > > >> > > > > > > > > >> > > > >> > > >> elect a durable replica, the
>> > > controller
>> > > >> > will
>> > > >> > > > try
>> > > >> > > > > > the
>> > > >> > > > > > > > > >> unclean
>> > > >> > > > > > > > > >> > > > >> recovery.
>> > > >> > > > > > > > > >> > > > >> > > >> Balanced. It is the balance
>> point
>> > of
>> > > >> the
>> > > >> > > > > > > availability
>> > > >> > > > > > > > > >> > > > >> > first(Aggressive)
>> > > >> > > > > > > > > >> > > > >> > > >> and least availability(None).
>> The
>> > > >> > controller
>> > > >> > > > > > > performs
>> > > >> > > > > > > > > >> unclean
>> > > >> > > > > > > > > >> > > > >> recovery
>> > > >> > > > > > > > > >> > > > >> > > when
>> > > >> > > > > > > > > >> > > > >> > > >> both ISR and ELR are empty.
>> > > >> > > > > > > > > >> > > > >> > > >>
>> > > >> > > > > > > > > >> > > > >> > > >>
>> > > >> > > > > > > > > >> > > > >> > > >> On Fri, Sep 15, 2023 at 11:42 AM
>> > > Calvin
>> > > >> > Liu
>> > > >> > > <
>> > > >> > > > > > > > > >> > > caliu@confluent.io>
>> > > >> > > > > > > > > >> > > > >> > wrote:
>> > > >> > > > > > > > > >> > > > >> > > >>
>> > > >> > > > > > > > > >> > > > >> > > >>> Hi Colin,
>> > > >> > > > > > > > > >> > > > >> > > >>>
>> > > >> > > > > > > > > >> > > > >> > > >>> > So, the proposal is that if
>> > > someone
>> > > >> > sets
>> > > >> > > > > > > > > >> > > > >> > > "unclean.leader.election.enable
>> > > >> > > > > > > > > >> > > > >> > > >>> = true"...
>> > > >> > > > > > > > > >> > > > >> > > >>>
>> > > >> > > > > > > > > >> > > > >> > > >>>
>> > > >> > > > > > > > > >> > > > >> > > >>> The idea is to use one of the
>> > > >> > > > > > > > > >> unclean.leader.election.enable
>> > > >> > > > > > > > > >> > > and
>> > > >> > > > > > > > > >> > > > >> > > >>> unclean.recovery.strategy
>> based on
>> > > the
>> > > >> > > > > > > > > >> > > > unclean.recovery.Enabled. A
>> > > >> > > > > > > > > >> > > > >> > > possible
>> > > >> > > > > > > > > >> > > > >> > > >>> version can be
>> > > >> > > > > > > > > >> > > > >> > > >>>
>> > > >> > > > > > > > > >> > > > >> > > >>> If unclean.recovery.Enabled:
>> > > >> > > > > > > > > >> > > > >> > > >>>
>> > > >> > > > > > > > > >> > > > >> > > >>> {
>> > > >> > > > > > > > > >> > > > >> > > >>>
>> > > >> > > > > > > > > >> > > > >> > > >>> Check
>> unclean.recovery.strategy.
>> > If
>> > > >> set,
>> > > >> > > use
>> > > >> > > > > it.
>> > > >> > > > > > > > > >> Otherwise,
>> > > >> > > > > > > > > >> > > > check
>> > > >> > > > > > > > > >> > > > >> > > >>> unclean.leader.election.enable
>> and
>> > > >> > > translate
>> > > >> > > > it
>> > > >> > > > > > to
>> > > >> > > > > > > > > >> > > > >> > > >>> unclean.recovery.strategy.
>> > > >> > > > > > > > > >> > > > >> > > >>>
>> > > >> > > > > > > > > >> > > > >> > > >>> } else {
>> > > >> > > > > > > > > >> > > > >> > > >>>
>> > > >> > > > > > > > > >> > > > >> > > >>> Use
>> unclean.leader.election.enable
>> > > >> > > > > > > > > >> > > > >> > > >>>
>> > > >> > > > > > > > > >> > > > >> > > >>> }
>> > > >> > > > > > > > > >> > > > >> > > >>>
>> > > >> > > > > > > > > >> > > > >> > > >>>
>> > > >> > > > > > > > > >> > > > >> > > >>> —--------
>> > > >> > > > > > > > > >> > > > >> > > >>>
>> > > >> > > > > > > > > >> > > > >> > > >>> >The configuration key should
>> be
>> > > >> > > > > > > > > >> > > > >> "unclean.recovery.manager.enabled",
>> > > >> > > > > > > > > >> > > > >> > > >>> right?
>> > > >> > > > > > > > > >> > > > >> > > >>>
>> > > >> > > > > > > > > >> > > > >> > > >>>
>> > > >> > > > > > > > > >> > > > >> > > >>> I think we have two ways of
>> > > choosing a
>> > > >> > > leader
>> > > >> > > > > > > > > uncleanly,
>> > > >> > > > > > > > > >> > > unclean
>> > > >> > > > > > > > > >> > > > >> > leader
>> > > >> > > > > > > > > >> > > > >> > > >>> election and unclean
>> recovery(log
>> > > >> > > inspection)
>> > > >> > > > > and
>> > > >> > > > > > > we
>> > > >> > > > > > > > > try
>> > > >> > > > > > > > > >> to
>> > > >> > > > > > > > > >> > > > switch
>> > > >> > > > > > > > > >> > > > >> > > between
>> > > >> > > > > > > > > >> > > > >> > > >>> them.
>> > > >> > > > > > > > > >> > > > >> > > >>>
>> > > >> > > > > > > > > >> > > > >> > > >>> Do you mean we want to develop
>> two
>> > > >> ways
>> > > >> > of
>> > > >> > > > > > > performing
>> > > >> > > > > > > > > the
>> > > >> > > > > > > > > >> > > > unclean
>> > > >> > > > > > > > > >> > > > >> > > >>> recovery and one of them is
>> using
>> > > >> > “unclean
>> > > >> > > > > > recovery
>> > > >> > > > > > > > > >> > manager”?
>> > > >> > > > > > > > > >> > > I
>> > > >> > > > > > > > > >> > > > >> guess
>> > > >> > > > > > > > > >> > > > >> > > we
>> > > >> > > > > > > > > >> > > > >> > > >>> haven’t discussed the second
>> way.
>> > > >> > > > > > > > > >> > > > >> > > >>>
>> > > >> > > > > > > > > >> > > > >> > > >>>
>> > > >> > > > > > > > > >> > > > >> > > >>> —-------
>> > > >> > > > > > > > > >> > > > >> > > >>>
>> > > >> > > > > > > > > >> > > > >> > > >>> >How do these 4 levels of
>> > overrides
>> > > >> > > interact
>> > > >> > > > > with
>> > > >> > > > > > > > your
>> > > >> > > > > > > > > >> new
>> > > >> > > > > > > > > >> > > > >> > > >>> configurations?
>> > > >> > > > > > > > > >> > > > >> > > >>>
>> > > >> > > > > > > > > >> > > > >> > > >>>
>> > > >> > > > > > > > > >> > > > >> > > >>> I do notice in the Kraft
>> > controller
>> > > >> code,
>> > > >> > > the
>> > > >> > > > > > > method
>> > > >> > > > > > > > to
>> > > >> > > > > > > > > >> > check
>> > > >> > > > > > > > > >> > > > >> whether
>> > > >> > > > > > > > > >> > > > >> > > >>> perform unclean leader
>> election is
>> > > >> hard
>> > > >> > > coded
>> > > >> > > > > to
>> > > >> > > > > > > > false
>> > > >> > > > > > > > > >> since
>> > > >> > > > > > > > > >> > > > >> > > >>>
>> > > >> > 2021(uncleanLeaderElectionEnabledForTopic).
>> > > >> > > > > Isn’t
>> > > >> > > > > > > it
>> > > >> > > > > > > > a
>> > > >> > > > > > > > > >> good
>> > > >> > > > > > > > > >> > > > chance
>> > > >> > > > > > > > > >> > > > >> to
>> > > >> > > > > > > > > >> > > > >> > > >>> completely deprecate the
>> > > >> > > > > > > > > unclean.leader.election.enable?
>> > > >> > > > > > > > > >> We
>> > > >> > > > > > > > > >> > > > don’t
>> > > >> > > > > > > > > >> > > > >> > even
>> > > >> > > > > > > > > >> > > > >> > > have
>> > > >> > > > > > > > > >> > > > >> > > >>> to worry about the config
>> > > conversion.
>> > > >> > > > > > > > > >> > > > >> > > >>>
>> > > >> > > > > > > > > >> > > > >> > > >>> On the other hand, whatever the
>> > > >> override
>> > > >> > > is,
>> > > >> > > > as
>> > > >> > > > > > > long
>> > > >> > > > > > > > as
>> > > >> > > > > > > > > >> the
>> > > >> > > > > > > > > >> > > > >> > controller
>> > > >> > > > > > > > > >> > > > >> > > >>> can have the final effective
>> > > >> > > > > > > > > >> unclean.leader.election.enable,
>> > > >> > > > > > > > > >> > > the
>> > > >> > > > > > > > > >> > > > >> > topic
>> > > >> > > > > > > > > >> > > > >> > > >>> level config
>> > > >> unclean.recovery.strategy,
>> > > >> > the
>> > > >> > > > > > cluster
>> > > >> > > > > > > > > level
>> > > >> > > > > > > > > >> > > config
>> > > >> > > > > > > > > >> > > > >> > > >>> unclean.recovery.Enabled, the
>> > > >> controller
>> > > >> > > can
>> > > >> > > > > > > > calculate
>> > > >> > > > > > > > > >> the
>> > > >> > > > > > > > > >> > > > correct
>> > > >> > > > > > > > > >> > > > >> > > methods
>> > > >> > > > > > > > > >> > > > >> > > >>> to use right?
>> > > >> > > > > > > > > >> > > > >> > > >>>
>> > > >> > > > > > > > > >> > > > >> > > >>>
>> > > >> > > > > > > > > >> > > > >> > > >>> On Fri, Sep 15, 2023 at
>> 10:02 AM
>> > > Colin
>> > > >> > > > McCabe <
>> > > >> > > > > > > > > >> > > > cmccabe@apache.org>
>> > > >> > > > > > > > > >> > > > >> > > wrote:
>> > > >> > > > > > > > > >> > > > >> > > >>>
>> > > >> > > > > > > > > >> > > > >> > > >>>> On Thu, Sep 14, 2023, at
>> 22:23,
>> > > >> Calvin
>> > > >> > Liu
>> > > >> > > > > > wrote:
>> > > >> > > > > > > > > >> > > > >> > > >>>> > Hi Colin
>> > > >> > > > > > > > > >> > > > >> > > >>>> > 1. I think using the new
>> config
>> > > >> name
>> > > >> > is
>> > > >> > > > more
>> > > >> > > > > > > > clear.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >        a. The unclean leader
>> > > >> election
>> > > >> > is
>> > > >> > > > > > > actually
>> > > >> > > > > > > > > >> removed
>> > > >> > > > > > > > > >> > > if
>> > > >> > > > > > > > > >> > > > >> > unclean
>> > > >> > > > > > > > > >> > > > >> > > >>>> > recovery is in use.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >        b. Using multiple
>> values
>> > > in
>> > > >> > > > > > > > > >> > > > >> unclean.leader.election.enable
>> > > >> > > > > > > > > >> > > > >> > is
>> > > >> > > > > > > > > >> > > > >> > > >>>> > confusing and it will be
>> more
>> > > >> > confusing
>> > > >> > > > > after
>> > > >> > > > > > > > people
>> > > >> > > > > > > > > >> > forget
>> > > >> > > > > > > > > >> > > > >> about
>> > > >> > > > > > > > > >> > > > >> > > this
>> > > >> > > > > > > > > >> > > > >> > > >>>> > discussion.
>> > > >> > > > > > > > > >> > > > >> > > >>>>
>> > > >> > > > > > > > > >> > > > >> > > >>>> Hi Calvin,
>> > > >> > > > > > > > > >> > > > >> > > >>>>
>> > > >> > > > > > > > > >> > > > >> > > >>>> So, the proposal is that if
>> > someone
>> > > >> sets
>> > > >> > > > > > > > > >> > > > >> > > "unclean.leader.election.enable
>> > > >> > > > > > > > > >> > > > >> > > >>>> = true" but then sets one of
>> your
>> > > new
>> > > >> > > > > > > > configurations,
>> > > >> > > > > > > > > >> the
>> > > >> > > > > > > > > >> > > > value of
>> > > >> > > > > > > > > >> > > > >> > > >>>>
>> unclean.leader.election.enable is
>> > > >> > ignored?
>> > > >> > > > > That
>> > > >> > > > > > > > seems
>> > > >> > > > > > > > > >> less
>> > > >> > > > > > > > > >> > > > clear
>> > > >> > > > > > > > > >> > > > >> to
>> > > >> > > > > > > > > >> > > > >> > > me, not
>> > > >> > > > > > > > > >> > > > >> > > >>>> more. Just in general, having
>> > > >> multiple
>> > > >> > > > > > > configuration
>> > > >> > > > > > > > > >> keys
>> > > >> > > > > > > > > >> > to
>> > > >> > > > > > > > > >> > > > >> control
>> > > >> > > > > > > > > >> > > > >> > > the
>> > > >> > > > > > > > > >> > > > >> > > >>>> same thing confuses users.
>> > > Basically,
>> > > >> > they
>> > > >> > > > are
>> > > >> > > > > > > > sitting
>> > > >> > > > > > > > > >> at a
>> > > >> > > > > > > > > >> > > > giant
>> > > >> > > > > > > > > >> > > > >> > > control
>> > > >> > > > > > > > > >> > > > >> > > >>>> panel, and some of the levers
>> do
>> > > >> > nothing.
>> > > >> > > > > > > > > >> > > > >> > > >>>>
>> > > >> > > > > > > > > >> > > > >> > > >>>> > 2. Sorry I forgot to
>> mention in
>> > > the
>> > > >> > > > response
>> > > >> > > > > > > that
>> > > >> > > > > > > > I
>> > > >> > > > > > > > > >> did
>> > > >> > > > > > > > > >> > add
>> > > >> > > > > > > > > >> > > > the
>> > > >> > > > > > > > > >> > > > >> > > >>>> > unclean.recovery.Enabled
>> flag.
>> > > >> > > > > > > > > >> > > > >> > > >>>>
>> > > >> > > > > > > > > >> > > > >> > > >>>> The configuration key should
>> be
>> > > >> > > > > > > > > >> > > > >> "unclean.recovery.manager.enabled",
>> > > >> > > > > > > > > >> > > > >> > > >>>> right? Becuase we can do
>> "unclean
>> > > >> > > recovery"
>> > > >> > > > > > > without
>> > > >> > > > > > > > > the
>> > > >> > > > > > > > > >> > > > manager.
>> > > >> > > > > > > > > >> > > > >> > > Disabling
>> > > >> > > > > > > > > >> > > > >> > > >>>> the manager just means we use
>> a
>> > > >> > different
>> > > >> > > > > > > mechanism
>> > > >> > > > > > > > > for
>> > > >> > > > > > > > > >> > > > recovery.
>> > > >> > > > > > > > > >> > > > >> > > >>>>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >        c. Maybe I
>> > underestimated
>> > > >> the
>> > > >> > > > > challenge
>> > > >> > > > > > > of
>> > > >> > > > > > > > > >> > replacing
>> > > >> > > > > > > > > >> > > > the
>> > > >> > > > > > > > > >> > > > >> > > >>>> config. Any
>> > > >> > > > > > > > > >> > > > >> > > >>>> > implementation problems
>> ahead?
>> > > >> > > > > > > > > >> > > > >> > > >>>>
>> > > >> > > > > > > > > >> > > > >> > > >>>> There are four levels of
>> > overrides
>> > > >> for
>> > > >> > > > > > > > > >> > > > >> > unclean.leader.election.enable.
>> > > >> > > > > > > > > >> > > > >> > > >>>>
>> > > >> > > > > > > > > >> > > > >> > > >>>> 1. static configuration for
>> node.
>> > > >> > > > > > > > > >> > > > >> > > >>>>     This goes in the
>> > configuration
>> > > >> file,
>> > > >> > > > > > typically
>> > > >> > > > > > > > > named
>> > > >> > > > > > > > > >> > > > >> > > >>>> server.properties
>> > > >> > > > > > > > > >> > > > >> > > >>>>
>> > > >> > > > > > > > > >> > > > >> > > >>>> 2. dynamic configuration for
>> node
>> > > >> > default
>> > > >> > > > > > > > > >> > > > >> > > >>>>   ConfigResource(type=BROKER,
>> > > >> name="")
>> > > >> > > > > > > > > >> > > > >> > > >>>>
>> > > >> > > > > > > > > >> > > > >> > > >>>> 3. dynamic configuration for
>> node
>> > > >> > > > > > > > > >> > > > >> > > >>>>   ConfigResource(type=BROKER,
>> > > >> > > > name=<controller
>> > > >> > > > > > > id>)
>> > > >> > > > > > > > > >> > > > >> > > >>>>
>> > > >> > > > > > > > > >> > > > >> > > >>>> 4. dynamic configuration for
>> > topic
>> > > >> > > > > > > > > >> > > > >> > > >>>>   ConfigResource(type=TOPIC,
>> > > >> > > > > name=<topic-name>)
>> > > >> > > > > > > > > >> > > > >> > > >>>>
>> > > >> > > > > > > > > >> > > > >> > > >>>> How do these 4 levels of
>> > overrides
>> > > >> > > interact
>> > > >> > > > > with
>> > > >> > > > > > > > your
>> > > >> > > > > > > > > >> new
>> > > >> > > > > > > > > >> > > > >> > > >>>> configurations? If the new
>> > > >> > configurations
>> > > >> > > > > > dominate
>> > > >> > > > > > > > > over
>> > > >> > > > > > > > > >> the
>> > > >> > > > > > > > > >> > > old
>> > > >> > > > > > > > > >> > > > >> > ones,
>> > > >> > > > > > > > > >> > > > >> > > it
>> > > >> > > > > > > > > >> > > > >> > > >>>> seems like this will get a lot
>> > more
>> > > >> > > > confusing
>> > > >> > > > > to
>> > > >> > > > > > > > > >> implement
>> > > >> > > > > > > > > >> > > (and
>> > > >> > > > > > > > > >> > > > >> also
>> > > >> > > > > > > > > >> > > > >> > > to
>> > > >> > > > > > > > > >> > > > >> > > >>>> use.)
>> > > >> > > > > > > > > >> > > > >> > > >>>>
>> > > >> > > > > > > > > >> > > > >> > > >>>> Again, I'd recommend just
>> adding
>> > > some
>> > > >> > new
>> > > >> > > > > values
>> > > >> > > > > > > to
>> > > >> > > > > > > > > >> > > > >> > > >>>>
>> unclean.leader.election.enable.
>> > > It's
>> > > >> > > simple
>> > > >> > > > > and
>> > > >> > > > > > > will
>> > > >> > > > > > > > > >> > prevent
>> > > >> > > > > > > > > >> > > > user
>> > > >> > > > > > > > > >> > > > >> > > confusion
>> > > >> > > > > > > > > >> > > > >> > > >>>> (as well as developer
>> confusion.)
>> > > >> > > > > > > > > >> > > > >> > > >>>>
>> > > >> > > > > > > > > >> > > > >> > > >>>> best,
>> > > >> > > > > > > > > >> > > > >> > > >>>> Colin
>> > > >> > > > > > > > > >> > > > >> > > >>>>
>> > > >> > > > > > > > > >> > > > >> > > >>>>
>> > > >> > > > > > > > > >> > > > >> > > >>>> > 3. About the admin client, I
>> > > >> > mentioned 3
>> > > >> > > > > > changes
>> > > >> > > > > > > > in
>> > > >> > > > > > > > > >> the
>> > > >> > > > > > > > > >> > > > client.
>> > > >> > > > > > > > > >> > > > >> > > >>>> Anything
>> > > >> > > > > > > > > >> > > > >> > > >>>> > else I missed in the KIP?
>> > > >> > > > > > > > > >> > > > >> > > >>>> >       a. The client will
>> switch
>> > > to
>> > > >> > using
>> > > >> > > > the
>> > > >> > > > > > new
>> > > >> > > > > > > > RPC
>> > > >> > > > > > > > > >> > > instead
>> > > >> > > > > > > > > >> > > > of
>> > > >> > > > > > > > > >> > > > >> > > >>>> > MetadataRequest for the
>> topics.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >       b. The
>> TopicPartitionInfo
>> > > >> used
>> > > >> > in
>> > > >> > > > > > > > > >> TopicDescription
>> > > >> > > > > > > > > >> > > > needs
>> > > >> > > > > > > > > >> > > > >> to
>> > > >> > > > > > > > > >> > > > >> > > add
>> > > >> > > > > > > > > >> > > > >> > > >>>> new
>> > > >> > > > > > > > > >> > > > >> > > >>>> > fields related to the ELR.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >       c. The outputs will
>> add
>> > the
>> > > >> ELR
>> > > >> > > > > related
>> > > >> > > > > > > > > fields.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> > On Thu, Sep 14, 2023 at
>> 9:19 PM
>> > > >> Colin
>> > > >> > > > > McCabe <
>> > > >> > > > > > > > > >> > > > >> cmccabe@apache.org>
>> > > >> > > > > > > > > >> > > > >> > > >>>> wrote:
>> > > >> > > > > > > > > >> > > > >> > > >>>> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> Hi Calvin,
>> > > >> > > > > > > > > >> > > > >> > > >>>> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> Thanks for the changes.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> 1. Earlier I commented that
>> > > >> creating
>> > > >> > > > > > > > > >> > > > >> "unclean.recovery.strategy "
>> > > >> > > > > > > > > >> > > > >> > > is
>> > > >> > > > > > > > > >> > > > >> > > >>>> not
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> necessary, and we can just
>> > reuse
>> > > >> the
>> > > >> > > > > existing
>> > > >> > > > > > > > > >> > > > >> > > >>>> >>
>> > "unclean.leader.election.enable"
>> > > >> > > > > > configuration
>> > > >> > > > > > > > key.
>> > > >> > > > > > > > > >> > Let's
>> > > >> > > > > > > > > >> > > > >> discuss
>> > > >> > > > > > > > > >> > > > >> > > >>>> that.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> 2.I also don't understand
>> why
>> > > you
>> > > >> > > didn't
>> > > >> > > > > add
>> > > >> > > > > > a
>> > > >> > > > > > > > > >> > > > configuration to
>> > > >> > > > > > > > > >> > > > >> > > >>>> enable or
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> disable the Unclean
>> Recovery
>> > > >> Manager.
>> > > >> > > > This
>> > > >> > > > > > > seems
>> > > >> > > > > > > > > >> like a
>> > > >> > > > > > > > > >> > > very
>> > > >> > > > > > > > > >> > > > >> > simple
>> > > >> > > > > > > > > >> > > > >> > > >>>> way to
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> handle the staging issue
>> which
>> > > we
>> > > >> > > > > discussed.
>> > > >> > > > > > > The
>> > > >> > > > > > > > > URM
>> > > >> > > > > > > > > >> can
>> > > >> > > > > > > > > >> > > > just
>> > > >> > > > > > > > > >> > > > >> be
>> > > >> > > > > > > > > >> > > > >> > > >>>> turned off
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> until it is production
>> ready.
>> > > >> Let's
>> > > >> > > > discuss
>> > > >> > > > > > > this.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> 3. You still need to
>> describe
>> > > the
>> > > >> > > changes
>> > > >> > > > > to
>> > > >> > > > > > > > > >> AdminClient
>> > > >> > > > > > > > > >> > > > that
>> > > >> > > > > > > > > >> > > > >> are
>> > > >> > > > > > > > > >> > > > >> > > >>>> needed
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> to use
>> DescribeTopicRequest.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> Keep at it. It's looking
>> > better.
>> > > >> :)
>> > > >> > > > > > > > > >> > > > >> > > >>>> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> best,
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> Colin
>> > > >> > > > > > > > > >> > > > >> > > >>>> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> On Thu, Sep 14, 2023, at
>> > 11:03,
>> > > >> > Calvin
>> > > >> > > > Liu
>> > > >> > > > > > > wrote:
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> > Hi Colin
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> > Thanks for the comments!
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> > I did the following
>> changes
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    1.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    Simplified the API
>> spec
>> > > >> section
>> > > >> > to
>> > > >> > > > > only
>> > > >> > > > > > > > > include
>> > > >> > > > > > > > > >> the
>> > > >> > > > > > > > > >> > > > diff.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    2.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    Reordered the HWM
>> > > requirement
>> > > >> > > > section.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    3.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    Removed the URM
>> > > >> implementation
>> > > >> > > > details
>> > > >> > > > > > to
>> > > >> > > > > > > > keep
>> > > >> > > > > > > > > >> the
>> > > >> > > > > > > > > >> > > > >> necessary
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    characteristics to
>> > perform
>> > > >> the
>> > > >> > > > unclean
>> > > >> > > > > > > > > recovery.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    1.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >       When to perform the
>> > > >> unclean
>> > > >> > > > > recovery
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >       2.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >       Under different
>> > config,
>> > > >> how
>> > > >> > the
>> > > >> > > > > > unclean
>> > > >> > > > > > > > > >> recovery
>> > > >> > > > > > > > > >> > > > finds
>> > > >> > > > > > > > > >> > > > >> > the
>> > > >> > > > > > > > > >> > > > >> > > >>>> leader.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >       3.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >       How the config
>> > > >> > > > > > > > > unclean.leader.election.enable
>> > > >> > > > > > > > > >> > and
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>> > >  unclean.recovery.strategy
>> > > >> are
>> > > >> > > > > > converted
>> > > >> > > > > > > > > when
>> > > >> > > > > > > > > >> > users
>> > > >> > > > > > > > > >> > > > >> > > >>>> enable/disable
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> the
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >       unclean recovery.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >       4.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    More details about
>> how we
>> > > >> change
>> > > >> > > > admin
>> > > >> > > > > > > > client.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    5.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    API limits on the
>> > > >> > > > > > GetReplicaLogInfoRequest
>> > > >> > > > > > > > and
>> > > >> > > > > > > > > >> > > > >> > > >>>> DescribeTopicRequest.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    6.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    Two metrics added
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    1.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > >> > > > > > > > > >> > > >
>> > > >> > Kafka.controller.global_under_min_isr_partition_count
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >       2.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > >> > > > > > > > > >>
>> kafka.controller.unclean_recovery_finished_count
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> > On Wed, Sep 13, 2023 at
>> > > 10:46 AM
>> > > >> > > Colin
>> > > >> > > > > > > McCabe <
>> > > >> > > > > > > > > >> > > > >> > > cmccabe@apache.org>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> wrote:
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> On Tue, Sep 12, 2023, at
>> > > 17:21,
>> > > >> > > Calvin
>> > > >> > > > > Liu
>> > > >> > > > > > > > > wrote:
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Hi Colin
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Thanks for the
>> comments!
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> Hi Calvin,
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> Thanks again for the
>> KIP.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> One meta-comment: it's
>> > > usually
>> > > >> > > better
>> > > >> > > > to
>> > > >> > > > > > > just
>> > > >> > > > > > > > > do a
>> > > >> > > > > > > > > >> > diff
>> > > >> > > > > > > > > >> > > > on a
>> > > >> > > > > > > > > >> > > > >> > > >>>> message
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> spec
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> file or java file if
>> you're
>> > > >> > > including
>> > > >> > > > > > > changes
>> > > >> > > > > > > > to
>> > > >> > > > > > > > > >> it
>> > > >> > > > > > > > > >> > in
>> > > >> > > > > > > > > >> > > > the
>> > > >> > > > > > > > > >> > > > >> > KIP.
>> > > >> > > > > > > > > >> > > > >> > > >>>> This is
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> easier to read than
>> looking
>> > > for
>> > > >> > "new
>> > > >> > > > > > fields
>> > > >> > > > > > > > > begin"
>> > > >> > > > > > > > > >> > etc.
>> > > >> > > > > > > > > >> > > > in
>> > > >> > > > > > > > > >> > > > >> the
>> > > >> > > > > > > > > >> > > > >> > > >>>> text, and
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> gracefully handles the
>> case
>> > > >> where
>> > > >> > > > > existing
>> > > >> > > > > > > > > fields
>> > > >> > > > > > > > > >> > were
>> > > >> > > > > > > > > >> > > > >> > changed.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Rewrite the Additional
>> > High
>> > > >> > > > Watermark
>> > > >> > > > > > > > > >> advancement
>> > > >> > > > > > > > > >> > > > >> > requirement
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > There was feedback on
>> > this
>> > > >> > section
>> > > >> > > > > that
>> > > >> > > > > > > some
>> > > >> > > > > > > > > >> > readers
>> > > >> > > > > > > > > >> > > > may
>> > > >> > > > > > > > > >> > > > >> not
>> > > >> > > > > > > > > >> > > > >> > > be
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> familiar
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > with HWM and
>> Ack=0,1,all
>> > > >> > requests.
>> > > >> > > > > This
>> > > >> > > > > > > can
>> > > >> > > > > > > > > help
>> > > >> > > > > > > > > >> > them
>> > > >> > > > > > > > > >> > > > >> > > understand
>> > > >> > > > > > > > > >> > > > >> > > >>>> the
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > proposal. I will
>> rewrite
>> > > this
>> > > >> > part
>> > > >> > > > for
>> > > >> > > > > > > more
>> > > >> > > > > > > > > >> > > > readability.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> To be clear, I wasn't
>> > > >> suggesting
>> > > >> > > > > dropping
>> > > >> > > > > > > > either
>> > > >> > > > > > > > > >> > > > section. I
>> > > >> > > > > > > > > >> > > > >> > > agree
>> > > >> > > > > > > > > >> > > > >> > > >>>> that
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> they add useful
>> > background. I
>> > > >> was
>> > > >> > > just
>> > > >> > > > > > > > > suggesting
>> > > >> > > > > > > > > >> > that
>> > > >> > > > > > > > > >> > > we
>> > > >> > > > > > > > > >> > > > >> > should
>> > > >> > > > > > > > > >> > > > >> > > >>>> discuss
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> the "acks" setting AFTER
>> > > >> > discussing
>> > > >> > > > the
>> > > >> > > > > > new
>> > > >> > > > > > > > high
>> > > >> > > > > > > > > >> > > > watermark
>> > > >> > > > > > > > > >> > > > >> > > >>>> advancement
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> conditions. We also
>> should
>> > > >> discuss
>> > > >> > > > > acks=0.
>> > > >> > > > > > > > While
>> > > >> > > > > > > > > >> it
>> > > >> > > > > > > > > >> > > isn't
>> > > >> > > > > > > > > >> > > > >> > > >>>> conceptually
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> much
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> different than acks=1
>> here,
>> > > its
>> > > >> > > > omission
>> > > >> > > > > > > from
>> > > >> > > > > > > > > this
>> > > >> > > > > > > > > >> > > > section
>> > > >> > > > > > > > > >> > > > >> is
>> > > >> > > > > > > > > >> > > > >> > > >>>> confusing.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Unclean recovery
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > The plan is to replace
>> > the
>> > > >> > > > > > > > > >> > > > unclean.leader.election.enable
>> > > >> > > > > > > > > >> > > > >> > with
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> > unclean.recovery.strategy.
>> > > If
>> > > >> > the
>> > > >> > > > > > Unclean
>> > > >> > > > > > > > > >> Recovery
>> > > >> > > > > > > > > >> > is
>> > > >> > > > > > > > > >> > > > >> > enabled
>> > > >> > > > > > > > > >> > > > >> > > >>>> then it
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> deals
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > with the three
>> options in
>> > > the
>> > > >> > > > > > > > > >> > > > unclean.recovery.strategy.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Let’s refine the
>> Unclean
>> > > >> > Recovery.
>> > > >> > > > We
>> > > >> > > > > > have
>> > > >> > > > > > > > > >> already
>> > > >> > > > > > > > > >> > > > taken a
>> > > >> > > > > > > > > >> > > > >> > > lot of
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > suggestions and I
>> hope to
>> > > >> > enhance
>> > > >> > > > the
>> > > >> > > > > > > > > >> durability of
>> > > >> > > > > > > > > >> > > > Kafka
>> > > >> > > > > > > > > >> > > > >> to
>> > > >> > > > > > > > > >> > > > >> > > the
>> > > >> > > > > > > > > >> > > > >> > > >>>> next
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> level
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > with this KIP.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> I am OK with doing the
>> > > unclean
>> > > >> > > leader
>> > > >> > > > > > > recovery
>> > > >> > > > > > > > > >> > > > improvements
>> > > >> > > > > > > > > >> > > > >> in
>> > > >> > > > > > > > > >> > > > >> > > >>>> this KIP.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> However, I think we
>> need to
>> > > >> really
>> > > >> > > > work
>> > > >> > > > > on
>> > > >> > > > > > > the
>> > > >> > > > > > > > > >> > > > configuration
>> > > >> > > > > > > > > >> > > > >> > > >>>> settings.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> Configuration overrides
>> are
>> > > >> often
>> > > >> > > > quite
>> > > >> > > > > > > messy.
>> > > >> > > > > > > > > For
>> > > >> > > > > > > > > >> > > > example,
>> > > >> > > > > > > > > >> > > > >> > the
>> > > >> > > > > > > > > >> > > > >> > > >>>> cases
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> where we have
>> > log.roll.hours
>> > > >> and
>> > > >> > > > > > > > > >> log.roll.segment.ms
>> > > >> > > > > > > > > >> > ,
>> > > >> > > > > > > > > >> > > > the
>> > > >> > > > > > > > > >> > > > >> > user
>> > > >> > > > > > > > > >> > > > >> > > >>>> has to
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> remember which one takes
>> > > >> > precedence,
>> > > >> > > > and
>> > > >> > > > > > it
>> > > >> > > > > > > is
>> > > >> > > > > > > > > not
>> > > >> > > > > > > > > >> > > > obvious.
>> > > >> > > > > > > > > >> > > > >> > So,
>> > > >> > > > > > > > > >> > > > >> > > >>>> rather
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> than
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> creating a new
>> > configuration,
>> > > >> why
>> > > >> > > not
>> > > >> > > > > add
>> > > >> > > > > > > > > >> additional
>> > > >> > > > > > > > > >> > > > values
>> > > >> > > > > > > > > >> > > > >> to
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> "unclean.leader.election.enable"?
>> > > >> > I
>> > > >> > > > > think
>> > > >> > > > > > > this
>> > > >> > > > > > > > > >> will
>> > > >> > > > > > > > > >> > be
>> > > >> > > > > > > > > >> > > > >> simpler
>> > > >> > > > > > > > > >> > > > >> > > for
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> people
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> to understand, and
>> simpler
>> > in
>> > > >> the
>> > > >> > > code
>> > > >> > > > > as
>> > > >> > > > > > > > well.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> What if we continued to
>> use
>> > > >> > > > > > > > > >> > > > "unclean.leader.election.enable"
>> > > >> > > > > > > > > >> > > > >> > but
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> extended
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> it so that it took a
>> > string?
>> > > >> Then
>> > > >> > > the
>> > > >> > > > > > string
>> > > >> > > > > > > > > could
>> > > >> > > > > > > > > >> > have
>> > > >> > > > > > > > > >> > > > >> these
>> > > >> > > > > > > > > >> > > > >> > > >>>> values:
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> never
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>     never automatically
>> do
>> > an
>> > > >> > > unclean
>> > > >> > > > > > leader
>> > > >> > > > > > > > > >> election
>> > > >> > > > > > > > > >> > > > under
>> > > >> > > > > > > > > >> > > > >> > any
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> conditions
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> false / default
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>     only do an unclean
>> > leader
>> > > >> > > election
>> > > >> > > > > if
>> > > >> > > > > > > > there
>> > > >> > > > > > > > > >> may
>> > > >> > > > > > > > > >> > be
>> > > >> > > > > > > > > >> > > > >> > possible
>> > > >> > > > > > > > > >> > > > >> > > >>>> data
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> loss
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> true / always
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>     always do an unclean
>> > > leader
>> > > >> > > > election
>> > > >> > > > > > if
>> > > >> > > > > > > we
>> > > >> > > > > > > > > >> can't
>> > > >> > > > > > > > > >> > > > >> > immediately
>> > > >> > > > > > > > > >> > > > >> > > >>>> elect a
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> leader
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> It's a bit awkward that
>> > false
>> > > >> maps
>> > > >> > > to
>> > > >> > > > > > > default
>> > > >> > > > > > > > > >> rather
>> > > >> > > > > > > > > >> > > > than to
>> > > >> > > > > > > > > >> > > > >> > > >>>> never. But
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> this awkwardness exists
>> if
>> > we
>> > > >> use
>> > > >> > > two
>> > > >> > > > > > > > different
>> > > >> > > > > > > > > >> > > > >> configuration
>> > > >> > > > > > > > > >> > > > >> > > keys
>> > > >> > > > > > > > > >> > > > >> > > >>>> as
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> well.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> The reason for the
>> > > awkwardness
>> > > >> is
>> > > >> > > that
>> > > >> > > > > we
>> > > >> > > > > > > > simply
>> > > >> > > > > > > > > >> > don't
>> > > >> > > > > > > > > >> > > > want
>> > > >> > > > > > > > > >> > > > >> > most
>> > > >> > > > > > > > > >> > > > >> > > >>>> of the
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> people currently setting
>> > > >> > > > > > > > > >> > > > >> unclean.leader.election.enable=false
>> > > >> > > > > > > > > >> > > > >> > to
>> > > >> > > > > > > > > >> > > > >> > > >>>> get the
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> "never" behavior. We
>> have
>> > to
>> > > >> bite
>> > > >> > > that
>> > > >> > > > > > > bullet.
>> > > >> > > > > > > > > >> Better
>> > > >> > > > > > > > > >> > > to
>> > > >> > > > > > > > > >> > > > be
>> > > >> > > > > > > > > >> > > > >> > > clear
>> > > >> > > > > > > > > >> > > > >> > > >>>> and
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> explicit than hide it.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> Another thing that's a
>> bit
>> > > >> awkward
>> > > >> > > is
>> > > >> > > > > > having
>> > > >> > > > > > > > two
>> > > >> > > > > > > > > >> > > > different
>> > > >> > > > > > > > > >> > > > >> > ways
>> > > >> > > > > > > > > >> > > > >> > > to
>> > > >> > > > > > > > > >> > > > >> > > >>>> do
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> unclean leader election
>> > > >> specified
>> > > >> > in
>> > > >> > > > the
>> > > >> > > > > > > KIP.
>> > > >> > > > > > > > > You
>> > > >> > > > > > > > > >> > > > descirbe
>> > > >> > > > > > > > > >> > > > >> two
>> > > >> > > > > > > > > >> > > > >> > > >>>> methods:
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> the
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> simple "choose the last
>> > > leader"
>> > > >> > > > method,
>> > > >> > > > > > and
>> > > >> > > > > > > > the
>> > > >> > > > > > > > > >> > > "unclean
>> > > >> > > > > > > > > >> > > > >> > > recovery
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> manager"
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> method. I understand why
>> > you
>> > > >> did
>> > > >> > it
>> > > >> > > > this
>> > > >> > > > > > way
>> > > >> > > > > > > > --
>> > > >> > > > > > > > > >> > "choose
>> > > >> > > > > > > > > >> > > > the
>> > > >> > > > > > > > > >> > > > >> > last
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> leader" is
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> simple, and will help us
>> > > >> deliver
>> > > >> > an
>> > > >> > > > > > > > > implementation
>> > > >> > > > > > > > > >> > > > quickly,
>> > > >> > > > > > > > > >> > > > >> > > while
>> > > >> > > > > > > > > >> > > > >> > > >>>> the
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> URM
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> is preferable in the
>> long
>> > > >> term. My
>> > > >> > > > > > > suggestion
>> > > >> > > > > > > > > >> here is
>> > > >> > > > > > > > > >> > > to
>> > > >> > > > > > > > > >> > > > >> > > separate
>> > > >> > > > > > > > > >> > > > >> > > >>>> the
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> decision of HOW to do
>> > unclean
>> > > >> > leader
>> > > >> > > > > > > election
>> > > >> > > > > > > > > from
>> > > >> > > > > > > > > >> > the
>> > > >> > > > > > > > > >> > > > >> > decision
>> > > >> > > > > > > > > >> > > > >> > > of
>> > > >> > > > > > > > > >> > > > >> > > >>>> WHEN
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> to
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> do it.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> So in other words, have
>> > > >> > > > > > > > > >> > > "unclean.leader.election.enable"
>> > > >> > > > > > > > > >> > > > >> > specify
>> > > >> > > > > > > > > >> > > > >> > > >>>> when we
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> do unclean leader
>> election,
>> > > and
>> > > >> > > have a
>> > > >> > > > > new
>> > > >> > > > > > > > > >> > > configuration
>> > > >> > > > > > > > > >> > > > >> like
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> "unclean.recovery.manager.enable"
>> > > >> > to
>> > > >> > > > > > > determine
>> > > >> > > > > > > > > if
>> > > >> > > > > > > > > >> we
>> > > >> > > > > > > > > >> > > use
>> > > >> > > > > > > > > >> > > > the
>> > > >> > > > > > > > > >> > > > >> > > URM.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> Presumably the URM will
>> > take
>> > > >> some
>> > > >> > > time
>> > > >> > > > > to
>> > > >> > > > > > > get
>> > > >> > > > > > > > > >> fully
>> > > >> > > > > > > > > >> > > > stable,
>> > > >> > > > > > > > > >> > > > >> so
>> > > >> > > > > > > > > >> > > > >> > > >>>> this can
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> default to false for a
>> > while,
>> > > >> and
>> > > >> > we
>> > > >> > > > can
>> > > >> > > > > > > flip
>> > > >> > > > > > > > > the
>> > > >> > > > > > > > > >> > > > default to
>> > > >> > > > > > > > > >> > > > >> > > true
>> > > >> > > > > > > > > >> > > > >> > > >>>> when
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> we
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> feel ready.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> The URM is somewhat
>> > > >> > under-described
>> > > >> > > > > here.
>> > > >> > > > > > I
>> > > >> > > > > > > > > think
>> > > >> > > > > > > > > >> we
>> > > >> > > > > > > > > >> > > > need a
>> > > >> > > > > > > > > >> > > > >> > few
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> configurations here for
>> it.
>> > > For
>> > > >> > > > example,
>> > > >> > > > > > we
>> > > >> > > > > > > > > need a
>> > > >> > > > > > > > > >> > > > >> > > configuration to
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> specify
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> how long it should wait
>> > for a
>> > > >> > broker
>> > > >> > > > to
>> > > >> > > > > > > > respond
>> > > >> > > > > > > > > to
>> > > >> > > > > > > > > >> > its
>> > > >> > > > > > > > > >> > > > RPCs
>> > > >> > > > > > > > > >> > > > >> > > before
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> moving
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> on. We also need to
>> > > understand
>> > > >> how
>> > > >> > > the
>> > > >> > > > > URM
>> > > >> > > > > > > > > >> interacts
>> > > >> > > > > > > > > >> > > with
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > unclean.leader.election.enable=always. I
>> > > >> > > > > > > > assume
>> > > >> > > > > > > > > >> that
>> > > >> > > > > > > > > >> > > with
>> > > >> > > > > > > > > >> > > > >> > > "always"
>> > > >> > > > > > > > > >> > > > >> > > >>>> we
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> will
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> just unconditionally use
>> > the
>> > > >> URM
>> > > >> > > > rather
>> > > >> > > > > > than
>> > > >> > > > > > > > > >> choosing
>> > > >> > > > > > > > > >> > > > >> > randomly.
>> > > >> > > > > > > > > >> > > > >> > > >>>> But this
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> should be spelled out in
>> > the
>> > > >> KIP.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > DescribeTopicRequest
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >    1.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >    Yes, the plan is to
>> > > >> replace
>> > > >> > the
>> > > >> > > > > > > > > >> MetadataRequest
>> > > >> > > > > > > > > >> > > with
>> > > >> > > > > > > > > >> > > > >> the
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> DescribeTopicRequest
>> > for
>> > > >> the
>> > > >> > > > admin
>> > > >> > > > > > > > clients.
>> > > >> > > > > > > > > >> Will
>> > > >> > > > > > > > > >> > > > check
>> > > >> > > > > > > > > >> > > > >> > the
>> > > >> > > > > > > > > >> > > > >> > > >>>> details.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> Sounds good. But as I
>> said,
>> > > you
>> > > >> > need
>> > > >> > > > to
>> > > >> > > > > > > > specify
>> > > >> > > > > > > > > >> how
>> > > >> > > > > > > > > >> > > > >> > AdminClient
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> interacts
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> with the new request.
>> This
>> > > will
>> > > >> > > > involve
>> > > >> > > > > > > adding
>> > > >> > > > > > > > > >> some
>> > > >> > > > > > > > > >> > > > fields
>> > > >> > > > > > > > > >> > > > >> to
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> TopicDescription.java.
>> And
>> > > you
>> > > >> > need
>> > > >> > > to
>> > > >> > > > > > > specify
>> > > >> > > > > > > > > the
>> > > >> > > > > > > > > >> > > > changes
>> > > >> > > > > > > > > >> > > > >> to
>> > > >> > > > > > > > > >> > > > >> > > the
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> kafka-topics.sh command
>> > line
>> > > >> tool.
>> > > >> > > > > > Otherwise
>> > > >> > > > > > > > we
>> > > >> > > > > > > > > >> > cannot
>> > > >> > > > > > > > > >> > > > use
>> > > >> > > > > > > > > >> > > > >> the
>> > > >> > > > > > > > > >> > > > >> > > >>>> tool to
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> see
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> the new information.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> The new requests,
>> > > >> > > DescribeTopicRequest
>> > > >> > > > > and
>> > > >> > > > > > > > > >> > > > >> > > >>>> GetReplicaLogInfoRequest,
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> need
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> to have limits placed on
>> > them
>> > > >> so
>> > > >> > > that
>> > > >> > > > > > their
>> > > >> > > > > > > > size
>> > > >> > > > > > > > > >> > can't
>> > > >> > > > > > > > > >> > > be
>> > > >> > > > > > > > > >> > > > >> > > >>>> infinite. We
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> don't want to propagate
>> the
>> > > >> > current
>> > > >> > > > > > problems
>> > > >> > > > > > > > of
>> > > >> > > > > > > > > >> > > > >> > MetadataRequest,
>> > > >> > > > > > > > > >> > > > >> > > >>>> where
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> clients can request
>> massive
>> > > >> > > responses
>> > > >> > > > > that
>> > > >> > > > > > > can
>> > > >> > > > > > > > > >> mess
>> > > >> > > > > > > > > >> > up
>> > > >> > > > > > > > > >> > > > the
>> > > >> > > > > > > > > >> > > > >> JVM
>> > > >> > > > > > > > > >> > > > >> > > when
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> handled.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> Adding limits is simple
>> for
>> > > >> > > > > > > > > >> GetReplicaLogInfoRequest
>> > > >> > > > > > > > > >> > --
>> > > >> > > > > > > > > >> > > > we
>> > > >> > > > > > > > > >> > > > >> can
>> > > >> > > > > > > > > >> > > > >> > > >>>> just say
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> that only 2000
>> partitions
>> > at
>> > > a
>> > > >> > time
>> > > >> > > > can
>> > > >> > > > > be
>> > > >> > > > > > > > > >> requested.
>> > > >> > > > > > > > > >> > > For
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> DescribeTopicRequest we
>> can
>> > > >> > probably
>> > > >> > > > > just
>> > > >> > > > > > > > limit
>> > > >> > > > > > > > > >> to 20
>> > > >> > > > > > > > > >> > > > topics
>> > > >> > > > > > > > > >> > > > >> > or
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> something
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> like that, to avoid the
>> > > >> complexity
>> > > >> > > of
>> > > >> > > > > > doing
>> > > >> > > > > > > > > >> > pagination
>> > > >> > > > > > > > > >> > > in
>> > > >> > > > > > > > > >> > > > >> this
>> > > >> > > > > > > > > >> > > > >> > > KIP.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >    2.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >    I can let the
>> broker
>> > > load
>> > > >> the
>> > > >> > > ELR
>> > > >> > > > > > info
>> > > >> > > > > > > so
>> > > >> > > > > > > > > >> that
>> > > >> > > > > > > > > >> > > they
>> > > >> > > > > > > > > >> > > > can
>> > > >> > > > > > > > > >> > > > >> > > serve
>> > > >> > > > > > > > > >> > > > >> > > >>>> the
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> DescribeTopicRequest
>> > as
>> > > >> well.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> Yes, it's fine to add to
>> > > >> > > > MetadataCache.
>> > > >> > > > > In
>> > > >> > > > > > > > fact,
>> > > >> > > > > > > > > >> > you'll
>> > > >> > > > > > > > > >> > > > be
>> > > >> > > > > > > > > >> > > > >> > > loading
>> > > >> > > > > > > > > >> > > > >> > > >>>> it
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> anyway once it's added
>> to
>> > > >> > > > > PartitionImage.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >    3.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >    Yeah, it does not
>> make
>> > > >> sense
>> > > >> > to
>> > > >> > > > > have
>> > > >> > > > > > > the
>> > > >> > > > > > > > > >> topic
>> > > >> > > > > > > > > >> > id
>> > > >> > > > > > > > > >> > > if
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> DescribeTopicRequest
>> > is
>> > > >> only
>> > > >> > > used
>> > > >> > > > > by
>> > > >> > > > > > > the
>> > > >> > > > > > > > > >> admin
>> > > >> > > > > > > > > >> > > > client.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> OK. That makes things
>> > > simpler.
>> > > >> We
>> > > >> > > can
>> > > >> > > > > > always
>> > > >> > > > > > > > > >> create a
>> > > >> > > > > > > > > >> > > new
>> > > >> > > > > > > > > >> > > > >> API
>> > > >> > > > > > > > > >> > > > >> > > later
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> (hopefully not in this
>> > KIP!)
>> > > to
>> > > >> > > query
>> > > >> > > > by
>> > > >> > > > > > > topic
>> > > >> > > > > > > > > ID.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Metrics
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > As for overall cluster
>> > > health
>> > > >> > > > > metrics, I
>> > > >> > > > > > > > think
>> > > >> > > > > > > > > >> > > > >> under-min-ISR
>> > > >> > > > > > > > > >> > > > >> > > is
>> > > >> > > > > > > > > >> > > > >> > > >>>> still
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> a
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > useful one. ELR is
>> more
>> > > like
>> > > >> a
>> > > >> > > > safety
>> > > >> > > > > > > belt.
>> > > >> > > > > > > > > When
>> > > >> > > > > > > > > >> > the
>> > > >> > > > > > > > > >> > > > ELR
>> > > >> > > > > > > > > >> > > > >> is
>> > > >> > > > > > > > > >> > > > >> > > >>>> used, the
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > cluster availability
>> has
>> > > >> already
>> > > >> > > > been
>> > > >> > > > > > > > > impacted.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Maybe we can have a
>> > metric
>> > > to
>> > > >> > > count
>> > > >> > > > > the
>> > > >> > > > > > > > > >> partitions
>> > > >> > > > > > > > > >> > > that
>> > > >> > > > > > > > > >> > > > >> > > sum(ISR,
>> > > >> > > > > > > > > >> > > > >> > > >>>> ELR)
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> <
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> min
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > ISR. What do you
>> think?
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> How about:
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> A.  a metric for the
>> totoal
>> > > >> number
>> > > >> > > of
>> > > >> > > > > > > > > >> under-min-isr
>> > > >> > > > > > > > > >> > > > >> > partitions?
>> > > >> > > > > > > > > >> > > > >> > > We
>> > > >> > > > > > > > > >> > > > >> > > >>>> don't
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> have that in Apache
>> Kafka
>> > at
>> > > >> the
>> > > >> > > > moment.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> B. a metric for the
>> number
>> > of
>> > > >> > > unclean
>> > > >> > > > > > leader
>> > > >> > > > > > > > > >> > elections
>> > > >> > > > > > > > > >> > > we
>> > > >> > > > > > > > > >> > > > >> did
>> > > >> > > > > > > > > >> > > > >> > > (for
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> simplicity, it can reset
>> > to 0
>> > > >> on
>> > > >> > > > > > controller
>> > > >> > > > > > > > > >> restart:
>> > > >> > > > > > > > > >> > we
>> > > >> > > > > > > > > >> > > > >> expect
>> > > >> > > > > > > > > >> > > > >> > > >>>> people to
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> monitor the change over
>> > time
>> > > >> > anyway)
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> best,
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> Colin
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Yeah, for the ongoing
>> > > unclean
>> > > >> > > > > > recoveries,
>> > > >> > > > > > > > the
>> > > >> > > > > > > > > >> > > > controller
>> > > >> > > > > > > > > >> > > > >> can
>> > > >> > > > > > > > > >> > > > >> > > >>>> keep an
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > accurate count through
>> > > >> failover
>> > > >> > > > > because
>> > > >> > > > > > > > > >> partition
>> > > >> > > > > > > > > >> > > > >> > registration
>> > > >> > > > > > > > > >> > > > >> > > >>>> can
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> indicate
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > whether a recovery is
>> > > needed.
>> > > >> > > > However,
>> > > >> > > > > > for
>> > > >> > > > > > > > the
>> > > >> > > > > > > > > >> > > happened
>> > > >> > > > > > > > > >> > > > >> > ones,
>> > > >> > > > > > > > > >> > > > >> > > >>>> unless
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> we
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > want to persist the
>> > number
>> > > >> > > > somewhere,
>> > > >> > > > > we
>> > > >> > > > > > > can
>> > > >> > > > > > > > > >> only
>> > > >> > > > > > > > > >> > > > figure
>> > > >> > > > > > > > > >> > > > >> it
>> > > >> > > > > > > > > >> > > > >> > > out
>> > > >> > > > > > > > > >> > > > >> > > >>>> from
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> the
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > log.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > On Tue, Sep 12, 2023
>> at
>> > > >> 3:16 PM
>> > > >> > > > Colin
>> > > >> > > > > > > > McCabe <
>> > > >> > > > > > > > > >> > > > >> > > cmccabe@apache.org
>> > > >> > > > > > > > > >> > > > >> > > >>>> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> wrote:
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> Also, we should have
>> > > metrics
>> > > >> > that
>> > > >> > > > > show
>> > > >> > > > > > > what
>> > > >> > > > > > > > > is
>> > > >> > > > > > > > > >> > going
>> > > >> > > > > > > > > >> > > > on
>> > > >> > > > > > > > > >> > > > >> > with
>> > > >> > > > > > > > > >> > > > >> > > >>>> regard
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> to
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> the
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> eligible replica set.
>> > I'm
>> > > >> not
>> > > >> > > sure
>> > > >> > > > > > > exactly
>> > > >> > > > > > > > > >> what to
>> > > >> > > > > > > > > >> > > > >> suggest,
>> > > >> > > > > > > > > >> > > > >> > > but
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> something
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> that could identify
>> when
>> > > >> things
>> > > >> > > are
>> > > >> > > > > > going
>> > > >> > > > > > > > > >> wrong in
>> > > >> > > > > > > > > >> > > the
>> > > >> > > > > > > > > >> > > > >> > > clsuter.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> For example, maybe a
>> > > metric
>> > > >> for
>> > > >> > > > > > > partitions
>> > > >> > > > > > > > > >> > > containing
>> > > >> > > > > > > > > >> > > > >> > > replicas
>> > > >> > > > > > > > > >> > > > >> > > >>>> that
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> are
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> ineligible to be
>> leader?
>> > > >> That
>> > > >> > > would
>> > > >> > > > > > show
>> > > >> > > > > > > a
>> > > >> > > > > > > > > >> spike
>> > > >> > > > > > > > > >> > > when
>> > > >> > > > > > > > > >> > > > a
>> > > >> > > > > > > > > >> > > > >> > > broker
>> > > >> > > > > > > > > >> > > > >> > > >>>> had an
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> unclean restart.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> Ideally, we'd also
>> have
>> > a
>> > > >> > metric
>> > > >> > > > that
>> > > >> > > > > > > > > indicates
>> > > >> > > > > > > > > >> > when
>> > > >> > > > > > > > > >> > > > an
>> > > >> > > > > > > > > >> > > > >> > > unclear
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> leader
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> election or a
>> recovery
>> > > >> > happened.
>> > > >> > > > > It's a
>> > > >> > > > > > > bit
>> > > >> > > > > > > > > >> tricky
>> > > >> > > > > > > > > >> > > > >> because
>> > > >> > > > > > > > > >> > > > >> > > the
>> > > >> > > > > > > > > >> > > > >> > > >>>> simple
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> thing, of tracking it
>> > per
>> > > >> > > > controller,
>> > > >> > > > > > may
>> > > >> > > > > > > > be
>> > > >> > > > > > > > > a
>> > > >> > > > > > > > > >> bit
>> > > >> > > > > > > > > >> > > > >> > confusing
>> > > >> > > > > > > > > >> > > > >> > > >>>> during
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> failovers.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> best,
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> Colin
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >>
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> On Tue, Sep 12,
>> 2023, at
>> > > >> 14:25,
>> > > >> > > > Colin
>> > > >> > > > > > > > McCabe
>> > > >> > > > > > > > > >> > wrote:
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > Hi Calvin,
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > Thanks for the
>> KIP. I
>> > > >> think
>> > > >> > > this
>> > > >> > > > > is a
>> > > >> > > > > > > > great
>> > > >> > > > > > > > > >> > > > >> improvement.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >> Additional High
>> > > Watermark
>> > > >> > > > advance
>> > > >> > > > > > > > > >> requirement
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > Typo: change
>> "advance"
>> > > to
>> > > >> > > > > > "advancement"
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >> A bit recap of
>> some
>> > key
>> > > >> > > > concepts.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > Typo: change "bit"
>> to
>> > > >> "quick"
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >> Ack=1/all produce
>> > > >> request.
>> > > >> > It
>> > > >> > > > > > defines
>> > > >> > > > > > > > when
>> > > >> > > > > > > > > >> the
>> > > >> > > > > > > > > >> > > > Kafka
>> > > >> > > > > > > > > >> > > > >> > > server
>> > > >> > > > > > > > > >> > > > >> > > >>>> should
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> respond to the
>> produce
>> > > >> request
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > I think this
>> section
>> > > >> would be
>> > > >> > > > > clearer
>> > > >> > > > > > > if
>> > > >> > > > > > > > we
>> > > >> > > > > > > > > >> > talked
>> > > >> > > > > > > > > >> > > > >> about
>> > > >> > > > > > > > > >> > > > >> > > the
>> > > >> > > > > > > > > >> > > > >> > > >>>> new
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> high
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > watermark
>> advancement
>> > > >> > > requirement
>> > > >> > > > > > > first,
>> > > >> > > > > > > > > and
>> > > >> > > > > > > > > >> > THEN
>> > > >> > > > > > > > > >> > > > >> talked
>> > > >> > > > > > > > > >> > > > >> > > >>>> about its
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > impact on acks=0,
>> > > acks=1,
>> > > >> and
>> > > >> > > > > > > >  acks=all.
>> > > >> > > > > > > > > >> > > > acks=all
>> > > >> > > > > > > > > >> > > > >> is
>> > > >> > > > > > > > > >> > > > >> > of
>> > > >> > > > > > > > > >> > > > >> > > >>>> course
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> the
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > main case we care
>> > about
>> > > >> here,
>> > > >> > > so
>> > > >> > > > it
>> > > >> > > > > > > would
>> > > >> > > > > > > > > be
>> > > >> > > > > > > > > >> > good
>> > > >> > > > > > > > > >> > > to
>> > > >> > > > > > > > > >> > > > >> lead
>> > > >> > > > > > > > > >> > > > >> > > with
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> that,
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > rather than delving
>> > into
>> > > >> the
>> > > >> > > > > > > > technicalities
>> > > >> > > > > > > > > >> of
>> > > >> > > > > > > > > >> > > > acks=0/1
>> > > >> > > > > > > > > >> > > > >> > > first.
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >> Unclean recovery
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > So, here you are
>> > > >> introducing
>> > > >> > a
>> > > >> > > > new
>> > > >> > > > > > > > > >> > configuration,
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
>> > > unclean.recovery.strategy.
>> > > >> > The
>> > > >> > > > > > > difficult
>> > > >> > > > > > > > > >> thing
>> > > >> > > > > > > > > >> > > here
>> > > >> > > > > > > > > >> > > > is
>> > > >> > > > > > > > > >> > > > >> > that
>> > > >> > > > > > > > > >> > > > >> > > >>>> there
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> is a
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > lot of overlap with
>> > > >> > > > > > > > > >> > > unclean.leader.election.enable.
>> > > >> > > > > > > > > >> > > > So
>> > > >> > > > > > > > > >> > > > >> we
>> > > >> > > > > > > > > >> > > > >> > > >>>> have 3
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > different settings
>> for
>> > > >> > > > > > > > > >> > unclean.recovery.strategy,
>> > > >> > > > > > > > > >> > > > plus
>> > > >> > > > > > > > > >> > > > >> 2
>> > > >> > > > > > > > > >> > > > >> > > >>>> different
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > settings for
>> > > >> > > > > > > > > unclean.leader.election.enable,
>> > > >> > > > > > > > > >> > > giving
>> > > >> > > > > > > > > >> > > > a
>> > > >> > > > > > > > > >> > > > >> > cross
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> product of
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > 6 different
>> options.
>> > The
>> > > >> > > > following
>> > > >> > > > > > > > "unclean
>> > > >> > > > > > > > > >> > > recovery
>> > > >> > > > > > > > > >> > > > >> > > manager"
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> section
>> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > on
>> > > >> > > > > >
>> > > >> > > > >
>> > > >> > > >
>> > > >> > >
>> > > >> >
>> > > >>
>> > > >
>> > >
>> >
>>
>>
>> --
>> David Arthur
>>
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Calvin Liu <ca...@confluent.io.INVALID>.
Hi David,
Thanks for the comment.
Yes, we can separate the ELR enablement from the metadata version. It is
also helpful to avoid blocking the following MV releases if the user is not
ready for ELR.
One thing to correct is that, the Unclean recovery is controlled
by unclean.recovery.manager.enabled, a separate config
from unclean.recovery.strategy. It determines whether unclean recovery will
be used in an unclean leader election.
Thanks

On Wed, Oct 11, 2023 at 4:11 PM David Arthur <mu...@gmail.com> wrote:

> One thing we should consider is a static config to totally enable/disable
> the ELR feature. If I understand the KIP correctly, we can effectively
> disable the unclean recovery by setting the recovery strategy config to
> "none".
>
> This would make development and rollout of this feature a bit smoother.
> Consider the case that we find bugs in ELR after a cluster has updated to
> its MetadataVersion. It's simpler to disable the feature through config
> rather than going through a MetadataVersion downgrade (once that's
> supported).
>
> Does that make sense?
>
> -David
>
> On Wed, Oct 11, 2023 at 1:40 PM Calvin Liu <ca...@confluent.io.invalid>
> wrote:
>
> > Hi Jun
> > -Good catch, yes, we don't need the -1 in the DescribeTopicRequest.
> > -No new value is added. The LeaderRecoveryState will still be set to 1 if
> > we have an unclean leader election. The unclean leader election includes
> > the old random way and the unclean recovery. During the unclean recovery,
> > the LeaderRecoveryState will not change until the controller decides to
> > update the records with the new leader.
> > Thanks
> >
> > On Wed, Oct 11, 2023 at 9:02 AM Jun Rao <ju...@confluent.io.invalid>
> wrote:
> >
> > > Hi, Calvin,
> > >
> > > Another thing. Currently, when there is an unclean leader election, we
> > set
> > > the LeaderRecoveryState in PartitionRecord and PartitionChangeRecord to
> > 1.
> > > With the KIP, will there be new values for LeaderRecoveryState? If not,
> > > when will LeaderRecoveryState be set to 1?
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Tue, Oct 10, 2023 at 4:24 PM Jun Rao <ju...@confluent.io> wrote:
> > >
> > > > Hi, Calvin,
> > > >
> > > > One more comment.
> > > >
> > > > "The first partition to fetch details for. -1 means to fetch all
> > > > partitions." It seems that FirstPartitionId of 0 naturally means
> > fetching
> > > > all partitions?
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > > On Tue, Oct 10, 2023 at 12:40 PM Calvin Liu
> <caliu@confluent.io.invalid
> > >
> > > > wrote:
> > > >
> > > >> Hi Jun,
> > > >> Yeah, with the current Metadata request handling, we only return
> > errors
> > > on
> > > >> the Topic level, like topic not found. It seems that querying a
> > specific
> > > >> partition is not a valid use case. Will update.
> > > >> Thanks
> > > >>
> > > >> On Tue, Oct 10, 2023 at 11:55 AM Jun Rao <ju...@confluent.io.invalid>
> > > >> wrote:
> > > >>
> > > >> > Hi, Calvin,
> > > >> >
> > > >> > 60.  If the range query has errors for some of the partitions, do
> we
> > > >> expect
> > > >> > different responses when querying particular partitions?
> > > >> >
> > > >> > Thanks,
> > > >> >
> > > >> > Jun
> > > >> >
> > > >> > On Tue, Oct 10, 2023 at 10:50 AM Calvin Liu
> > > <caliu@confluent.io.invalid
> > > >> >
> > > >> > wrote:
> > > >> >
> > > >> > > Hi Jun
> > > >> > > 60. Yes, it is a good question. I was thinking the API could be
> > > >> flexible
> > > >> > to
> > > >> > > query the particular partitions if the range query has errors
> for
> > > >> some of
> > > >> > > the partitions. Not sure whether it is a valid assumption, what
> do
> > > you
> > > >> > > think?
> > > >> > >
> > > >> > > 61. Good point, I will update them to partition level with the
> > same
> > > >> > limit.
> > > >> > >
> > > >> > > 62. Sure, will do.
> > > >> > >
> > > >> > > Thanks
> > > >> > >
> > > >> > > On Tue, Oct 10, 2023 at 10:12 AM Jun Rao
> <jun@confluent.io.invalid
> > >
> > > >> > wrote:
> > > >> > >
> > > >> > > > Hi, Calvin,
> > > >> > > >
> > > >> > > > A few more minor comments on your latest update.
> > > >> > > >
> > > >> > > > 60. DescribeTopicRequest: When will the Partitions field be
> > used?
> > > It
> > > >> > > seems
> > > >> > > > that the FirstPartitionId field is enough for AdminClient
> usage.
> > > >> > > >
> > > >> > > > 61. Could we make the limit for DescribeTopicRequest,
> > > >> > > ElectLeadersRequest,
> > > >> > > > GetReplicaLogInfo consistent? Currently, ElectLeadersRequest's
> > > >> limit is
> > > >> > > at
> > > >> > > > topic level and GetReplicaLogInfo has a different partition
> > level
> > > >> limit
> > > >> > > > from DescribeTopicRequest.
> > > >> > > >
> > > >> > > > 62. Should ElectLeadersRequest.DesiredLeaders be at the same
> > level
> > > >> as
> > > >> > > > ElectLeadersRequest.TopicPartitions.Partitions? In the KIP, it
> > > looks
> > > >> > like
> > > >> > > > it's at the same level as ElectLeadersRequest.TopicPartitions.
> > > >> > > >
> > > >> > > > Thanks,
> > > >> > > >
> > > >> > > > Jun
> > > >> > > >
> > > >> > > > On Wed, Oct 4, 2023 at 3:55 PM Calvin Liu
> > > >> <ca...@confluent.io.invalid>
> > > >> > > > wrote:
> > > >> > > >
> > > >> > > > > Hi David,
> > > >> > > > > Thanks for the comments.
> > > >> > > > > ----
> > > >> > > > > I thought that a new snapshot with the downgraded MV is
> > created
> > > in
> > > >> > this
> > > >> > > > > case. Isn’t it the case?
> > > >> > > > > Yes, you are right, a metadata delta will be generated after
> > the
> > > >> MV
> > > >> > > > > downgrade. Then the user can start the software downgrade.
> > > >> > > > > -----
> > > >> > > > > Could you also elaborate a bit more on the reasoning behind
> > > adding
> > > >> > the
> > > >> > > > > limits to the admin RPCs? This is a new pattern in Kafka so
> it
> > > >> would
> > > >> > be
> > > >> > > > > good to clear on the motivation.
> > > >> > > > > Thanks to Colin for bringing it up. The current
> > MetadataRequest
> > > >> does
> > > >> > > not
> > > >> > > > > have a limit on the number of topics to query in a single
> > > request.
> > > >> > > > Massive
> > > >> > > > > requests can mess up the JVM. We want to have some sort of
> > > >> throttle
> > > >> > on
> > > >> > > > the
> > > >> > > > > new APIs.
> > > >> > > > > -----
> > > >> > > > > Could you also explain how the client is supposed to handle
> > the
> > > >> > > > > topics/partitions above the limit? I suppose that it will
> have
> > > to
> > > >> > retry
> > > >> > > > > those, correct?
> > > >> > > > > Corrent. For the official admin clients, it will split the
> > large
> > > >> > > request
> > > >> > > > > into proper pieces and query one after another.
> > > >> > > > > -----
> > > >> > > > > My understanding is that the topics/partitions above the
> limit
> > > >> will
> > > >> > be
> > > >> > > > > failed with an invalid exception error. I wonder if this
> > choice
> > > is
> > > >> > > > > judicious because the invalide request exception is usually
> > > >> fatal. It
> > > >> > > may
> > > >> > > > > be better to use an new and explicit error for this case.
> > > >> > > > >
> > > >> > > > > Thanks for bringing this up. How about
> > "REQUEST_LIMIT_REACHED"?
> > > >> > > > > --------
> > > >> > > > > It seems that we still need to specify the changes to the
> > admin
> > > >> api
> > > >> > to
> > > >> > > > > accommodate the new or updated apis. Do you plan to add
> them?
> > > >> > > > > Try to cover the following
> > > >> > > > > 1. The admin client will use the new DescribeTopicRequest to
> > > query
> > > >> > the
> > > >> > > > > topics
> > > >> > > > > 2. Mention the API limit and the new retriable error.
> > > >> > > > > 3. Output changes for the admin client when describing a
> topic
> > > >> (new
> > > >> > > > fields
> > > >> > > > > of ELR...)
> > > >> > > > > 4. Changes to data structures like TopicPartitionInfo to
> > include
> > > >> the
> > > >> > > ELR.
> > > >> > > > > Anything else I missed?
> > > >> > > > >
> > > >> > > > > Thanks!
> > > >> > > > >
> > > >> > > > >
> > > >> > > > >
> > > >> > > > >
> > > >> > > > >
> > > >> > > > > On Wed, Oct 4, 2023 at 12:27 PM David Jacot <
> > > >> david.jacot@gmail.com>
> > > >> > > > wrote:
> > > >> > > > >
> > > >> > > > > > Hi Calvin,
> > > >> > > > > >
> > > >> > > > > > I thought that a new snapshot with the downgraded MV is
> > > created
> > > >> in
> > > >> > > this
> > > >> > > > > > case. Isn’t it the case?
> > > >> > > > > >
> > > >> > > > > > Could you also elaborate a bit more on the reasoning
> behind
> > > >> adding
> > > >> > > the
> > > >> > > > > > limits to the admin RPCs? This is a new pattern in Kafka
> so
> > it
> > > >> > would
> > > >> > > be
> > > >> > > > > > good to clear on the motivation.
> > > >> > > > > >
> > > >> > > > > > Could you also explain how the client is supposed to
> handle
> > > the
> > > >> > > > > > topics/partitions above the limit? I suppose that it will
> > have
> > > >> to
> > > >> > > retry
> > > >> > > > > > those, correct?
> > > >> > > > > >
> > > >> > > > > > My understanding is that the topics/partitions above the
> > limit
> > > >> will
> > > >> > > be
> > > >> > > > > > failed with an invalid exception error. I wonder if this
> > > choice
> > > >> is
> > > >> > > > > > judicious because the invalide request exception is
> usually
> > > >> fatal.
> > > >> > It
> > > >> > > > may
> > > >> > > > > > be better to use an new and explicit error for this case.
> > > >> > > > > >
> > > >> > > > > > It seems that we still need to specify the changes to the
> > > admin
> > > >> api
> > > >> > > to
> > > >> > > > > > accommodate the new or updated apis. Do you plan to add
> > them?
> > > >> > > > > >
> > > >> > > > > > Best,
> > > >> > > > > > David
> > > >> > > > > >
> > > >> > > > > > Le mer. 4 oct. 2023 à 20:39, Calvin Liu
> > > >> <caliu@confluent.io.invalid
> > > >> > >
> > > >> > > a
> > > >> > > > > > écrit :
> > > >> > > > > >
> > > >> > > > > > > Hi Jun,
> > > >> > > > > > > After the MV downgrade, the controller will write in the
> > old
> > > >> > > version
> > > >> > > > of
> > > >> > > > > > the
> > > >> > > > > > > PartitionRecord/PartitionChangeRecord. If I understand
> > > >> correctly,
> > > >> > > it
> > > >> > > > is
> > > >> > > > > > > possible to downgrade the software version if the
> > controller
> > > >> only
> > > >> > > has
> > > >> > > > > to
> > > >> > > > > > > handle old version records.
> > > >> > > > > > > However, the controller will not automatically rewrite
> the
> > > >> > > > > > PartitionRecord
> > > >> > > > > > > with the old version unless there is a partition update.
> > > Then,
> > > >> > the
> > > >> > > > user
> > > >> > > > > > may
> > > >> > > > > > > have to wait an unknown amount of time before the
> software
> > > >> > > downgrades
> > > >> > > > > > > unless they do a roll to force update every partition.
> If
> > it
> > > >> > makes
> > > >> > > > > > sense, I
> > > >> > > > > > > can mention these steps to do a software downgrade.
> > > >> > > > > > > Thanks
> > > >> > > > > > >
> > > >> > > > > > > On Wed, Oct 4, 2023 at 11:20 AM Jun Rao
> > > >> <jun@confluent.io.invalid
> > > >> > >
> > > >> > > > > > wrote:
> > > >> > > > > > >
> > > >> > > > > > > > Hi, Calvin and Justine,
> > > >> > > > > > > >
> > > >> > > > > > > > Historically, when we change the record format in the
> > log,
> > > >> we
> > > >> > > don't
> > > >> > > > > > > support
> > > >> > > > > > > > software version downgrading.
> > > >> > > > > > > >
> > > >> > > > > > > > For the record format change in the metadata log, have
> > we
> > > >> > thought
> > > >> > > > > about
> > > >> > > > > > > > forcing the write of the latest metadata records with
> > the
> > > >> old
> > > >> > > > version
> > > >> > > > > > > > during MV downgrading? This will in theory allow the
> old
> > > >> > version
> > > >> > > of
> > > >> > > > > the
> > > >> > > > > > > > software to obtain the latest metadata.
> > > >> > > > > > > >
> > > >> > > > > > > > Thanks,
> > > >> > > > > > > >
> > > >> > > > > > > > Jun
> > > >> > > > > > > >
> > > >> > > > > > > > On Wed, Oct 4, 2023 at 9:53 AM Justine Olshan
> > > >> > > > > > > <jolshan@confluent.io.invalid
> > > >> > > > > > > > >
> > > >> > > > > > > > wrote:
> > > >> > > > > > > >
> > > >> > > > > > > > > Sorry -- not MV but software version.
> > > >> > > > > > > > >
> > > >> > > > > > > > > On Wed, Oct 4, 2023 at 9:51 AM Justine Olshan <
> > > >> > > > > jolshan@confluent.io>
> > > >> > > > > > > > > wrote:
> > > >> > > > > > > > >
> > > >> > > > > > > > > > Catching up with this discussion.
> > > >> > > > > > > > > >
> > > >> > > > > > > > > > I was just curious -- have we had other instances
> > > where
> > > >> > > > > downgrading
> > > >> > > > > > > MV
> > > >> > > > > > > > is
> > > >> > > > > > > > > > not supported? I think Kafka typically tries to
> > > support
> > > >> > > > > downgrades,
> > > >> > > > > > > > and I
> > > >> > > > > > > > > > couldn't think of other examples.
> > > >> > > > > > > > > >
> > > >> > > > > > > > > > Thanks,
> > > >> > > > > > > > > > Justine
> > > >> > > > > > > > > >
> > > >> > > > > > > > > > On Wed, Oct 4, 2023 at 9:40 AM Calvin Liu
> > > >> > > > > > <caliu@confluent.io.invalid
> > > >> > > > > > > >
> > > >> > > > > > > > > > wrote:
> > > >> > > > > > > > > >
> > > >> > > > > > > > > >> Hi Jun,
> > > >> > > > > > > > > >> 54. Marked the software downgrading is not
> > supported.
> > > >> As
> > > >> > the
> > > >> > > > old
> > > >> > > > > > > > > >> controller
> > > >> > > > > > > > > >> will not understand the new PartitionRecord and
> > > >> > > > > > > PartitionChangeRecord.
> > > >> > > > > > > > > >> Thanks!
> > > >> > > > > > > > > >>
> > > >> > > > > > > > > >> On Wed, Oct 4, 2023 at 9:12 AM Jun Rao
> > > >> > > > <jun@confluent.io.invalid
> > > >> > > > > >
> > > >> > > > > > > > > wrote:
> > > >> > > > > > > > > >>
> > > >> > > > > > > > > >> > Hi, Calvin,
> > > >> > > > > > > > > >> >
> > > >> > > > > > > > > >> > Thanks for the reply. Just one more comment.
> > > >> > > > > > > > > >> >
> > > >> > > > > > > > > >> > 54. It seems that downgrading MV is supported.
> Is
> > > >> > > > downgrading
> > > >> > > > > > the
> > > >> > > > > > > > > >> software
> > > >> > > > > > > > > >> > version supported? It would be useful to
> document
> > > >> that.
> > > >> > > > > > > > > >> >
> > > >> > > > > > > > > >> > Thanks,
> > > >> > > > > > > > > >> >
> > > >> > > > > > > > > >> > Jun
> > > >> > > > > > > > > >> >
> > > >> > > > > > > > > >> > On Tue, Oct 3, 2023 at 4:55 PM Artem Livshits
> > > >> > > > > > > > > >> > <al...@confluent.io.invalid> wrote:
> > > >> > > > > > > > > >> >
> > > >> > > > > > > > > >> > > Hi Colin,
> > > >> > > > > > > > > >> > >
> > > >> > > > > > > > > >> > > I think in your example "do_unclean_recovery"
> > > would
> > > >> > need
> > > >> > > > to
> > > >> > > > > do
> > > >> > > > > > > > > >> different
> > > >> > > > > > > > > >> > > things depending on the strategy.
> > > >> > > > > > > > > >> > >
> > > >> > > > > > > > > >> > > do_unclean_recovery() {
> > > >> > > > > > > > > >> > >    if (unclean.recovery.manager.enabled) {
> > > >> > > > > > > > > >> > >     if (strategy == Aggressive)
> > > >> > > > > > > > > >> > >       use
> > > >> > UncleanRecoveryManager(waitLastKnownERL=false)
> > > >> > > > //
> > > >> > > > > > > just
> > > >> > > > > > > > > >> inspect
> > > >> > > > > > > > > >> > > logs from whoever is available
> > > >> > > > > > > > > >> > >     else
> > > >> > > > > > > > > >> > >       use
> > > >> > UncleanRecoveryManager(waitLastKnownERL=true)
> > > >> > > > //
> > > >> > > > > > > must
> > > >> > > > > > > > > wait
> > > >> > > > > > > > > >> > for
> > > >> > > > > > > > > >> > > at least last known ELR
> > > >> > > > > > > > > >> > >   } else {
> > > >> > > > > > > > > >> > >     if (strategy == Aggressive)
> > > >> > > > > > > > > >> > >       choose the last known leader if that is
> > > >> > available,
> > > >> > > > or
> > > >> > > > > a
> > > >> > > > > > > > random
> > > >> > > > > > > > > >> > leader
> > > >> > > > > > > > > >> > > if not)
> > > >> > > > > > > > > >> > >     else
> > > >> > > > > > > > > >> > >       wait for last known leader to get back
> > > >> > > > > > > > > >> > >   }
> > > >> > > > > > > > > >> > > }
> > > >> > > > > > > > > >> > >
> > > >> > > > > > > > > >> > > The idea is that the Aggressive strategy
> would
> > > >> kick in
> > > >> > > as
> > > >> > > > > soon
> > > >> > > > > > > as
> > > >> > > > > > > > we
> > > >> > > > > > > > > >> lost
> > > >> > > > > > > > > >> > > the leader and would pick a leader from
> whoever
> > > is
> > > >> > > > > available;
> > > >> > > > > > > but
> > > >> > > > > > > > > the
> > > >> > > > > > > > > >> > > Balanced will only kick in when ELR is empty
> > and
> > > >> will
> > > >> > > wait
> > > >> > > > > for
> > > >> > > > > > > the
> > > >> > > > > > > > > >> > brokers
> > > >> > > > > > > > > >> > > that likely have most data to be available.
> > > >> > > > > > > > > >> > >
> > > >> > > > > > > > > >> > > On Tue, Oct 3, 2023 at 3:04 PM Colin McCabe <
> > > >> > > > > > cmccabe@apache.org
> > > >> > > > > > > >
> > > >> > > > > > > > > >> wrote:
> > > >> > > > > > > > > >> > >
> > > >> > > > > > > > > >> > > > On Tue, Oct 3, 2023, at 10:49, Jun Rao
> wrote:
> > > >> > > > > > > > > >> > > > > Hi, Calvin,
> > > >> > > > > > > > > >> > > > >
> > > >> > > > > > > > > >> > > > > Thanks for the update KIP. A few more
> > > comments.
> > > >> > > > > > > > > >> > > > >
> > > >> > > > > > > > > >> > > > > 41. Why would a user choose the option to
> > > >> select a
> > > >> > > > > random
> > > >> > > > > > > > > replica
> > > >> > > > > > > > > >> as
> > > >> > > > > > > > > >> > > the
> > > >> > > > > > > > > >> > > > > leader instead of using
> > > >> > > > > > unclean.recovery.strateg=Aggressive?
> > > >> > > > > > > > It
> > > >> > > > > > > > > >> seems
> > > >> > > > > > > > > >> > > > that
> > > >> > > > > > > > > >> > > > > the latter is strictly better? If that's
> > not
> > > >> the
> > > >> > > case,
> > > >> > > > > > could
> > > >> > > > > > > > we
> > > >> > > > > > > > > >> fold
> > > >> > > > > > > > > >> > > this
> > > >> > > > > > > > > >> > > > > option under unclean.recovery.strategy
> > > instead
> > > >> of
> > > >> > > > > > > introducing
> > > >> > > > > > > > a
> > > >> > > > > > > > > >> > > separate
> > > >> > > > > > > > > >> > > > > config?
> > > >> > > > > > > > > >> > > >
> > > >> > > > > > > > > >> > > > Hi Jun,
> > > >> > > > > > > > > >> > > >
> > > >> > > > > > > > > >> > > > I thought the flow of control was:
> > > >> > > > > > > > > >> > > >
> > > >> > > > > > > > > >> > > > If there is no leader for the partition {
> > > >> > > > > > > > > >> > > >   If (there are unfenced ELR members) {
> > > >> > > > > > > > > >> > > >     choose_an_unfenced_ELR_member
> > > >> > > > > > > > > >> > > >   } else if (there are fenced ELR members
> AND
> > > >> > > > > > > > > strategy=Aggressive) {
> > > >> > > > > > > > > >> > > >     do_unclean_recovery
> > > >> > > > > > > > > >> > > >   } else if (there are no ELR members AND
> > > >> strategy
> > > >> > !=
> > > >> > > > > None)
> > > >> > > > > > {
> > > >> > > > > > > > > >> > > >     do_unclean_recovery
> > > >> > > > > > > > > >> > > >   } else {
> > > >> > > > > > > > > >> > > >     do nothing about the missing leader
> > > >> > > > > > > > > >> > > >   }
> > > >> > > > > > > > > >> > > > }
> > > >> > > > > > > > > >> > > >
> > > >> > > > > > > > > >> > > > do_unclean_recovery() {
> > > >> > > > > > > > > >> > > >    if (unclean.recovery.manager.enabled) {
> > > >> > > > > > > > > >> > > >     use UncleanRecoveryManager
> > > >> > > > > > > > > >> > > >   } else {
> > > >> > > > > > > > > >> > > >     choose the last known leader if that is
> > > >> > available,
> > > >> > > > or
> > > >> > > > > a
> > > >> > > > > > > > random
> > > >> > > > > > > > > >> > leader
> > > >> > > > > > > > > >> > > > if not)
> > > >> > > > > > > > > >> > > >   }
> > > >> > > > > > > > > >> > > > }
> > > >> > > > > > > > > >> > > >
> > > >> > > > > > > > > >> > > > However, I think this could be clarified,
> > > >> especially
> > > >> > > the
> > > >> > > > > > > > behavior
> > > >> > > > > > > > > >> when
> > > >> > > > > > > > > >> > > > unclean.recovery.manager.enabled=false.
> > > >> Inuitively
> > > >> > the
> > > >> > > > > goal
> > > >> > > > > > > for
> > > >> > > > > > > > > >> > > > unclean.recovery.manager.enabled=false is
> to
> > be
> > > >> "the
> > > >> > > > same
> > > >> > > > > as
> > > >> > > > > > > > now,
> > > >> > > > > > > > > >> > mostly"
> > > >> > > > > > > > > >> > > > but it's very underspecified in the KIP, I
> > > agree.
> > > >> > > > > > > > > >> > > >
> > > >> > > > > > > > > >> > > > >
> > > >> > > > > > > > > >> > > > > 50. ElectLeadersRequest: "If more than 20
> > > >> topics
> > > >> > are
> > > >> > > > > > > included,
> > > >> > > > > > > > > >> only
> > > >> > > > > > > > > >> > the
> > > >> > > > > > > > > >> > > > > first 20 will be served. Others will be
> > > >> returned
> > > >> > > with
> > > >> > > > > > > > > >> > DesiredLeaders."
> > > >> > > > > > > > > >> > > > Hmm,
> > > >> > > > > > > > > >> > > > > not sure that I understand this.
> > > >> > > ElectLeadersResponse
> > > >> > > > > > > doesn't
> > > >> > > > > > > > > >> have a
> > > >> > > > > > > > > >> > > > > DesiredLeaders field.
> > > >> > > > > > > > > >> > > > >
> > > >> > > > > > > > > >> > > > > 51. GetReplicaLogInfo: "If more than 2000
> > > >> > partitions
> > > >> > > > are
> > > >> > > > > > > > > included,
> > > >> > > > > > > > > >> > only
> > > >> > > > > > > > > >> > > > the
> > > >> > > > > > > > > >> > > > > first 2000 will be served" Do we return
> an
> > > >> error
> > > >> > for
> > > >> > > > the
> > > >> > > > > > > > > remaining
> > > >> > > > > > > > > >> > > > > partitions? Actually, should we include
> an
> > > >> > errorCode
> > > >> > > > > field
> > > >> > > > > > > at
> > > >> > > > > > > > > the
> > > >> > > > > > > > > >> > > > partition
> > > >> > > > > > > > > >> > > > > level in GetReplicaLogInfoResponse to
> cover
> > > >> > > > non-existing
> > > >> > > > > > > > > >> partitions
> > > >> > > > > > > > > >> > and
> > > >> > > > > > > > > >> > > > no
> > > >> > > > > > > > > >> > > > > authorization, etc?
> > > >> > > > > > > > > >> > > > >
> > > >> > > > > > > > > >> > > > > 52. The entry should matches => The entry
> > > >> should
> > > >> > > match
> > > >> > > > > > > > > >> > > > >
> > > >> > > > > > > > > >> > > > > 53. ElectLeadersRequest.DesiredLeaders:
> > > Should
> > > >> it
> > > >> > be
> > > >> > > > > > > nullable
> > > >> > > > > > > > > >> since a
> > > >> > > > > > > > > >> > > > user
> > > >> > > > > > > > > >> > > > > may not specify DesiredLeaders?
> > > >> > > > > > > > > >> > > > >
> > > >> > > > > > > > > >> > > > > 54. Downgrade: Is that indeed possible? I
> > > >> thought
> > > >> > > > > earlier
> > > >> > > > > > > you
> > > >> > > > > > > > > said
> > > >> > > > > > > > > >> > that
> > > >> > > > > > > > > >> > > > > once the new version of the records are
> in
> > > the
> > > >> > > > metadata
> > > >> > > > > > log,
> > > >> > > > > > > > one
> > > >> > > > > > > > > >> > can't
> > > >> > > > > > > > > >> > > > > downgrade since the old broker doesn't
> know
> > > >> how to
> > > >> > > > parse
> > > >> > > > > > the
> > > >> > > > > > > > new
> > > >> > > > > > > > > >> > > version
> > > >> > > > > > > > > >> > > > of
> > > >> > > > > > > > > >> > > > > the metadata records?
> > > >> > > > > > > > > >> > > > >
> > > >> > > > > > > > > >> > > >
> > > >> > > > > > > > > >> > > > MetadataVersion downgrade is currently
> broken
> > > >> but we
> > > >> > > > have
> > > >> > > > > > > fixing
> > > >> > > > > > > > > it
> > > >> > > > > > > > > >> on
> > > >> > > > > > > > > >> > > our
> > > >> > > > > > > > > >> > > > plate for Kafka 3.7.
> > > >> > > > > > > > > >> > > >
> > > >> > > > > > > > > >> > > > The way downgrade works is that "new
> > features"
> > > >> are
> > > >> > > > > dropped,
> > > >> > > > > > > > > leaving
> > > >> > > > > > > > > >> > only
> > > >> > > > > > > > > >> > > > the old ones.
> > > >> > > > > > > > > >> > > >
> > > >> > > > > > > > > >> > > > > 55. CleanShutdownFile: Should we add a
> > > version
> > > >> > field
> > > >> > > > for
> > > >> > > > > > > > future
> > > >> > > > > > > > > >> > > > extension?
> > > >> > > > > > > > > >> > > > >
> > > >> > > > > > > > > >> > > > > 56. Config changes are public facing.
> Could
> > > we
> > > >> > have
> > > >> > > a
> > > >> > > > > > > separate
> > > >> > > > > > > > > >> > section
> > > >> > > > > > > > > >> > > to
> > > >> > > > > > > > > >> > > > > document all the config changes?
> > > >> > > > > > > > > >> > > >
> > > >> > > > > > > > > >> > > > +1. A separate section for this would be
> > good.
> > > >> > > > > > > > > >> > > >
> > > >> > > > > > > > > >> > > > best,
> > > >> > > > > > > > > >> > > > Colin
> > > >> > > > > > > > > >> > > >
> > > >> > > > > > > > > >> > > > >
> > > >> > > > > > > > > >> > > > > Thanks,
> > > >> > > > > > > > > >> > > > >
> > > >> > > > > > > > > >> > > > > Jun
> > > >> > > > > > > > > >> > > > >
> > > >> > > > > > > > > >> > > > > On Mon, Sep 25, 2023 at 4:29 PM Calvin
> Liu
> > > >> > > > > > > > > >> > <caliu@confluent.io.invalid
> > > >> > > > > > > > > >> > > >
> > > >> > > > > > > > > >> > > > > wrote:
> > > >> > > > > > > > > >> > > > >
> > > >> > > > > > > > > >> > > > >> Hi Jun
> > > >> > > > > > > > > >> > > > >> Thanks for the comments.
> > > >> > > > > > > > > >> > > > >>
> > > >> > > > > > > > > >> > > > >> 40. If we change to None, it is not
> > > guaranteed
> > > >> > for
> > > >> > > no
> > > >> > > > > > data
> > > >> > > > > > > > > loss.
> > > >> > > > > > > > > >> For
> > > >> > > > > > > > > >> > > > users
> > > >> > > > > > > > > >> > > > >> who are not able to validate the data
> with
> > > >> > external
> > > >> > > > > > > > resources,
> > > >> > > > > > > > > >> > manual
> > > >> > > > > > > > > >> > > > >> intervention does not give a better
> result
> > > >> but a
> > > >> > > loss
> > > >> > > > > of
> > > >> > > > > > > > > >> > availability.
> > > >> > > > > > > > > >> > > > So
> > > >> > > > > > > > > >> > > > >> practically speaking, the Balance mode
> > would
> > > >> be a
> > > >> > > > > better
> > > >> > > > > > > > > default
> > > >> > > > > > > > > >> > > value.
> > > >> > > > > > > > > >> > > > >>
> > > >> > > > > > > > > >> > > > >> 41. No, it represents how we want to do
> > the
> > > >> > unclean
> > > >> > > > > > leader
> > > >> > > > > > > > > >> election.
> > > >> > > > > > > > > >> > > If
> > > >> > > > > > > > > >> > > > it
> > > >> > > > > > > > > >> > > > >> is false, the unclean leader election
> will
> > > be
> > > >> the
> > > >> > > old
> > > >> > > > > > > random
> > > >> > > > > > > > > way.
> > > >> > > > > > > > > >> > > > >> Otherwise, the unclean recovery will be
> > > used.
> > > >> > > > > > > > > >> > > > >>
> > > >> > > > > > > > > >> > > > >> 42. Good catch. Updated.
> > > >> > > > > > > > > >> > > > >>
> > > >> > > > > > > > > >> > > > >> 43. Only the first 20 topics will be
> > served.
> > > >> > Others
> > > >> > > > > will
> > > >> > > > > > be
> > > >> > > > > > > > > >> returned
> > > >> > > > > > > > > >> > > > with
> > > >> > > > > > > > > >> > > > >> InvalidRequestError
> > > >> > > > > > > > > >> > > > >>
> > > >> > > > > > > > > >> > > > >> 44. The order matters. The desired
> leader
> > > >> entries
> > > >> > > > match
> > > >> > > > > > > with
> > > >> > > > > > > > > the
> > > >> > > > > > > > > >> > topic
> > > >> > > > > > > > > >> > > > >> partition list by the index.
> > > >> > > > > > > > > >> > > > >>
> > > >> > > > > > > > > >> > > > >> 45. Thanks! Updated.
> > > >> > > > > > > > > >> > > > >>
> > > >> > > > > > > > > >> > > > >> 46. Good advice! Updated.
> > > >> > > > > > > > > >> > > > >>
> > > >> > > > > > > > > >> > > > >> 47.1, updated the comment. Basically it
> > will
> > > >> > elect
> > > >> > > > the
> > > >> > > > > > > > replica
> > > >> > > > > > > > > in
> > > >> > > > > > > > > >> > the
> > > >> > > > > > > > > >> > > > >> desiredLeader field to be the leader
> > > >> > > > > > > > > >> > > > >>
> > > >> > > > > > > > > >> > > > >> 47.2 We can let the admin client do the
> > > >> > conversion.
> > > >> > > > > Using
> > > >> > > > > > > the
> > > >> > > > > > > > > >> > > > desiredLeader
> > > >> > > > > > > > > >> > > > >> field in the json format seems easier
> for
> > > >> users.
> > > >> > > > > > > > > >> > > > >>
> > > >> > > > > > > > > >> > > > >> 48. Once the MV version is downgraded,
> all
> > > the
> > > >> > ELR
> > > >> > > > > > related
> > > >> > > > > > > > > fields
> > > >> > > > > > > > > >> > will
> > > >> > > > > > > > > >> > > > be
> > > >> > > > > > > > > >> > > > >> removed on the next partition change.
> The
> > > >> > > controller
> > > >> > > > > will
> > > >> > > > > > > > also
> > > >> > > > > > > > > >> > ignore
> > > >> > > > > > > > > >> > > > the
> > > >> > > > > > > > > >> > > > >> ELR fields. Updated the KIP.
> > > >> > > > > > > > > >> > > > >>
> > > >> > > > > > > > > >> > > > >> 49. Yes, it would be deprecated/removed.
> > > >> > > > > > > > > >> > > > >>
> > > >> > > > > > > > > >> > > > >>
> > > >> > > > > > > > > >> > > > >> On Mon, Sep 25, 2023 at 3:49 PM Jun Rao
> > > >> > > > > > > > > <jun@confluent.io.invalid
> > > >> > > > > > > > > >> >
> > > >> > > > > > > > > >> > > > wrote:
> > > >> > > > > > > > > >> > > > >>
> > > >> > > > > > > > > >> > > > >> > Hi, Calvin,
> > > >> > > > > > > > > >> > > > >> >
> > > >> > > > > > > > > >> > > > >> > Thanks for the updated KIP. Made
> another
> > > >> pass.
> > > >> > A
> > > >> > > > few
> > > >> > > > > > more
> > > >> > > > > > > > > >> comments
> > > >> > > > > > > > > >> > > > below.
> > > >> > > > > > > > > >> > > > >> >
> > > >> > > > > > > > > >> > > > >> > 40.
> unclean.leader.election.enable.false
> > > ->
> > > >> > > > > > > > > >> > > > >> > unclean.recovery.strategy.Balanced:
> The
> > > >> > Balanced
> > > >> > > > mode
> > > >> > > > > > > could
> > > >> > > > > > > > > >> still
> > > >> > > > > > > > > >> > > > lead to
> > > >> > > > > > > > > >> > > > >> > data loss. So, I am wondering if
> > > >> > > > > > > > > >> > > unclean.leader.election.enable.false
> > > >> > > > > > > > > >> > > > >> > should map to None?
> > > >> > > > > > > > > >> > > > >> >
> > > >> > > > > > > > > >> > > > >> > 41. unclean.recovery.manager.enabled:
> I
> > am
> > > >> not
> > > >> > > sure
> > > >> > > > > why
> > > >> > > > > > > we
> > > >> > > > > > > > > >> > introduce
> > > >> > > > > > > > > >> > > > this
> > > >> > > > > > > > > >> > > > >> > additional config. Is it the same as
> > > >> > > > > > > > > >> > unclean.recovery.strategy=None?
> > > >> > > > > > > > > >> > > > >> >
> > > >> > > > > > > > > >> > > > >> > 42.
> > > >> > > > DescribeTopicResponse.TopicAuthorizedOperations:
> > > >> > > > > > > Should
> > > >> > > > > > > > > >> this
> > > >> > > > > > > > > >> > be
> > > >> > > > > > > > > >> > > at
> > > >> > > > > > > > > >> > > > >> the
> > > >> > > > > > > > > >> > > > >> > topic level?
> > > >> > > > > > > > > >> > > > >> >
> > > >> > > > > > > > > >> > > > >> > 43. "Limit: 20 topics max per
> request":
> > > >> Could
> > > >> > we
> > > >> > > > > > describe
> > > >> > > > > > > > > what
> > > >> > > > > > > > > >> > > > happens if
> > > >> > > > > > > > > >> > > > >> > the request includes more than 20
> > topics?
> > > >> > > > > > > > > >> > > > >> >
> > > >> > > > > > > > > >> > > > >> > 44.
> ElectLeadersRequest.DesiredLeaders:
> > > >> Could
> > > >> > we
> > > >> > > > > > describe
> > > >> > > > > > > > > >> whether
> > > >> > > > > > > > > >> > > the
> > > >> > > > > > > > > >> > > > >> > ordering matters?
> > > >> > > > > > > > > >> > > > >> >
> > > >> > > > > > > > > >> > > > >> > 45. GetReplicaLogInfo.TopicPartitions:
> > > >> "about":
> > > >> > > > "The
> > > >> > > > > > > topic
> > > >> > > > > > > > > >> > > partitions
> > > >> > > > > > > > > >> > > > to
> > > >> > > > > > > > > >> > > > >> > elect leaders.": The description in
> > > "about"
> > > >> is
> > > >> > > > > > incorrect.
> > > >> > > > > > > > > >> > > > >> >
> > > >> > > > > > > > > >> > > > >> > 46. GetReplicaLogInfoResponse: Should
> we
> > > >> nest
> > > >> > > > > > partitions
> > > >> > > > > > > > > under
> > > >> > > > > > > > > >> > > > topicId to
> > > >> > > > > > > > > >> > > > >> > be consistent with other types of
> > > responses?
> > > >> > > > > > > > > >> > > > >> >
> > > >> > > > > > > > > >> > > > >> > 47. kafka-leader-election.sh:
> > > >> > > > > > > > > >> > > > >> > 47.1 Could we explain DESIGNATION?
> > > >> > > > > > > > > >> > > > >> > 47.2 desiredLeader: Should it be a
> list
> > to
> > > >> > match
> > > >> > > > the
> > > >> > > > > > > field
> > > >> > > > > > > > in
> > > >> > > > > > > > > >> > > > >> > ElectLeadersRequest?
> > > >> > > > > > > > > >> > > > >> >
> > > >> > > > > > > > > >> > > > >> > 48. We could add a section on
> downgrade?
> > > >> > > > > > > > > >> > > > >> >
> > > >> > > > > > > > > >> > > > >> > 49. LastKnownLeader: This seems only
> > > needed
> > > >> in
> > > >> > > the
> > > >> > > > > > first
> > > >> > > > > > > > > phase
> > > >> > > > > > > > > >> of
> > > >> > > > > > > > > >> > > > >> > delivering ELR. Will it be removed
> when
> > > the
> > > >> > > > complete
> > > >> > > > > > KIP
> > > >> > > > > > > is
> > > >> > > > > > > > > >> > > delivered?
> > > >> > > > > > > > > >> > > > >> >
> > > >> > > > > > > > > >> > > > >> > Thanks,
> > > >> > > > > > > > > >> > > > >> >
> > > >> > > > > > > > > >> > > > >> > Jun
> > > >> > > > > > > > > >> > > > >> >
> > > >> > > > > > > > > >> > > > >> > On Tue, Sep 19, 2023 at 1:30 PM Colin
> > > >> McCabe <
> > > >> > > > > > > > > >> cmccabe@apache.org>
> > > >> > > > > > > > > >> > > > wrote:
> > > >> > > > > > > > > >> > > > >> >
> > > >> > > > > > > > > >> > > > >> > > Hi Calvin,
> > > >> > > > > > > > > >> > > > >> > >
> > > >> > > > > > > > > >> > > > >> > > Thanks for the explanations. I like
> > the
> > > >> idea
> > > >> > of
> > > >> > > > > using
> > > >> > > > > > > > none,
> > > >> > > > > > > > > >> > > > balanced,
> > > >> > > > > > > > > >> > > > >> > > aggressive. We also had an offline
> > > >> discussion
> > > >> > > > about
> > > >> > > > > > why
> > > >> > > > > > > > it
> > > >> > > > > > > > > is
> > > >> > > > > > > > > >> > good
> > > >> > > > > > > > > >> > > > to
> > > >> > > > > > > > > >> > > > >> > use a
> > > >> > > > > > > > > >> > > > >> > > new config key (basically, so that
> we
> > > can
> > > >> > > > deprecate
> > > >> > > > > > the
> > > >> > > > > > > > old
> > > >> > > > > > > > > >> one
> > > >> > > > > > > > > >> > > > which
> > > >> > > > > > > > > >> > > > >> had
> > > >> > > > > > > > > >> > > > >> > > only false/true values in 4.0) With
> > > these
> > > >> > > > changes,
> > > >> > > > > I
> > > >> > > > > > am
> > > >> > > > > > > > +1.
> > > >> > > > > > > > > >> > > > >> > >
> > > >> > > > > > > > > >> > > > >> > > best,
> > > >> > > > > > > > > >> > > > >> > > Colin
> > > >> > > > > > > > > >> > > > >> > >
> > > >> > > > > > > > > >> > > > >> > > On Mon, Sep 18, 2023, at 15:54,
> Calvin
> > > Liu
> > > >> > > wrote:
> > > >> > > > > > > > > >> > > > >> > > > Hi Colin,
> > > >> > > > > > > > > >> > > > >> > > > Also, can we deprecate
> > > >> > > > > > unclean.leader.election.enable
> > > >> > > > > > > > in
> > > >> > > > > > > > > >> 4.0?
> > > >> > > > > > > > > >> > > > Before
> > > >> > > > > > > > > >> > > > >> > > that,
> > > >> > > > > > > > > >> > > > >> > > > we can have both the config
> > > >> > > > > > unclean.recovery.strategy
> > > >> > > > > > > > and
> > > >> > > > > > > > > >> > > > >> > > > unclean.leader.election.enable
> > > >> > > > > > > > > >> > > > >> > > > and using the
> > unclean.recovery.Enabled
> > > >> to
> > > >> > > > > determine
> > > >> > > > > > > > which
> > > >> > > > > > > > > >> > config
> > > >> > > > > > > > > >> > > > to
> > > >> > > > > > > > > >> > > > >> use
> > > >> > > > > > > > > >> > > > >> > > > during the unclean leader
> election.
> > > >> > > > > > > > > >> > > > >> > > >
> > > >> > > > > > > > > >> > > > >> > > > On Mon, Sep 18, 2023 at 3:51 PM
> > Calvin
> > > >> Liu
> > > >> > <
> > > >> > > > > > > > > >> > caliu@confluent.io>
> > > >> > > > > > > > > >> > > > >> wrote:
> > > >> > > > > > > > > >> > > > >> > > >
> > > >> > > > > > > > > >> > > > >> > > >> Hi Colin,
> > > >> > > > > > > > > >> > > > >> > > >> For the unclean.recovery.strategy
> > > >> config
> > > >> > > name,
> > > >> > > > > how
> > > >> > > > > > > > about
> > > >> > > > > > > > > >> we
> > > >> > > > > > > > > >> > use
> > > >> > > > > > > > > >> > > > the
> > > >> > > > > > > > > >> > > > >> > > >> following
> > > >> > > > > > > > > >> > > > >> > > >> None. It basically means no
> unclean
> > > >> > recovery
> > > >> > > > > will
> > > >> > > > > > be
> > > >> > > > > > > > > >> > performed.
> > > >> > > > > > > > > >> > > > >> > > >> Aggressive. It means availability
> > > goes
> > > >> > > first.
> > > >> > > > > > > Whenever
> > > >> > > > > > > > > the
> > > >> > > > > > > > > >> > > > partition
> > > >> > > > > > > > > >> > > > >> > > can't
> > > >> > > > > > > > > >> > > > >> > > >> elect a durable replica, the
> > > controller
> > > >> > will
> > > >> > > > try
> > > >> > > > > > the
> > > >> > > > > > > > > >> unclean
> > > >> > > > > > > > > >> > > > >> recovery.
> > > >> > > > > > > > > >> > > > >> > > >> Balanced. It is the balance point
> > of
> > > >> the
> > > >> > > > > > > availability
> > > >> > > > > > > > > >> > > > >> > first(Aggressive)
> > > >> > > > > > > > > >> > > > >> > > >> and least availability(None). The
> > > >> > controller
> > > >> > > > > > > performs
> > > >> > > > > > > > > >> unclean
> > > >> > > > > > > > > >> > > > >> recovery
> > > >> > > > > > > > > >> > > > >> > > when
> > > >> > > > > > > > > >> > > > >> > > >> both ISR and ELR are empty.
> > > >> > > > > > > > > >> > > > >> > > >>
> > > >> > > > > > > > > >> > > > >> > > >>
> > > >> > > > > > > > > >> > > > >> > > >> On Fri, Sep 15, 2023 at 11:42 AM
> > > Calvin
> > > >> > Liu
> > > >> > > <
> > > >> > > > > > > > > >> > > caliu@confluent.io>
> > > >> > > > > > > > > >> > > > >> > wrote:
> > > >> > > > > > > > > >> > > > >> > > >>
> > > >> > > > > > > > > >> > > > >> > > >>> Hi Colin,
> > > >> > > > > > > > > >> > > > >> > > >>>
> > > >> > > > > > > > > >> > > > >> > > >>> > So, the proposal is that if
> > > someone
> > > >> > sets
> > > >> > > > > > > > > >> > > > >> > > "unclean.leader.election.enable
> > > >> > > > > > > > > >> > > > >> > > >>> = true"...
> > > >> > > > > > > > > >> > > > >> > > >>>
> > > >> > > > > > > > > >> > > > >> > > >>>
> > > >> > > > > > > > > >> > > > >> > > >>> The idea is to use one of the
> > > >> > > > > > > > > >> unclean.leader.election.enable
> > > >> > > > > > > > > >> > > and
> > > >> > > > > > > > > >> > > > >> > > >>> unclean.recovery.strategy based
> on
> > > the
> > > >> > > > > > > > > >> > > > unclean.recovery.Enabled. A
> > > >> > > > > > > > > >> > > > >> > > possible
> > > >> > > > > > > > > >> > > > >> > > >>> version can be
> > > >> > > > > > > > > >> > > > >> > > >>>
> > > >> > > > > > > > > >> > > > >> > > >>> If unclean.recovery.Enabled:
> > > >> > > > > > > > > >> > > > >> > > >>>
> > > >> > > > > > > > > >> > > > >> > > >>> {
> > > >> > > > > > > > > >> > > > >> > > >>>
> > > >> > > > > > > > > >> > > > >> > > >>> Check unclean.recovery.strategy.
> > If
> > > >> set,
> > > >> > > use
> > > >> > > > > it.
> > > >> > > > > > > > > >> Otherwise,
> > > >> > > > > > > > > >> > > > check
> > > >> > > > > > > > > >> > > > >> > > >>> unclean.leader.election.enable
> and
> > > >> > > translate
> > > >> > > > it
> > > >> > > > > > to
> > > >> > > > > > > > > >> > > > >> > > >>> unclean.recovery.strategy.
> > > >> > > > > > > > > >> > > > >> > > >>>
> > > >> > > > > > > > > >> > > > >> > > >>> } else {
> > > >> > > > > > > > > >> > > > >> > > >>>
> > > >> > > > > > > > > >> > > > >> > > >>> Use
> unclean.leader.election.enable
> > > >> > > > > > > > > >> > > > >> > > >>>
> > > >> > > > > > > > > >> > > > >> > > >>> }
> > > >> > > > > > > > > >> > > > >> > > >>>
> > > >> > > > > > > > > >> > > > >> > > >>>
> > > >> > > > > > > > > >> > > > >> > > >>> —--------
> > > >> > > > > > > > > >> > > > >> > > >>>
> > > >> > > > > > > > > >> > > > >> > > >>> >The configuration key should be
> > > >> > > > > > > > > >> > > > >> "unclean.recovery.manager.enabled",
> > > >> > > > > > > > > >> > > > >> > > >>> right?
> > > >> > > > > > > > > >> > > > >> > > >>>
> > > >> > > > > > > > > >> > > > >> > > >>>
> > > >> > > > > > > > > >> > > > >> > > >>> I think we have two ways of
> > > choosing a
> > > >> > > leader
> > > >> > > > > > > > > uncleanly,
> > > >> > > > > > > > > >> > > unclean
> > > >> > > > > > > > > >> > > > >> > leader
> > > >> > > > > > > > > >> > > > >> > > >>> election and unclean
> recovery(log
> > > >> > > inspection)
> > > >> > > > > and
> > > >> > > > > > > we
> > > >> > > > > > > > > try
> > > >> > > > > > > > > >> to
> > > >> > > > > > > > > >> > > > switch
> > > >> > > > > > > > > >> > > > >> > > between
> > > >> > > > > > > > > >> > > > >> > > >>> them.
> > > >> > > > > > > > > >> > > > >> > > >>>
> > > >> > > > > > > > > >> > > > >> > > >>> Do you mean we want to develop
> two
> > > >> ways
> > > >> > of
> > > >> > > > > > > performing
> > > >> > > > > > > > > the
> > > >> > > > > > > > > >> > > > unclean
> > > >> > > > > > > > > >> > > > >> > > >>> recovery and one of them is
> using
> > > >> > “unclean
> > > >> > > > > > recovery
> > > >> > > > > > > > > >> > manager”?
> > > >> > > > > > > > > >> > > I
> > > >> > > > > > > > > >> > > > >> guess
> > > >> > > > > > > > > >> > > > >> > > we
> > > >> > > > > > > > > >> > > > >> > > >>> haven’t discussed the second
> way.
> > > >> > > > > > > > > >> > > > >> > > >>>
> > > >> > > > > > > > > >> > > > >> > > >>>
> > > >> > > > > > > > > >> > > > >> > > >>> —-------
> > > >> > > > > > > > > >> > > > >> > > >>>
> > > >> > > > > > > > > >> > > > >> > > >>> >How do these 4 levels of
> > overrides
> > > >> > > interact
> > > >> > > > > with
> > > >> > > > > > > > your
> > > >> > > > > > > > > >> new
> > > >> > > > > > > > > >> > > > >> > > >>> configurations?
> > > >> > > > > > > > > >> > > > >> > > >>>
> > > >> > > > > > > > > >> > > > >> > > >>>
> > > >> > > > > > > > > >> > > > >> > > >>> I do notice in the Kraft
> > controller
> > > >> code,
> > > >> > > the
> > > >> > > > > > > method
> > > >> > > > > > > > to
> > > >> > > > > > > > > >> > check
> > > >> > > > > > > > > >> > > > >> whether
> > > >> > > > > > > > > >> > > > >> > > >>> perform unclean leader election
> is
> > > >> hard
> > > >> > > coded
> > > >> > > > > to
> > > >> > > > > > > > false
> > > >> > > > > > > > > >> since
> > > >> > > > > > > > > >> > > > >> > > >>>
> > > >> > 2021(uncleanLeaderElectionEnabledForTopic).
> > > >> > > > > Isn’t
> > > >> > > > > > > it
> > > >> > > > > > > > a
> > > >> > > > > > > > > >> good
> > > >> > > > > > > > > >> > > > chance
> > > >> > > > > > > > > >> > > > >> to
> > > >> > > > > > > > > >> > > > >> > > >>> completely deprecate the
> > > >> > > > > > > > > unclean.leader.election.enable?
> > > >> > > > > > > > > >> We
> > > >> > > > > > > > > >> > > > don’t
> > > >> > > > > > > > > >> > > > >> > even
> > > >> > > > > > > > > >> > > > >> > > have
> > > >> > > > > > > > > >> > > > >> > > >>> to worry about the config
> > > conversion.
> > > >> > > > > > > > > >> > > > >> > > >>>
> > > >> > > > > > > > > >> > > > >> > > >>> On the other hand, whatever the
> > > >> override
> > > >> > > is,
> > > >> > > > as
> > > >> > > > > > > long
> > > >> > > > > > > > as
> > > >> > > > > > > > > >> the
> > > >> > > > > > > > > >> > > > >> > controller
> > > >> > > > > > > > > >> > > > >> > > >>> can have the final effective
> > > >> > > > > > > > > >> unclean.leader.election.enable,
> > > >> > > > > > > > > >> > > the
> > > >> > > > > > > > > >> > > > >> > topic
> > > >> > > > > > > > > >> > > > >> > > >>> level config
> > > >> unclean.recovery.strategy,
> > > >> > the
> > > >> > > > > > cluster
> > > >> > > > > > > > > level
> > > >> > > > > > > > > >> > > config
> > > >> > > > > > > > > >> > > > >> > > >>> unclean.recovery.Enabled, the
> > > >> controller
> > > >> > > can
> > > >> > > > > > > > calculate
> > > >> > > > > > > > > >> the
> > > >> > > > > > > > > >> > > > correct
> > > >> > > > > > > > > >> > > > >> > > methods
> > > >> > > > > > > > > >> > > > >> > > >>> to use right?
> > > >> > > > > > > > > >> > > > >> > > >>>
> > > >> > > > > > > > > >> > > > >> > > >>>
> > > >> > > > > > > > > >> > > > >> > > >>> On Fri, Sep 15, 2023 at 10:02 AM
> > > Colin
> > > >> > > > McCabe <
> > > >> > > > > > > > > >> > > > cmccabe@apache.org>
> > > >> > > > > > > > > >> > > > >> > > wrote:
> > > >> > > > > > > > > >> > > > >> > > >>>
> > > >> > > > > > > > > >> > > > >> > > >>>> On Thu, Sep 14, 2023, at 22:23,
> > > >> Calvin
> > > >> > Liu
> > > >> > > > > > wrote:
> > > >> > > > > > > > > >> > > > >> > > >>>> > Hi Colin
> > > >> > > > > > > > > >> > > > >> > > >>>> > 1. I think using the new
> config
> > > >> name
> > > >> > is
> > > >> > > > more
> > > >> > > > > > > > clear.
> > > >> > > > > > > > > >> > > > >> > > >>>> >        a. The unclean leader
> > > >> election
> > > >> > is
> > > >> > > > > > > actually
> > > >> > > > > > > > > >> removed
> > > >> > > > > > > > > >> > > if
> > > >> > > > > > > > > >> > > > >> > unclean
> > > >> > > > > > > > > >> > > > >> > > >>>> > recovery is in use.
> > > >> > > > > > > > > >> > > > >> > > >>>> >        b. Using multiple
> values
> > > in
> > > >> > > > > > > > > >> > > > >> unclean.leader.election.enable
> > > >> > > > > > > > > >> > > > >> > is
> > > >> > > > > > > > > >> > > > >> > > >>>> > confusing and it will be more
> > > >> > confusing
> > > >> > > > > after
> > > >> > > > > > > > people
> > > >> > > > > > > > > >> > forget
> > > >> > > > > > > > > >> > > > >> about
> > > >> > > > > > > > > >> > > > >> > > this
> > > >> > > > > > > > > >> > > > >> > > >>>> > discussion.
> > > >> > > > > > > > > >> > > > >> > > >>>>
> > > >> > > > > > > > > >> > > > >> > > >>>> Hi Calvin,
> > > >> > > > > > > > > >> > > > >> > > >>>>
> > > >> > > > > > > > > >> > > > >> > > >>>> So, the proposal is that if
> > someone
> > > >> sets
> > > >> > > > > > > > > >> > > > >> > > "unclean.leader.election.enable
> > > >> > > > > > > > > >> > > > >> > > >>>> = true" but then sets one of
> your
> > > new
> > > >> > > > > > > > configurations,
> > > >> > > > > > > > > >> the
> > > >> > > > > > > > > >> > > > value of
> > > >> > > > > > > > > >> > > > >> > > >>>> unclean.leader.election.enable
> is
> > > >> > ignored?
> > > >> > > > > That
> > > >> > > > > > > > seems
> > > >> > > > > > > > > >> less
> > > >> > > > > > > > > >> > > > clear
> > > >> > > > > > > > > >> > > > >> to
> > > >> > > > > > > > > >> > > > >> > > me, not
> > > >> > > > > > > > > >> > > > >> > > >>>> more. Just in general, having
> > > >> multiple
> > > >> > > > > > > configuration
> > > >> > > > > > > > > >> keys
> > > >> > > > > > > > > >> > to
> > > >> > > > > > > > > >> > > > >> control
> > > >> > > > > > > > > >> > > > >> > > the
> > > >> > > > > > > > > >> > > > >> > > >>>> same thing confuses users.
> > > Basically,
> > > >> > they
> > > >> > > > are
> > > >> > > > > > > > sitting
> > > >> > > > > > > > > >> at a
> > > >> > > > > > > > > >> > > > giant
> > > >> > > > > > > > > >> > > > >> > > control
> > > >> > > > > > > > > >> > > > >> > > >>>> panel, and some of the levers
> do
> > > >> > nothing.
> > > >> > > > > > > > > >> > > > >> > > >>>>
> > > >> > > > > > > > > >> > > > >> > > >>>> > 2. Sorry I forgot to mention
> in
> > > the
> > > >> > > > response
> > > >> > > > > > > that
> > > >> > > > > > > > I
> > > >> > > > > > > > > >> did
> > > >> > > > > > > > > >> > add
> > > >> > > > > > > > > >> > > > the
> > > >> > > > > > > > > >> > > > >> > > >>>> > unclean.recovery.Enabled
> flag.
> > > >> > > > > > > > > >> > > > >> > > >>>>
> > > >> > > > > > > > > >> > > > >> > > >>>> The configuration key should be
> > > >> > > > > > > > > >> > > > >> "unclean.recovery.manager.enabled",
> > > >> > > > > > > > > >> > > > >> > > >>>> right? Becuase we can do
> "unclean
> > > >> > > recovery"
> > > >> > > > > > > without
> > > >> > > > > > > > > the
> > > >> > > > > > > > > >> > > > manager.
> > > >> > > > > > > > > >> > > > >> > > Disabling
> > > >> > > > > > > > > >> > > > >> > > >>>> the manager just means we use a
> > > >> > different
> > > >> > > > > > > mechanism
> > > >> > > > > > > > > for
> > > >> > > > > > > > > >> > > > recovery.
> > > >> > > > > > > > > >> > > > >> > > >>>>
> > > >> > > > > > > > > >> > > > >> > > >>>> >        c. Maybe I
> > underestimated
> > > >> the
> > > >> > > > > challenge
> > > >> > > > > > > of
> > > >> > > > > > > > > >> > replacing
> > > >> > > > > > > > > >> > > > the
> > > >> > > > > > > > > >> > > > >> > > >>>> config. Any
> > > >> > > > > > > > > >> > > > >> > > >>>> > implementation problems
> ahead?
> > > >> > > > > > > > > >> > > > >> > > >>>>
> > > >> > > > > > > > > >> > > > >> > > >>>> There are four levels of
> > overrides
> > > >> for
> > > >> > > > > > > > > >> > > > >> > unclean.leader.election.enable.
> > > >> > > > > > > > > >> > > > >> > > >>>>
> > > >> > > > > > > > > >> > > > >> > > >>>> 1. static configuration for
> node.
> > > >> > > > > > > > > >> > > > >> > > >>>>     This goes in the
> > configuration
> > > >> file,
> > > >> > > > > > typically
> > > >> > > > > > > > > named
> > > >> > > > > > > > > >> > > > >> > > >>>> server.properties
> > > >> > > > > > > > > >> > > > >> > > >>>>
> > > >> > > > > > > > > >> > > > >> > > >>>> 2. dynamic configuration for
> node
> > > >> > default
> > > >> > > > > > > > > >> > > > >> > > >>>>   ConfigResource(type=BROKER,
> > > >> name="")
> > > >> > > > > > > > > >> > > > >> > > >>>>
> > > >> > > > > > > > > >> > > > >> > > >>>> 3. dynamic configuration for
> node
> > > >> > > > > > > > > >> > > > >> > > >>>>   ConfigResource(type=BROKER,
> > > >> > > > name=<controller
> > > >> > > > > > > id>)
> > > >> > > > > > > > > >> > > > >> > > >>>>
> > > >> > > > > > > > > >> > > > >> > > >>>> 4. dynamic configuration for
> > topic
> > > >> > > > > > > > > >> > > > >> > > >>>>   ConfigResource(type=TOPIC,
> > > >> > > > > name=<topic-name>)
> > > >> > > > > > > > > >> > > > >> > > >>>>
> > > >> > > > > > > > > >> > > > >> > > >>>> How do these 4 levels of
> > overrides
> > > >> > > interact
> > > >> > > > > with
> > > >> > > > > > > > your
> > > >> > > > > > > > > >> new
> > > >> > > > > > > > > >> > > > >> > > >>>> configurations? If the new
> > > >> > configurations
> > > >> > > > > > dominate
> > > >> > > > > > > > > over
> > > >> > > > > > > > > >> the
> > > >> > > > > > > > > >> > > old
> > > >> > > > > > > > > >> > > > >> > ones,
> > > >> > > > > > > > > >> > > > >> > > it
> > > >> > > > > > > > > >> > > > >> > > >>>> seems like this will get a lot
> > more
> > > >> > > > confusing
> > > >> > > > > to
> > > >> > > > > > > > > >> implement
> > > >> > > > > > > > > >> > > (and
> > > >> > > > > > > > > >> > > > >> also
> > > >> > > > > > > > > >> > > > >> > > to
> > > >> > > > > > > > > >> > > > >> > > >>>> use.)
> > > >> > > > > > > > > >> > > > >> > > >>>>
> > > >> > > > > > > > > >> > > > >> > > >>>> Again, I'd recommend just
> adding
> > > some
> > > >> > new
> > > >> > > > > values
> > > >> > > > > > > to
> > > >> > > > > > > > > >> > > > >> > > >>>> unclean.leader.election.enable.
> > > It's
> > > >> > > simple
> > > >> > > > > and
> > > >> > > > > > > will
> > > >> > > > > > > > > >> > prevent
> > > >> > > > > > > > > >> > > > user
> > > >> > > > > > > > > >> > > > >> > > confusion
> > > >> > > > > > > > > >> > > > >> > > >>>> (as well as developer
> confusion.)
> > > >> > > > > > > > > >> > > > >> > > >>>>
> > > >> > > > > > > > > >> > > > >> > > >>>> best,
> > > >> > > > > > > > > >> > > > >> > > >>>> Colin
> > > >> > > > > > > > > >> > > > >> > > >>>>
> > > >> > > > > > > > > >> > > > >> > > >>>>
> > > >> > > > > > > > > >> > > > >> > > >>>> > 3. About the admin client, I
> > > >> > mentioned 3
> > > >> > > > > > changes
> > > >> > > > > > > > in
> > > >> > > > > > > > > >> the
> > > >> > > > > > > > > >> > > > client.
> > > >> > > > > > > > > >> > > > >> > > >>>> Anything
> > > >> > > > > > > > > >> > > > >> > > >>>> > else I missed in the KIP?
> > > >> > > > > > > > > >> > > > >> > > >>>> >       a. The client will
> switch
> > > to
> > > >> > using
> > > >> > > > the
> > > >> > > > > > new
> > > >> > > > > > > > RPC
> > > >> > > > > > > > > >> > > instead
> > > >> > > > > > > > > >> > > > of
> > > >> > > > > > > > > >> > > > >> > > >>>> > MetadataRequest for the
> topics.
> > > >> > > > > > > > > >> > > > >> > > >>>> >       b. The
> TopicPartitionInfo
> > > >> used
> > > >> > in
> > > >> > > > > > > > > >> TopicDescription
> > > >> > > > > > > > > >> > > > needs
> > > >> > > > > > > > > >> > > > >> to
> > > >> > > > > > > > > >> > > > >> > > add
> > > >> > > > > > > > > >> > > > >> > > >>>> new
> > > >> > > > > > > > > >> > > > >> > > >>>> > fields related to the ELR.
> > > >> > > > > > > > > >> > > > >> > > >>>> >       c. The outputs will add
> > the
> > > >> ELR
> > > >> > > > > related
> > > >> > > > > > > > > fields.
> > > >> > > > > > > > > >> > > > >> > > >>>> >
> > > >> > > > > > > > > >> > > > >> > > >>>> > On Thu, Sep 14, 2023 at
> 9:19 PM
> > > >> Colin
> > > >> > > > > McCabe <
> > > >> > > > > > > > > >> > > > >> cmccabe@apache.org>
> > > >> > > > > > > > > >> > > > >> > > >>>> wrote:
> > > >> > > > > > > > > >> > > > >> > > >>>> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> Hi Calvin,
> > > >> > > > > > > > > >> > > > >> > > >>>> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> Thanks for the changes.
> > > >> > > > > > > > > >> > > > >> > > >>>> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> 1. Earlier I commented that
> > > >> creating
> > > >> > > > > > > > > >> > > > >> "unclean.recovery.strategy "
> > > >> > > > > > > > > >> > > > >> > > is
> > > >> > > > > > > > > >> > > > >> > > >>>> not
> > > >> > > > > > > > > >> > > > >> > > >>>> >> necessary, and we can just
> > reuse
> > > >> the
> > > >> > > > > existing
> > > >> > > > > > > > > >> > > > >> > > >>>> >>
> > "unclean.leader.election.enable"
> > > >> > > > > > configuration
> > > >> > > > > > > > key.
> > > >> > > > > > > > > >> > Let's
> > > >> > > > > > > > > >> > > > >> discuss
> > > >> > > > > > > > > >> > > > >> > > >>>> that.
> > > >> > > > > > > > > >> > > > >> > > >>>> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> 2.I also don't understand
> why
> > > you
> > > >> > > didn't
> > > >> > > > > add
> > > >> > > > > > a
> > > >> > > > > > > > > >> > > > configuration to
> > > >> > > > > > > > > >> > > > >> > > >>>> enable or
> > > >> > > > > > > > > >> > > > >> > > >>>> >> disable the Unclean Recovery
> > > >> Manager.
> > > >> > > > This
> > > >> > > > > > > seems
> > > >> > > > > > > > > >> like a
> > > >> > > > > > > > > >> > > very
> > > >> > > > > > > > > >> > > > >> > simple
> > > >> > > > > > > > > >> > > > >> > > >>>> way to
> > > >> > > > > > > > > >> > > > >> > > >>>> >> handle the staging issue
> which
> > > we
> > > >> > > > > discussed.
> > > >> > > > > > > The
> > > >> > > > > > > > > URM
> > > >> > > > > > > > > >> can
> > > >> > > > > > > > > >> > > > just
> > > >> > > > > > > > > >> > > > >> be
> > > >> > > > > > > > > >> > > > >> > > >>>> turned off
> > > >> > > > > > > > > >> > > > >> > > >>>> >> until it is production
> ready.
> > > >> Let's
> > > >> > > > discuss
> > > >> > > > > > > this.
> > > >> > > > > > > > > >> > > > >> > > >>>> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> 3. You still need to
> describe
> > > the
> > > >> > > changes
> > > >> > > > > to
> > > >> > > > > > > > > >> AdminClient
> > > >> > > > > > > > > >> > > > that
> > > >> > > > > > > > > >> > > > >> are
> > > >> > > > > > > > > >> > > > >> > > >>>> needed
> > > >> > > > > > > > > >> > > > >> > > >>>> >> to use DescribeTopicRequest.
> > > >> > > > > > > > > >> > > > >> > > >>>> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> Keep at it. It's looking
> > better.
> > > >> :)
> > > >> > > > > > > > > >> > > > >> > > >>>> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> best,
> > > >> > > > > > > > > >> > > > >> > > >>>> >> Colin
> > > >> > > > > > > > > >> > > > >> > > >>>> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> On Thu, Sep 14, 2023, at
> > 11:03,
> > > >> > Calvin
> > > >> > > > Liu
> > > >> > > > > > > wrote:
> > > >> > > > > > > > > >> > > > >> > > >>>> >> > Hi Colin
> > > >> > > > > > > > > >> > > > >> > > >>>> >> > Thanks for the comments!
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> > I did the following
> changes
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    1.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    Simplified the API spec
> > > >> section
> > > >> > to
> > > >> > > > > only
> > > >> > > > > > > > > include
> > > >> > > > > > > > > >> the
> > > >> > > > > > > > > >> > > > diff.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    2.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    Reordered the HWM
> > > requirement
> > > >> > > > section.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    3.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    Removed the URM
> > > >> implementation
> > > >> > > > details
> > > >> > > > > > to
> > > >> > > > > > > > keep
> > > >> > > > > > > > > >> the
> > > >> > > > > > > > > >> > > > >> necessary
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    characteristics to
> > perform
> > > >> the
> > > >> > > > unclean
> > > >> > > > > > > > > recovery.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    1.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >       When to perform the
> > > >> unclean
> > > >> > > > > recovery
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >       2.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >       Under different
> > config,
> > > >> how
> > > >> > the
> > > >> > > > > > unclean
> > > >> > > > > > > > > >> recovery
> > > >> > > > > > > > > >> > > > finds
> > > >> > > > > > > > > >> > > > >> > the
> > > >> > > > > > > > > >> > > > >> > > >>>> leader.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >       3.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >       How the config
> > > >> > > > > > > > > unclean.leader.election.enable
> > > >> > > > > > > > > >> > and
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > >  unclean.recovery.strategy
> > > >> are
> > > >> > > > > > converted
> > > >> > > > > > > > > when
> > > >> > > > > > > > > >> > users
> > > >> > > > > > > > > >> > > > >> > > >>>> enable/disable
> > > >> > > > > > > > > >> > > > >> > > >>>> >> the
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >       unclean recovery.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >       4.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    More details about how
> we
> > > >> change
> > > >> > > > admin
> > > >> > > > > > > > client.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    5.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    API limits on the
> > > >> > > > > > GetReplicaLogInfoRequest
> > > >> > > > > > > > and
> > > >> > > > > > > > > >> > > > >> > > >>>> DescribeTopicRequest.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    6.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    Two metrics added
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >    1.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > > >> > > > > > > > > >> > > >
> > > >> > Kafka.controller.global_under_min_isr_partition_count
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >       2.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > > >> > > > > > > > > >>  kafka.controller.unclean_recovery_finished_count
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> > On Wed, Sep 13, 2023 at
> > > 10:46 AM
> > > >> > > Colin
> > > >> > > > > > > McCabe <
> > > >> > > > > > > > > >> > > > >> > > cmccabe@apache.org>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> wrote:
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> On Tue, Sep 12, 2023, at
> > > 17:21,
> > > >> > > Calvin
> > > >> > > > > Liu
> > > >> > > > > > > > > wrote:
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Hi Colin
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Thanks for the
> comments!
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> Hi Calvin,
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> Thanks again for the KIP.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> One meta-comment: it's
> > > usually
> > > >> > > better
> > > >> > > > to
> > > >> > > > > > > just
> > > >> > > > > > > > > do a
> > > >> > > > > > > > > >> > diff
> > > >> > > > > > > > > >> > > > on a
> > > >> > > > > > > > > >> > > > >> > > >>>> message
> > > >> > > > > > > > > >> > > > >> > > >>>> >> spec
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> file or java file if
> you're
> > > >> > > including
> > > >> > > > > > > changes
> > > >> > > > > > > > to
> > > >> > > > > > > > > >> it
> > > >> > > > > > > > > >> > in
> > > >> > > > > > > > > >> > > > the
> > > >> > > > > > > > > >> > > > >> > KIP.
> > > >> > > > > > > > > >> > > > >> > > >>>> This is
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> easier to read than
> looking
> > > for
> > > >> > "new
> > > >> > > > > > fields
> > > >> > > > > > > > > begin"
> > > >> > > > > > > > > >> > etc.
> > > >> > > > > > > > > >> > > > in
> > > >> > > > > > > > > >> > > > >> the
> > > >> > > > > > > > > >> > > > >> > > >>>> text, and
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> gracefully handles the
> case
> > > >> where
> > > >> > > > > existing
> > > >> > > > > > > > > fields
> > > >> > > > > > > > > >> > were
> > > >> > > > > > > > > >> > > > >> > changed.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Rewrite the Additional
> > High
> > > >> > > > Watermark
> > > >> > > > > > > > > >> advancement
> > > >> > > > > > > > > >> > > > >> > requirement
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > There was feedback on
> > this
> > > >> > section
> > > >> > > > > that
> > > >> > > > > > > some
> > > >> > > > > > > > > >> > readers
> > > >> > > > > > > > > >> > > > may
> > > >> > > > > > > > > >> > > > >> not
> > > >> > > > > > > > > >> > > > >> > > be
> > > >> > > > > > > > > >> > > > >> > > >>>> >> familiar
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > with HWM and
> Ack=0,1,all
> > > >> > requests.
> > > >> > > > > This
> > > >> > > > > > > can
> > > >> > > > > > > > > help
> > > >> > > > > > > > > >> > them
> > > >> > > > > > > > > >> > > > >> > > understand
> > > >> > > > > > > > > >> > > > >> > > >>>> the
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > proposal. I will
> rewrite
> > > this
> > > >> > part
> > > >> > > > for
> > > >> > > > > > > more
> > > >> > > > > > > > > >> > > > readability.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> To be clear, I wasn't
> > > >> suggesting
> > > >> > > > > dropping
> > > >> > > > > > > > either
> > > >> > > > > > > > > >> > > > section. I
> > > >> > > > > > > > > >> > > > >> > > agree
> > > >> > > > > > > > > >> > > > >> > > >>>> that
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> they add useful
> > background. I
> > > >> was
> > > >> > > just
> > > >> > > > > > > > > suggesting
> > > >> > > > > > > > > >> > that
> > > >> > > > > > > > > >> > > we
> > > >> > > > > > > > > >> > > > >> > should
> > > >> > > > > > > > > >> > > > >> > > >>>> discuss
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> the "acks" setting AFTER
> > > >> > discussing
> > > >> > > > the
> > > >> > > > > > new
> > > >> > > > > > > > high
> > > >> > > > > > > > > >> > > > watermark
> > > >> > > > > > > > > >> > > > >> > > >>>> advancement
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> conditions. We also
> should
> > > >> discuss
> > > >> > > > > acks=0.
> > > >> > > > > > > > While
> > > >> > > > > > > > > >> it
> > > >> > > > > > > > > >> > > isn't
> > > >> > > > > > > > > >> > > > >> > > >>>> conceptually
> > > >> > > > > > > > > >> > > > >> > > >>>> >> much
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> different than acks=1
> here,
> > > its
> > > >> > > > omission
> > > >> > > > > > > from
> > > >> > > > > > > > > this
> > > >> > > > > > > > > >> > > > section
> > > >> > > > > > > > > >> > > > >> is
> > > >> > > > > > > > > >> > > > >> > > >>>> confusing.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Unclean recovery
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > The plan is to replace
> > the
> > > >> > > > > > > > > >> > > > unclean.leader.election.enable
> > > >> > > > > > > > > >> > > > >> > with
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > unclean.recovery.strategy.
> > > If
> > > >> > the
> > > >> > > > > > Unclean
> > > >> > > > > > > > > >> Recovery
> > > >> > > > > > > > > >> > is
> > > >> > > > > > > > > >> > > > >> > enabled
> > > >> > > > > > > > > >> > > > >> > > >>>> then it
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> deals
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > with the three options
> in
> > > the
> > > >> > > > > > > > > >> > > > unclean.recovery.strategy.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Let’s refine the
> Unclean
> > > >> > Recovery.
> > > >> > > > We
> > > >> > > > > > have
> > > >> > > > > > > > > >> already
> > > >> > > > > > > > > >> > > > taken a
> > > >> > > > > > > > > >> > > > >> > > lot of
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > suggestions and I hope
> to
> > > >> > enhance
> > > >> > > > the
> > > >> > > > > > > > > >> durability of
> > > >> > > > > > > > > >> > > > Kafka
> > > >> > > > > > > > > >> > > > >> to
> > > >> > > > > > > > > >> > > > >> > > the
> > > >> > > > > > > > > >> > > > >> > > >>>> next
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> level
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > with this KIP.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> I am OK with doing the
> > > unclean
> > > >> > > leader
> > > >> > > > > > > recovery
> > > >> > > > > > > > > >> > > > improvements
> > > >> > > > > > > > > >> > > > >> in
> > > >> > > > > > > > > >> > > > >> > > >>>> this KIP.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> However, I think we need
> to
> > > >> really
> > > >> > > > work
> > > >> > > > > on
> > > >> > > > > > > the
> > > >> > > > > > > > > >> > > > configuration
> > > >> > > > > > > > > >> > > > >> > > >>>> settings.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> Configuration overrides
> are
> > > >> often
> > > >> > > > quite
> > > >> > > > > > > messy.
> > > >> > > > > > > > > For
> > > >> > > > > > > > > >> > > > example,
> > > >> > > > > > > > > >> > > > >> > the
> > > >> > > > > > > > > >> > > > >> > > >>>> cases
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> where we have
> > log.roll.hours
> > > >> and
> > > >> > > > > > > > > >> log.roll.segment.ms
> > > >> > > > > > > > > >> > ,
> > > >> > > > > > > > > >> > > > the
> > > >> > > > > > > > > >> > > > >> > user
> > > >> > > > > > > > > >> > > > >> > > >>>> has to
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> remember which one takes
> > > >> > precedence,
> > > >> > > > and
> > > >> > > > > > it
> > > >> > > > > > > is
> > > >> > > > > > > > > not
> > > >> > > > > > > > > >> > > > obvious.
> > > >> > > > > > > > > >> > > > >> > So,
> > > >> > > > > > > > > >> > > > >> > > >>>> rather
> > > >> > > > > > > > > >> > > > >> > > >>>> >> than
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> creating a new
> > configuration,
> > > >> why
> > > >> > > not
> > > >> > > > > add
> > > >> > > > > > > > > >> additional
> > > >> > > > > > > > > >> > > > values
> > > >> > > > > > > > > >> > > > >> to
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> "unclean.leader.election.enable"?
> > > >> > I
> > > >> > > > > think
> > > >> > > > > > > this
> > > >> > > > > > > > > >> will
> > > >> > > > > > > > > >> > be
> > > >> > > > > > > > > >> > > > >> simpler
> > > >> > > > > > > > > >> > > > >> > > for
> > > >> > > > > > > > > >> > > > >> > > >>>> >> people
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> to understand, and
> simpler
> > in
> > > >> the
> > > >> > > code
> > > >> > > > > as
> > > >> > > > > > > > well.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> What if we continued to
> use
> > > >> > > > > > > > > >> > > > "unclean.leader.election.enable"
> > > >> > > > > > > > > >> > > > >> > but
> > > >> > > > > > > > > >> > > > >> > > >>>> >> extended
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> it so that it took a
> > string?
> > > >> Then
> > > >> > > the
> > > >> > > > > > string
> > > >> > > > > > > > > could
> > > >> > > > > > > > > >> > have
> > > >> > > > > > > > > >> > > > >> these
> > > >> > > > > > > > > >> > > > >> > > >>>> values:
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> never
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>     never automatically
> do
> > an
> > > >> > > unclean
> > > >> > > > > > leader
> > > >> > > > > > > > > >> election
> > > >> > > > > > > > > >> > > > under
> > > >> > > > > > > > > >> > > > >> > any
> > > >> > > > > > > > > >> > > > >> > > >>>> >> conditions
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> false / default
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>     only do an unclean
> > leader
> > > >> > > election
> > > >> > > > > if
> > > >> > > > > > > > there
> > > >> > > > > > > > > >> may
> > > >> > > > > > > > > >> > be
> > > >> > > > > > > > > >> > > > >> > possible
> > > >> > > > > > > > > >> > > > >> > > >>>> data
> > > >> > > > > > > > > >> > > > >> > > >>>> >> loss
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> true / always
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>     always do an unclean
> > > leader
> > > >> > > > election
> > > >> > > > > > if
> > > >> > > > > > > we
> > > >> > > > > > > > > >> can't
> > > >> > > > > > > > > >> > > > >> > immediately
> > > >> > > > > > > > > >> > > > >> > > >>>> elect a
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> leader
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> It's a bit awkward that
> > false
> > > >> maps
> > > >> > > to
> > > >> > > > > > > default
> > > >> > > > > > > > > >> rather
> > > >> > > > > > > > > >> > > > than to
> > > >> > > > > > > > > >> > > > >> > > >>>> never. But
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> this awkwardness exists
> if
> > we
> > > >> use
> > > >> > > two
> > > >> > > > > > > > different
> > > >> > > > > > > > > >> > > > >> configuration
> > > >> > > > > > > > > >> > > > >> > > keys
> > > >> > > > > > > > > >> > > > >> > > >>>> as
> > > >> > > > > > > > > >> > > > >> > > >>>> >> well.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> The reason for the
> > > awkwardness
> > > >> is
> > > >> > > that
> > > >> > > > > we
> > > >> > > > > > > > simply
> > > >> > > > > > > > > >> > don't
> > > >> > > > > > > > > >> > > > want
> > > >> > > > > > > > > >> > > > >> > most
> > > >> > > > > > > > > >> > > > >> > > >>>> of the
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> people currently setting
> > > >> > > > > > > > > >> > > > >> unclean.leader.election.enable=false
> > > >> > > > > > > > > >> > > > >> > to
> > > >> > > > > > > > > >> > > > >> > > >>>> get the
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> "never" behavior. We have
> > to
> > > >> bite
> > > >> > > that
> > > >> > > > > > > bullet.
> > > >> > > > > > > > > >> Better
> > > >> > > > > > > > > >> > > to
> > > >> > > > > > > > > >> > > > be
> > > >> > > > > > > > > >> > > > >> > > clear
> > > >> > > > > > > > > >> > > > >> > > >>>> and
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> explicit than hide it.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> Another thing that's a
> bit
> > > >> awkward
> > > >> > > is
> > > >> > > > > > having
> > > >> > > > > > > > two
> > > >> > > > > > > > > >> > > > different
> > > >> > > > > > > > > >> > > > >> > ways
> > > >> > > > > > > > > >> > > > >> > > to
> > > >> > > > > > > > > >> > > > >> > > >>>> do
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> unclean leader election
> > > >> specified
> > > >> > in
> > > >> > > > the
> > > >> > > > > > > KIP.
> > > >> > > > > > > > > You
> > > >> > > > > > > > > >> > > > descirbe
> > > >> > > > > > > > > >> > > > >> two
> > > >> > > > > > > > > >> > > > >> > > >>>> methods:
> > > >> > > > > > > > > >> > > > >> > > >>>> >> the
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> simple "choose the last
> > > leader"
> > > >> > > > method,
> > > >> > > > > > and
> > > >> > > > > > > > the
> > > >> > > > > > > > > >> > > "unclean
> > > >> > > > > > > > > >> > > > >> > > recovery
> > > >> > > > > > > > > >> > > > >> > > >>>> >> manager"
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> method. I understand why
> > you
> > > >> did
> > > >> > it
> > > >> > > > this
> > > >> > > > > > way
> > > >> > > > > > > > --
> > > >> > > > > > > > > >> > "choose
> > > >> > > > > > > > > >> > > > the
> > > >> > > > > > > > > >> > > > >> > last
> > > >> > > > > > > > > >> > > > >> > > >>>> >> leader" is
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> simple, and will help us
> > > >> deliver
> > > >> > an
> > > >> > > > > > > > > implementation
> > > >> > > > > > > > > >> > > > quickly,
> > > >> > > > > > > > > >> > > > >> > > while
> > > >> > > > > > > > > >> > > > >> > > >>>> the
> > > >> > > > > > > > > >> > > > >> > > >>>> >> URM
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> is preferable in the long
> > > >> term. My
> > > >> > > > > > > suggestion
> > > >> > > > > > > > > >> here is
> > > >> > > > > > > > > >> > > to
> > > >> > > > > > > > > >> > > > >> > > separate
> > > >> > > > > > > > > >> > > > >> > > >>>> the
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> decision of HOW to do
> > unclean
> > > >> > leader
> > > >> > > > > > > election
> > > >> > > > > > > > > from
> > > >> > > > > > > > > >> > the
> > > >> > > > > > > > > >> > > > >> > decision
> > > >> > > > > > > > > >> > > > >> > > of
> > > >> > > > > > > > > >> > > > >> > > >>>> WHEN
> > > >> > > > > > > > > >> > > > >> > > >>>> >> to
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> do it.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> So in other words, have
> > > >> > > > > > > > > >> > > "unclean.leader.election.enable"
> > > >> > > > > > > > > >> > > > >> > specify
> > > >> > > > > > > > > >> > > > >> > > >>>> when we
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> do unclean leader
> election,
> > > and
> > > >> > > have a
> > > >> > > > > new
> > > >> > > > > > > > > >> > > configuration
> > > >> > > > > > > > > >> > > > >> like
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> "unclean.recovery.manager.enable"
> > > >> > to
> > > >> > > > > > > determine
> > > >> > > > > > > > > if
> > > >> > > > > > > > > >> we
> > > >> > > > > > > > > >> > > use
> > > >> > > > > > > > > >> > > > the
> > > >> > > > > > > > > >> > > > >> > > URM.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> Presumably the URM will
> > take
> > > >> some
> > > >> > > time
> > > >> > > > > to
> > > >> > > > > > > get
> > > >> > > > > > > > > >> fully
> > > >> > > > > > > > > >> > > > stable,
> > > >> > > > > > > > > >> > > > >> so
> > > >> > > > > > > > > >> > > > >> > > >>>> this can
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> default to false for a
> > while,
> > > >> and
> > > >> > we
> > > >> > > > can
> > > >> > > > > > > flip
> > > >> > > > > > > > > the
> > > >> > > > > > > > > >> > > > default to
> > > >> > > > > > > > > >> > > > >> > > true
> > > >> > > > > > > > > >> > > > >> > > >>>> when
> > > >> > > > > > > > > >> > > > >> > > >>>> >> we
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> feel ready.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> The URM is somewhat
> > > >> > under-described
> > > >> > > > > here.
> > > >> > > > > > I
> > > >> > > > > > > > > think
> > > >> > > > > > > > > >> we
> > > >> > > > > > > > > >> > > > need a
> > > >> > > > > > > > > >> > > > >> > few
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> configurations here for
> it.
> > > For
> > > >> > > > example,
> > > >> > > > > > we
> > > >> > > > > > > > > need a
> > > >> > > > > > > > > >> > > > >> > > configuration to
> > > >> > > > > > > > > >> > > > >> > > >>>> >> specify
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> how long it should wait
> > for a
> > > >> > broker
> > > >> > > > to
> > > >> > > > > > > > respond
> > > >> > > > > > > > > to
> > > >> > > > > > > > > >> > its
> > > >> > > > > > > > > >> > > > RPCs
> > > >> > > > > > > > > >> > > > >> > > before
> > > >> > > > > > > > > >> > > > >> > > >>>> >> moving
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> on. We also need to
> > > understand
> > > >> how
> > > >> > > the
> > > >> > > > > URM
> > > >> > > > > > > > > >> interacts
> > > >> > > > > > > > > >> > > with
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > unclean.leader.election.enable=always. I
> > > >> > > > > > > > assume
> > > >> > > > > > > > > >> that
> > > >> > > > > > > > > >> > > with
> > > >> > > > > > > > > >> > > > >> > > "always"
> > > >> > > > > > > > > >> > > > >> > > >>>> we
> > > >> > > > > > > > > >> > > > >> > > >>>> >> will
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> just unconditionally use
> > the
> > > >> URM
> > > >> > > > rather
> > > >> > > > > > than
> > > >> > > > > > > > > >> choosing
> > > >> > > > > > > > > >> > > > >> > randomly.
> > > >> > > > > > > > > >> > > > >> > > >>>> But this
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> should be spelled out in
> > the
> > > >> KIP.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > DescribeTopicRequest
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >    1.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >    Yes, the plan is to
> > > >> replace
> > > >> > the
> > > >> > > > > > > > > >> MetadataRequest
> > > >> > > > > > > > > >> > > with
> > > >> > > > > > > > > >> > > > >> the
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest
> > for
> > > >> the
> > > >> > > > admin
> > > >> > > > > > > > clients.
> > > >> > > > > > > > > >> Will
> > > >> > > > > > > > > >> > > > check
> > > >> > > > > > > > > >> > > > >> > the
> > > >> > > > > > > > > >> > > > >> > > >>>> details.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> Sounds good. But as I
> said,
> > > you
> > > >> > need
> > > >> > > > to
> > > >> > > > > > > > specify
> > > >> > > > > > > > > >> how
> > > >> > > > > > > > > >> > > > >> > AdminClient
> > > >> > > > > > > > > >> > > > >> > > >>>> >> interacts
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> with the new request.
> This
> > > will
> > > >> > > > involve
> > > >> > > > > > > adding
> > > >> > > > > > > > > >> some
> > > >> > > > > > > > > >> > > > fields
> > > >> > > > > > > > > >> > > > >> to
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> TopicDescription.java.
> And
> > > you
> > > >> > need
> > > >> > > to
> > > >> > > > > > > specify
> > > >> > > > > > > > > the
> > > >> > > > > > > > > >> > > > changes
> > > >> > > > > > > > > >> > > > >> to
> > > >> > > > > > > > > >> > > > >> > > the
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> kafka-topics.sh command
> > line
> > > >> tool.
> > > >> > > > > > Otherwise
> > > >> > > > > > > > we
> > > >> > > > > > > > > >> > cannot
> > > >> > > > > > > > > >> > > > use
> > > >> > > > > > > > > >> > > > >> the
> > > >> > > > > > > > > >> > > > >> > > >>>> tool to
> > > >> > > > > > > > > >> > > > >> > > >>>> >> see
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> the new information.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> The new requests,
> > > >> > > DescribeTopicRequest
> > > >> > > > > and
> > > >> > > > > > > > > >> > > > >> > > >>>> GetReplicaLogInfoRequest,
> > > >> > > > > > > > > >> > > > >> > > >>>> >> need
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> to have limits placed on
> > them
> > > >> so
> > > >> > > that
> > > >> > > > > > their
> > > >> > > > > > > > size
> > > >> > > > > > > > > >> > can't
> > > >> > > > > > > > > >> > > be
> > > >> > > > > > > > > >> > > > >> > > >>>> infinite. We
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> don't want to propagate
> the
> > > >> > current
> > > >> > > > > > problems
> > > >> > > > > > > > of
> > > >> > > > > > > > > >> > > > >> > MetadataRequest,
> > > >> > > > > > > > > >> > > > >> > > >>>> where
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> clients can request
> massive
> > > >> > > responses
> > > >> > > > > that
> > > >> > > > > > > can
> > > >> > > > > > > > > >> mess
> > > >> > > > > > > > > >> > up
> > > >> > > > > > > > > >> > > > the
> > > >> > > > > > > > > >> > > > >> JVM
> > > >> > > > > > > > > >> > > > >> > > when
> > > >> > > > > > > > > >> > > > >> > > >>>> >> handled.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> Adding limits is simple
> for
> > > >> > > > > > > > > >> GetReplicaLogInfoRequest
> > > >> > > > > > > > > >> > --
> > > >> > > > > > > > > >> > > > we
> > > >> > > > > > > > > >> > > > >> can
> > > >> > > > > > > > > >> > > > >> > > >>>> just say
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> that only 2000 partitions
> > at
> > > a
> > > >> > time
> > > >> > > > can
> > > >> > > > > be
> > > >> > > > > > > > > >> requested.
> > > >> > > > > > > > > >> > > For
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> DescribeTopicRequest we
> can
> > > >> > probably
> > > >> > > > > just
> > > >> > > > > > > > limit
> > > >> > > > > > > > > >> to 20
> > > >> > > > > > > > > >> > > > topics
> > > >> > > > > > > > > >> > > > >> > or
> > > >> > > > > > > > > >> > > > >> > > >>>> >> something
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> like that, to avoid the
> > > >> complexity
> > > >> > > of
> > > >> > > > > > doing
> > > >> > > > > > > > > >> > pagination
> > > >> > > > > > > > > >> > > in
> > > >> > > > > > > > > >> > > > >> this
> > > >> > > > > > > > > >> > > > >> > > KIP.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >    2.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >    I can let the broker
> > > load
> > > >> the
> > > >> > > ELR
> > > >> > > > > > info
> > > >> > > > > > > so
> > > >> > > > > > > > > >> that
> > > >> > > > > > > > > >> > > they
> > > >> > > > > > > > > >> > > > can
> > > >> > > > > > > > > >> > > > >> > > serve
> > > >> > > > > > > > > >> > > > >> > > >>>> the
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest
> > as
> > > >> well.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> Yes, it's fine to add to
> > > >> > > > MetadataCache.
> > > >> > > > > In
> > > >> > > > > > > > fact,
> > > >> > > > > > > > > >> > you'll
> > > >> > > > > > > > > >> > > > be
> > > >> > > > > > > > > >> > > > >> > > loading
> > > >> > > > > > > > > >> > > > >> > > >>>> it
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> anyway once it's added to
> > > >> > > > > PartitionImage.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >    3.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >    Yeah, it does not
> make
> > > >> sense
> > > >> > to
> > > >> > > > > have
> > > >> > > > > > > the
> > > >> > > > > > > > > >> topic
> > > >> > > > > > > > > >> > id
> > > >> > > > > > > > > >> > > if
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest
> > is
> > > >> only
> > > >> > > used
> > > >> > > > > by
> > > >> > > > > > > the
> > > >> > > > > > > > > >> admin
> > > >> > > > > > > > > >> > > > client.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> OK. That makes things
> > > simpler.
> > > >> We
> > > >> > > can
> > > >> > > > > > always
> > > >> > > > > > > > > >> create a
> > > >> > > > > > > > > >> > > new
> > > >> > > > > > > > > >> > > > >> API
> > > >> > > > > > > > > >> > > > >> > > later
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> (hopefully not in this
> > KIP!)
> > > to
> > > >> > > query
> > > >> > > > by
> > > >> > > > > > > topic
> > > >> > > > > > > > > ID.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Metrics
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > As for overall cluster
> > > health
> > > >> > > > > metrics, I
> > > >> > > > > > > > think
> > > >> > > > > > > > > >> > > > >> under-min-ISR
> > > >> > > > > > > > > >> > > > >> > > is
> > > >> > > > > > > > > >> > > > >> > > >>>> still
> > > >> > > > > > > > > >> > > > >> > > >>>> >> a
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > useful one. ELR is more
> > > like
> > > >> a
> > > >> > > > safety
> > > >> > > > > > > belt.
> > > >> > > > > > > > > When
> > > >> > > > > > > > > >> > the
> > > >> > > > > > > > > >> > > > ELR
> > > >> > > > > > > > > >> > > > >> is
> > > >> > > > > > > > > >> > > > >> > > >>>> used, the
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > cluster availability
> has
> > > >> already
> > > >> > > > been
> > > >> > > > > > > > > impacted.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Maybe we can have a
> > metric
> > > to
> > > >> > > count
> > > >> > > > > the
> > > >> > > > > > > > > >> partitions
> > > >> > > > > > > > > >> > > that
> > > >> > > > > > > > > >> > > > >> > > sum(ISR,
> > > >> > > > > > > > > >> > > > >> > > >>>> ELR)
> > > >> > > > > > > > > >> > > > >> > > >>>> >> <
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> min
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > ISR. What do you think?
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> How about:
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> A.  a metric for the
> totoal
> > > >> number
> > > >> > > of
> > > >> > > > > > > > > >> under-min-isr
> > > >> > > > > > > > > >> > > > >> > partitions?
> > > >> > > > > > > > > >> > > > >> > > We
> > > >> > > > > > > > > >> > > > >> > > >>>> don't
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> have that in Apache Kafka
> > at
> > > >> the
> > > >> > > > moment.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> B. a metric for the
> number
> > of
> > > >> > > unclean
> > > >> > > > > > leader
> > > >> > > > > > > > > >> > elections
> > > >> > > > > > > > > >> > > we
> > > >> > > > > > > > > >> > > > >> did
> > > >> > > > > > > > > >> > > > >> > > (for
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> simplicity, it can reset
> > to 0
> > > >> on
> > > >> > > > > > controller
> > > >> > > > > > > > > >> restart:
> > > >> > > > > > > > > >> > we
> > > >> > > > > > > > > >> > > > >> expect
> > > >> > > > > > > > > >> > > > >> > > >>>> people to
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> monitor the change over
> > time
> > > >> > anyway)
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> best,
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> Colin
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Yeah, for the ongoing
> > > unclean
> > > >> > > > > > recoveries,
> > > >> > > > > > > > the
> > > >> > > > > > > > > >> > > > controller
> > > >> > > > > > > > > >> > > > >> can
> > > >> > > > > > > > > >> > > > >> > > >>>> keep an
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > accurate count through
> > > >> failover
> > > >> > > > > because
> > > >> > > > > > > > > >> partition
> > > >> > > > > > > > > >> > > > >> > registration
> > > >> > > > > > > > > >> > > > >> > > >>>> can
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> indicate
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > whether a recovery is
> > > needed.
> > > >> > > > However,
> > > >> > > > > > for
> > > >> > > > > > > > the
> > > >> > > > > > > > > >> > > happened
> > > >> > > > > > > > > >> > > > >> > ones,
> > > >> > > > > > > > > >> > > > >> > > >>>> unless
> > > >> > > > > > > > > >> > > > >> > > >>>> >> we
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > want to persist the
> > number
> > > >> > > > somewhere,
> > > >> > > > > we
> > > >> > > > > > > can
> > > >> > > > > > > > > >> only
> > > >> > > > > > > > > >> > > > figure
> > > >> > > > > > > > > >> > > > >> it
> > > >> > > > > > > > > >> > > > >> > > out
> > > >> > > > > > > > > >> > > > >> > > >>>> from
> > > >> > > > > > > > > >> > > > >> > > >>>> >> the
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > log.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> > On Tue, Sep 12, 2023 at
> > > >> 3:16 PM
> > > >> > > > Colin
> > > >> > > > > > > > McCabe <
> > > >> > > > > > > > > >> > > > >> > > cmccabe@apache.org
> > > >> > > > > > > > > >> > > > >> > > >>>> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> wrote:
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> Also, we should have
> > > metrics
> > > >> > that
> > > >> > > > > show
> > > >> > > > > > > what
> > > >> > > > > > > > > is
> > > >> > > > > > > > > >> > going
> > > >> > > > > > > > > >> > > > on
> > > >> > > > > > > > > >> > > > >> > with
> > > >> > > > > > > > > >> > > > >> > > >>>> regard
> > > >> > > > > > > > > >> > > > >> > > >>>> >> to
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> the
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> eligible replica set.
> > I'm
> > > >> not
> > > >> > > sure
> > > >> > > > > > > exactly
> > > >> > > > > > > > > >> what to
> > > >> > > > > > > > > >> > > > >> suggest,
> > > >> > > > > > > > > >> > > > >> > > but
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> something
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> that could identify
> when
> > > >> things
> > > >> > > are
> > > >> > > > > > going
> > > >> > > > > > > > > >> wrong in
> > > >> > > > > > > > > >> > > the
> > > >> > > > > > > > > >> > > > >> > > clsuter.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> For example, maybe a
> > > metric
> > > >> for
> > > >> > > > > > > partitions
> > > >> > > > > > > > > >> > > containing
> > > >> > > > > > > > > >> > > > >> > > replicas
> > > >> > > > > > > > > >> > > > >> > > >>>> that
> > > >> > > > > > > > > >> > > > >> > > >>>> >> are
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> ineligible to be
> leader?
> > > >> That
> > > >> > > would
> > > >> > > > > > show
> > > >> > > > > > > a
> > > >> > > > > > > > > >> spike
> > > >> > > > > > > > > >> > > when
> > > >> > > > > > > > > >> > > > a
> > > >> > > > > > > > > >> > > > >> > > broker
> > > >> > > > > > > > > >> > > > >> > > >>>> had an
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> unclean restart.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> Ideally, we'd also
> have
> > a
> > > >> > metric
> > > >> > > > that
> > > >> > > > > > > > > indicates
> > > >> > > > > > > > > >> > when
> > > >> > > > > > > > > >> > > > an
> > > >> > > > > > > > > >> > > > >> > > unclear
> > > >> > > > > > > > > >> > > > >> > > >>>> >> leader
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> election or a recovery
> > > >> > happened.
> > > >> > > > > It's a
> > > >> > > > > > > bit
> > > >> > > > > > > > > >> tricky
> > > >> > > > > > > > > >> > > > >> because
> > > >> > > > > > > > > >> > > > >> > > the
> > > >> > > > > > > > > >> > > > >> > > >>>> simple
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> thing, of tracking it
> > per
> > > >> > > > controller,
> > > >> > > > > > may
> > > >> > > > > > > > be
> > > >> > > > > > > > > a
> > > >> > > > > > > > > >> bit
> > > >> > > > > > > > > >> > > > >> > confusing
> > > >> > > > > > > > > >> > > > >> > > >>>> during
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> failovers.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> best,
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> Colin
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >>
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> On Tue, Sep 12, 2023,
> at
> > > >> 14:25,
> > > >> > > > Colin
> > > >> > > > > > > > McCabe
> > > >> > > > > > > > > >> > wrote:
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > Hi Calvin,
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > Thanks for the KIP.
> I
> > > >> think
> > > >> > > this
> > > >> > > > > is a
> > > >> > > > > > > > great
> > > >> > > > > > > > > >> > > > >> improvement.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >> Additional High
> > > Watermark
> > > >> > > > advance
> > > >> > > > > > > > > >> requirement
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > Typo: change
> "advance"
> > > to
> > > >> > > > > > "advancement"
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >> A bit recap of some
> > key
> > > >> > > > concepts.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > Typo: change "bit"
> to
> > > >> "quick"
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >> Ack=1/all produce
> > > >> request.
> > > >> > It
> > > >> > > > > > defines
> > > >> > > > > > > > when
> > > >> > > > > > > > > >> the
> > > >> > > > > > > > > >> > > > Kafka
> > > >> > > > > > > > > >> > > > >> > > server
> > > >> > > > > > > > > >> > > > >> > > >>>> should
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> respond to the produce
> > > >> request
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > I think this section
> > > >> would be
> > > >> > > > > clearer
> > > >> > > > > > > if
> > > >> > > > > > > > we
> > > >> > > > > > > > > >> > talked
> > > >> > > > > > > > > >> > > > >> about
> > > >> > > > > > > > > >> > > > >> > > the
> > > >> > > > > > > > > >> > > > >> > > >>>> new
> > > >> > > > > > > > > >> > > > >> > > >>>> >> high
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > watermark
> advancement
> > > >> > > requirement
> > > >> > > > > > > first,
> > > >> > > > > > > > > and
> > > >> > > > > > > > > >> > THEN
> > > >> > > > > > > > > >> > > > >> talked
> > > >> > > > > > > > > >> > > > >> > > >>>> about its
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > impact on acks=0,
> > > acks=1,
> > > >> and
> > > >> > > > > > > >  acks=all.
> > > >> > > > > > > > > >> > > > acks=all
> > > >> > > > > > > > > >> > > > >> is
> > > >> > > > > > > > > >> > > > >> > of
> > > >> > > > > > > > > >> > > > >> > > >>>> course
> > > >> > > > > > > > > >> > > > >> > > >>>> >> the
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > main case we care
> > about
> > > >> here,
> > > >> > > so
> > > >> > > > it
> > > >> > > > > > > would
> > > >> > > > > > > > > be
> > > >> > > > > > > > > >> > good
> > > >> > > > > > > > > >> > > to
> > > >> > > > > > > > > >> > > > >> lead
> > > >> > > > > > > > > >> > > > >> > > with
> > > >> > > > > > > > > >> > > > >> > > >>>> >> that,
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > rather than delving
> > into
> > > >> the
> > > >> > > > > > > > technicalities
> > > >> > > > > > > > > >> of
> > > >> > > > > > > > > >> > > > acks=0/1
> > > >> > > > > > > > > >> > > > >> > > first.
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >> Unclean recovery
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > So, here you are
> > > >> introducing
> > > >> > a
> > > >> > > > new
> > > >> > > > > > > > > >> > configuration,
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > unclean.recovery.strategy.
> > > >> > The
> > > >> > > > > > > difficult
> > > >> > > > > > > > > >> thing
> > > >> > > > > > > > > >> > > here
> > > >> > > > > > > > > >> > > > is
> > > >> > > > > > > > > >> > > > >> > that
> > > >> > > > > > > > > >> > > > >> > > >>>> there
> > > >> > > > > > > > > >> > > > >> > > >>>> >> is a
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > lot of overlap with
> > > >> > > > > > > > > >> > > unclean.leader.election.enable.
> > > >> > > > > > > > > >> > > > So
> > > >> > > > > > > > > >> > > > >> we
> > > >> > > > > > > > > >> > > > >> > > >>>> have 3
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > different settings
> for
> > > >> > > > > > > > > >> > unclean.recovery.strategy,
> > > >> > > > > > > > > >> > > > plus
> > > >> > > > > > > > > >> > > > >> 2
> > > >> > > > > > > > > >> > > > >> > > >>>> different
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > settings for
> > > >> > > > > > > > > unclean.leader.election.enable,
> > > >> > > > > > > > > >> > > giving
> > > >> > > > > > > > > >> > > > a
> > > >> > > > > > > > > >> > > > >> > cross
> > > >> > > > > > > > > >> > > > >> > > >>>> >> product of
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > 6 different options.
> > The
> > > >> > > > following
> > > >> > > > > > > > "unclean
> > > >> > > > > > > > > >> > > recovery
> > > >> > > > > > > > > >> > > > >> > > manager"
> > > >> > > > > > > > > >> > > > >> > > >>>> >> section
> > > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > on
> > > >> > > > > >
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > > >
> > >
> >
>
>
> --
> David Arthur
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by David Arthur <mu...@gmail.com>.
One thing we should consider is a static config to totally enable/disable
the ELR feature. If I understand the KIP correctly, we can effectively
disable the unclean recovery by setting the recovery strategy config to
"none".

This would make development and rollout of this feature a bit smoother.
Consider the case that we find bugs in ELR after a cluster has updated to
its MetadataVersion. It's simpler to disable the feature through config
rather than going through a MetadataVersion downgrade (once that's
supported).

Does that make sense?

-David

On Wed, Oct 11, 2023 at 1:40 PM Calvin Liu <ca...@confluent.io.invalid>
wrote:

> Hi Jun
> -Good catch, yes, we don't need the -1 in the DescribeTopicRequest.
> -No new value is added. The LeaderRecoveryState will still be set to 1 if
> we have an unclean leader election. The unclean leader election includes
> the old random way and the unclean recovery. During the unclean recovery,
> the LeaderRecoveryState will not change until the controller decides to
> update the records with the new leader.
> Thanks
>
> On Wed, Oct 11, 2023 at 9:02 AM Jun Rao <ju...@confluent.io.invalid> wrote:
>
> > Hi, Calvin,
> >
> > Another thing. Currently, when there is an unclean leader election, we
> set
> > the LeaderRecoveryState in PartitionRecord and PartitionChangeRecord to
> 1.
> > With the KIP, will there be new values for LeaderRecoveryState? If not,
> > when will LeaderRecoveryState be set to 1?
> >
> > Thanks,
> >
> > Jun
> >
> > On Tue, Oct 10, 2023 at 4:24 PM Jun Rao <ju...@confluent.io> wrote:
> >
> > > Hi, Calvin,
> > >
> > > One more comment.
> > >
> > > "The first partition to fetch details for. -1 means to fetch all
> > > partitions." It seems that FirstPartitionId of 0 naturally means
> fetching
> > > all partitions?
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Tue, Oct 10, 2023 at 12:40 PM Calvin Liu <caliu@confluent.io.invalid
> >
> > > wrote:
> > >
> > >> Hi Jun,
> > >> Yeah, with the current Metadata request handling, we only return
> errors
> > on
> > >> the Topic level, like topic not found. It seems that querying a
> specific
> > >> partition is not a valid use case. Will update.
> > >> Thanks
> > >>
> > >> On Tue, Oct 10, 2023 at 11:55 AM Jun Rao <ju...@confluent.io.invalid>
> > >> wrote:
> > >>
> > >> > Hi, Calvin,
> > >> >
> > >> > 60.  If the range query has errors for some of the partitions, do we
> > >> expect
> > >> > different responses when querying particular partitions?
> > >> >
> > >> > Thanks,
> > >> >
> > >> > Jun
> > >> >
> > >> > On Tue, Oct 10, 2023 at 10:50 AM Calvin Liu
> > <caliu@confluent.io.invalid
> > >> >
> > >> > wrote:
> > >> >
> > >> > > Hi Jun
> > >> > > 60. Yes, it is a good question. I was thinking the API could be
> > >> flexible
> > >> > to
> > >> > > query the particular partitions if the range query has errors for
> > >> some of
> > >> > > the partitions. Not sure whether it is a valid assumption, what do
> > you
> > >> > > think?
> > >> > >
> > >> > > 61. Good point, I will update them to partition level with the
> same
> > >> > limit.
> > >> > >
> > >> > > 62. Sure, will do.
> > >> > >
> > >> > > Thanks
> > >> > >
> > >> > > On Tue, Oct 10, 2023 at 10:12 AM Jun Rao <jun@confluent.io.invalid
> >
> > >> > wrote:
> > >> > >
> > >> > > > Hi, Calvin,
> > >> > > >
> > >> > > > A few more minor comments on your latest update.
> > >> > > >
> > >> > > > 60. DescribeTopicRequest: When will the Partitions field be
> used?
> > It
> > >> > > seems
> > >> > > > that the FirstPartitionId field is enough for AdminClient usage.
> > >> > > >
> > >> > > > 61. Could we make the limit for DescribeTopicRequest,
> > >> > > ElectLeadersRequest,
> > >> > > > GetReplicaLogInfo consistent? Currently, ElectLeadersRequest's
> > >> limit is
> > >> > > at
> > >> > > > topic level and GetReplicaLogInfo has a different partition
> level
> > >> limit
> > >> > > > from DescribeTopicRequest.
> > >> > > >
> > >> > > > 62. Should ElectLeadersRequest.DesiredLeaders be at the same
> level
> > >> as
> > >> > > > ElectLeadersRequest.TopicPartitions.Partitions? In the KIP, it
> > looks
> > >> > like
> > >> > > > it's at the same level as ElectLeadersRequest.TopicPartitions.
> > >> > > >
> > >> > > > Thanks,
> > >> > > >
> > >> > > > Jun
> > >> > > >
> > >> > > > On Wed, Oct 4, 2023 at 3:55 PM Calvin Liu
> > >> <ca...@confluent.io.invalid>
> > >> > > > wrote:
> > >> > > >
> > >> > > > > Hi David,
> > >> > > > > Thanks for the comments.
> > >> > > > > ----
> > >> > > > > I thought that a new snapshot with the downgraded MV is
> created
> > in
> > >> > this
> > >> > > > > case. Isn’t it the case?
> > >> > > > > Yes, you are right, a metadata delta will be generated after
> the
> > >> MV
> > >> > > > > downgrade. Then the user can start the software downgrade.
> > >> > > > > -----
> > >> > > > > Could you also elaborate a bit more on the reasoning behind
> > adding
> > >> > the
> > >> > > > > limits to the admin RPCs? This is a new pattern in Kafka so it
> > >> would
> > >> > be
> > >> > > > > good to clear on the motivation.
> > >> > > > > Thanks to Colin for bringing it up. The current
> MetadataRequest
> > >> does
> > >> > > not
> > >> > > > > have a limit on the number of topics to query in a single
> > request.
> > >> > > > Massive
> > >> > > > > requests can mess up the JVM. We want to have some sort of
> > >> throttle
> > >> > on
> > >> > > > the
> > >> > > > > new APIs.
> > >> > > > > -----
> > >> > > > > Could you also explain how the client is supposed to handle
> the
> > >> > > > > topics/partitions above the limit? I suppose that it will have
> > to
> > >> > retry
> > >> > > > > those, correct?
> > >> > > > > Corrent. For the official admin clients, it will split the
> large
> > >> > > request
> > >> > > > > into proper pieces and query one after another.
> > >> > > > > -----
> > >> > > > > My understanding is that the topics/partitions above the limit
> > >> will
> > >> > be
> > >> > > > > failed with an invalid exception error. I wonder if this
> choice
> > is
> > >> > > > > judicious because the invalide request exception is usually
> > >> fatal. It
> > >> > > may
> > >> > > > > be better to use an new and explicit error for this case.
> > >> > > > >
> > >> > > > > Thanks for bringing this up. How about
> "REQUEST_LIMIT_REACHED"?
> > >> > > > > --------
> > >> > > > > It seems that we still need to specify the changes to the
> admin
> > >> api
> > >> > to
> > >> > > > > accommodate the new or updated apis. Do you plan to add them?
> > >> > > > > Try to cover the following
> > >> > > > > 1. The admin client will use the new DescribeTopicRequest to
> > query
> > >> > the
> > >> > > > > topics
> > >> > > > > 2. Mention the API limit and the new retriable error.
> > >> > > > > 3. Output changes for the admin client when describing a topic
> > >> (new
> > >> > > > fields
> > >> > > > > of ELR...)
> > >> > > > > 4. Changes to data structures like TopicPartitionInfo to
> include
> > >> the
> > >> > > ELR.
> > >> > > > > Anything else I missed?
> > >> > > > >
> > >> > > > > Thanks!
> > >> > > > >
> > >> > > > >
> > >> > > > >
> > >> > > > >
> > >> > > > >
> > >> > > > > On Wed, Oct 4, 2023 at 12:27 PM David Jacot <
> > >> david.jacot@gmail.com>
> > >> > > > wrote:
> > >> > > > >
> > >> > > > > > Hi Calvin,
> > >> > > > > >
> > >> > > > > > I thought that a new snapshot with the downgraded MV is
> > created
> > >> in
> > >> > > this
> > >> > > > > > case. Isn’t it the case?
> > >> > > > > >
> > >> > > > > > Could you also elaborate a bit more on the reasoning behind
> > >> adding
> > >> > > the
> > >> > > > > > limits to the admin RPCs? This is a new pattern in Kafka so
> it
> > >> > would
> > >> > > be
> > >> > > > > > good to clear on the motivation.
> > >> > > > > >
> > >> > > > > > Could you also explain how the client is supposed to handle
> > the
> > >> > > > > > topics/partitions above the limit? I suppose that it will
> have
> > >> to
> > >> > > retry
> > >> > > > > > those, correct?
> > >> > > > > >
> > >> > > > > > My understanding is that the topics/partitions above the
> limit
> > >> will
> > >> > > be
> > >> > > > > > failed with an invalid exception error. I wonder if this
> > choice
> > >> is
> > >> > > > > > judicious because the invalide request exception is usually
> > >> fatal.
> > >> > It
> > >> > > > may
> > >> > > > > > be better to use an new and explicit error for this case.
> > >> > > > > >
> > >> > > > > > It seems that we still need to specify the changes to the
> > admin
> > >> api
> > >> > > to
> > >> > > > > > accommodate the new or updated apis. Do you plan to add
> them?
> > >> > > > > >
> > >> > > > > > Best,
> > >> > > > > > David
> > >> > > > > >
> > >> > > > > > Le mer. 4 oct. 2023 à 20:39, Calvin Liu
> > >> <caliu@confluent.io.invalid
> > >> > >
> > >> > > a
> > >> > > > > > écrit :
> > >> > > > > >
> > >> > > > > > > Hi Jun,
> > >> > > > > > > After the MV downgrade, the controller will write in the
> old
> > >> > > version
> > >> > > > of
> > >> > > > > > the
> > >> > > > > > > PartitionRecord/PartitionChangeRecord. If I understand
> > >> correctly,
> > >> > > it
> > >> > > > is
> > >> > > > > > > possible to downgrade the software version if the
> controller
> > >> only
> > >> > > has
> > >> > > > > to
> > >> > > > > > > handle old version records.
> > >> > > > > > > However, the controller will not automatically rewrite the
> > >> > > > > > PartitionRecord
> > >> > > > > > > with the old version unless there is a partition update.
> > Then,
> > >> > the
> > >> > > > user
> > >> > > > > > may
> > >> > > > > > > have to wait an unknown amount of time before the software
> > >> > > downgrades
> > >> > > > > > > unless they do a roll to force update every partition. If
> it
> > >> > makes
> > >> > > > > > sense, I
> > >> > > > > > > can mention these steps to do a software downgrade.
> > >> > > > > > > Thanks
> > >> > > > > > >
> > >> > > > > > > On Wed, Oct 4, 2023 at 11:20 AM Jun Rao
> > >> <jun@confluent.io.invalid
> > >> > >
> > >> > > > > > wrote:
> > >> > > > > > >
> > >> > > > > > > > Hi, Calvin and Justine,
> > >> > > > > > > >
> > >> > > > > > > > Historically, when we change the record format in the
> log,
> > >> we
> > >> > > don't
> > >> > > > > > > support
> > >> > > > > > > > software version downgrading.
> > >> > > > > > > >
> > >> > > > > > > > For the record format change in the metadata log, have
> we
> > >> > thought
> > >> > > > > about
> > >> > > > > > > > forcing the write of the latest metadata records with
> the
> > >> old
> > >> > > > version
> > >> > > > > > > > during MV downgrading? This will in theory allow the old
> > >> > version
> > >> > > of
> > >> > > > > the
> > >> > > > > > > > software to obtain the latest metadata.
> > >> > > > > > > >
> > >> > > > > > > > Thanks,
> > >> > > > > > > >
> > >> > > > > > > > Jun
> > >> > > > > > > >
> > >> > > > > > > > On Wed, Oct 4, 2023 at 9:53 AM Justine Olshan
> > >> > > > > > > <jolshan@confluent.io.invalid
> > >> > > > > > > > >
> > >> > > > > > > > wrote:
> > >> > > > > > > >
> > >> > > > > > > > > Sorry -- not MV but software version.
> > >> > > > > > > > >
> > >> > > > > > > > > On Wed, Oct 4, 2023 at 9:51 AM Justine Olshan <
> > >> > > > > jolshan@confluent.io>
> > >> > > > > > > > > wrote:
> > >> > > > > > > > >
> > >> > > > > > > > > > Catching up with this discussion.
> > >> > > > > > > > > >
> > >> > > > > > > > > > I was just curious -- have we had other instances
> > where
> > >> > > > > downgrading
> > >> > > > > > > MV
> > >> > > > > > > > is
> > >> > > > > > > > > > not supported? I think Kafka typically tries to
> > support
> > >> > > > > downgrades,
> > >> > > > > > > > and I
> > >> > > > > > > > > > couldn't think of other examples.
> > >> > > > > > > > > >
> > >> > > > > > > > > > Thanks,
> > >> > > > > > > > > > Justine
> > >> > > > > > > > > >
> > >> > > > > > > > > > On Wed, Oct 4, 2023 at 9:40 AM Calvin Liu
> > >> > > > > > <caliu@confluent.io.invalid
> > >> > > > > > > >
> > >> > > > > > > > > > wrote:
> > >> > > > > > > > > >
> > >> > > > > > > > > >> Hi Jun,
> > >> > > > > > > > > >> 54. Marked the software downgrading is not
> supported.
> > >> As
> > >> > the
> > >> > > > old
> > >> > > > > > > > > >> controller
> > >> > > > > > > > > >> will not understand the new PartitionRecord and
> > >> > > > > > > PartitionChangeRecord.
> > >> > > > > > > > > >> Thanks!
> > >> > > > > > > > > >>
> > >> > > > > > > > > >> On Wed, Oct 4, 2023 at 9:12 AM Jun Rao
> > >> > > > <jun@confluent.io.invalid
> > >> > > > > >
> > >> > > > > > > > > wrote:
> > >> > > > > > > > > >>
> > >> > > > > > > > > >> > Hi, Calvin,
> > >> > > > > > > > > >> >
> > >> > > > > > > > > >> > Thanks for the reply. Just one more comment.
> > >> > > > > > > > > >> >
> > >> > > > > > > > > >> > 54. It seems that downgrading MV is supported. Is
> > >> > > > downgrading
> > >> > > > > > the
> > >> > > > > > > > > >> software
> > >> > > > > > > > > >> > version supported? It would be useful to document
> > >> that.
> > >> > > > > > > > > >> >
> > >> > > > > > > > > >> > Thanks,
> > >> > > > > > > > > >> >
> > >> > > > > > > > > >> > Jun
> > >> > > > > > > > > >> >
> > >> > > > > > > > > >> > On Tue, Oct 3, 2023 at 4:55 PM Artem Livshits
> > >> > > > > > > > > >> > <al...@confluent.io.invalid> wrote:
> > >> > > > > > > > > >> >
> > >> > > > > > > > > >> > > Hi Colin,
> > >> > > > > > > > > >> > >
> > >> > > > > > > > > >> > > I think in your example "do_unclean_recovery"
> > would
> > >> > need
> > >> > > > to
> > >> > > > > do
> > >> > > > > > > > > >> different
> > >> > > > > > > > > >> > > things depending on the strategy.
> > >> > > > > > > > > >> > >
> > >> > > > > > > > > >> > > do_unclean_recovery() {
> > >> > > > > > > > > >> > >    if (unclean.recovery.manager.enabled) {
> > >> > > > > > > > > >> > >     if (strategy == Aggressive)
> > >> > > > > > > > > >> > >       use
> > >> > UncleanRecoveryManager(waitLastKnownERL=false)
> > >> > > > //
> > >> > > > > > > just
> > >> > > > > > > > > >> inspect
> > >> > > > > > > > > >> > > logs from whoever is available
> > >> > > > > > > > > >> > >     else
> > >> > > > > > > > > >> > >       use
> > >> > UncleanRecoveryManager(waitLastKnownERL=true)
> > >> > > > //
> > >> > > > > > > must
> > >> > > > > > > > > wait
> > >> > > > > > > > > >> > for
> > >> > > > > > > > > >> > > at least last known ELR
> > >> > > > > > > > > >> > >   } else {
> > >> > > > > > > > > >> > >     if (strategy == Aggressive)
> > >> > > > > > > > > >> > >       choose the last known leader if that is
> > >> > available,
> > >> > > > or
> > >> > > > > a
> > >> > > > > > > > random
> > >> > > > > > > > > >> > leader
> > >> > > > > > > > > >> > > if not)
> > >> > > > > > > > > >> > >     else
> > >> > > > > > > > > >> > >       wait for last known leader to get back
> > >> > > > > > > > > >> > >   }
> > >> > > > > > > > > >> > > }
> > >> > > > > > > > > >> > >
> > >> > > > > > > > > >> > > The idea is that the Aggressive strategy would
> > >> kick in
> > >> > > as
> > >> > > > > soon
> > >> > > > > > > as
> > >> > > > > > > > we
> > >> > > > > > > > > >> lost
> > >> > > > > > > > > >> > > the leader and would pick a leader from whoever
> > is
> > >> > > > > available;
> > >> > > > > > > but
> > >> > > > > > > > > the
> > >> > > > > > > > > >> > > Balanced will only kick in when ELR is empty
> and
> > >> will
> > >> > > wait
> > >> > > > > for
> > >> > > > > > > the
> > >> > > > > > > > > >> > brokers
> > >> > > > > > > > > >> > > that likely have most data to be available.
> > >> > > > > > > > > >> > >
> > >> > > > > > > > > >> > > On Tue, Oct 3, 2023 at 3:04 PM Colin McCabe <
> > >> > > > > > cmccabe@apache.org
> > >> > > > > > > >
> > >> > > > > > > > > >> wrote:
> > >> > > > > > > > > >> > >
> > >> > > > > > > > > >> > > > On Tue, Oct 3, 2023, at 10:49, Jun Rao wrote:
> > >> > > > > > > > > >> > > > > Hi, Calvin,
> > >> > > > > > > > > >> > > > >
> > >> > > > > > > > > >> > > > > Thanks for the update KIP. A few more
> > comments.
> > >> > > > > > > > > >> > > > >
> > >> > > > > > > > > >> > > > > 41. Why would a user choose the option to
> > >> select a
> > >> > > > > random
> > >> > > > > > > > > replica
> > >> > > > > > > > > >> as
> > >> > > > > > > > > >> > > the
> > >> > > > > > > > > >> > > > > leader instead of using
> > >> > > > > > unclean.recovery.strateg=Aggressive?
> > >> > > > > > > > It
> > >> > > > > > > > > >> seems
> > >> > > > > > > > > >> > > > that
> > >> > > > > > > > > >> > > > > the latter is strictly better? If that's
> not
> > >> the
> > >> > > case,
> > >> > > > > > could
> > >> > > > > > > > we
> > >> > > > > > > > > >> fold
> > >> > > > > > > > > >> > > this
> > >> > > > > > > > > >> > > > > option under unclean.recovery.strategy
> > instead
> > >> of
> > >> > > > > > > introducing
> > >> > > > > > > > a
> > >> > > > > > > > > >> > > separate
> > >> > > > > > > > > >> > > > > config?
> > >> > > > > > > > > >> > > >
> > >> > > > > > > > > >> > > > Hi Jun,
> > >> > > > > > > > > >> > > >
> > >> > > > > > > > > >> > > > I thought the flow of control was:
> > >> > > > > > > > > >> > > >
> > >> > > > > > > > > >> > > > If there is no leader for the partition {
> > >> > > > > > > > > >> > > >   If (there are unfenced ELR members) {
> > >> > > > > > > > > >> > > >     choose_an_unfenced_ELR_member
> > >> > > > > > > > > >> > > >   } else if (there are fenced ELR members AND
> > >> > > > > > > > > strategy=Aggressive) {
> > >> > > > > > > > > >> > > >     do_unclean_recovery
> > >> > > > > > > > > >> > > >   } else if (there are no ELR members AND
> > >> strategy
> > >> > !=
> > >> > > > > None)
> > >> > > > > > {
> > >> > > > > > > > > >> > > >     do_unclean_recovery
> > >> > > > > > > > > >> > > >   } else {
> > >> > > > > > > > > >> > > >     do nothing about the missing leader
> > >> > > > > > > > > >> > > >   }
> > >> > > > > > > > > >> > > > }
> > >> > > > > > > > > >> > > >
> > >> > > > > > > > > >> > > > do_unclean_recovery() {
> > >> > > > > > > > > >> > > >    if (unclean.recovery.manager.enabled) {
> > >> > > > > > > > > >> > > >     use UncleanRecoveryManager
> > >> > > > > > > > > >> > > >   } else {
> > >> > > > > > > > > >> > > >     choose the last known leader if that is
> > >> > available,
> > >> > > > or
> > >> > > > > a
> > >> > > > > > > > random
> > >> > > > > > > > > >> > leader
> > >> > > > > > > > > >> > > > if not)
> > >> > > > > > > > > >> > > >   }
> > >> > > > > > > > > >> > > > }
> > >> > > > > > > > > >> > > >
> > >> > > > > > > > > >> > > > However, I think this could be clarified,
> > >> especially
> > >> > > the
> > >> > > > > > > > behavior
> > >> > > > > > > > > >> when
> > >> > > > > > > > > >> > > > unclean.recovery.manager.enabled=false.
> > >> Inuitively
> > >> > the
> > >> > > > > goal
> > >> > > > > > > for
> > >> > > > > > > > > >> > > > unclean.recovery.manager.enabled=false is to
> be
> > >> "the
> > >> > > > same
> > >> > > > > as
> > >> > > > > > > > now,
> > >> > > > > > > > > >> > mostly"
> > >> > > > > > > > > >> > > > but it's very underspecified in the KIP, I
> > agree.
> > >> > > > > > > > > >> > > >
> > >> > > > > > > > > >> > > > >
> > >> > > > > > > > > >> > > > > 50. ElectLeadersRequest: "If more than 20
> > >> topics
> > >> > are
> > >> > > > > > > included,
> > >> > > > > > > > > >> only
> > >> > > > > > > > > >> > the
> > >> > > > > > > > > >> > > > > first 20 will be served. Others will be
> > >> returned
> > >> > > with
> > >> > > > > > > > > >> > DesiredLeaders."
> > >> > > > > > > > > >> > > > Hmm,
> > >> > > > > > > > > >> > > > > not sure that I understand this.
> > >> > > ElectLeadersResponse
> > >> > > > > > > doesn't
> > >> > > > > > > > > >> have a
> > >> > > > > > > > > >> > > > > DesiredLeaders field.
> > >> > > > > > > > > >> > > > >
> > >> > > > > > > > > >> > > > > 51. GetReplicaLogInfo: "If more than 2000
> > >> > partitions
> > >> > > > are
> > >> > > > > > > > > included,
> > >> > > > > > > > > >> > only
> > >> > > > > > > > > >> > > > the
> > >> > > > > > > > > >> > > > > first 2000 will be served" Do we return an
> > >> error
> > >> > for
> > >> > > > the
> > >> > > > > > > > > remaining
> > >> > > > > > > > > >> > > > > partitions? Actually, should we include an
> > >> > errorCode
> > >> > > > > field
> > >> > > > > > > at
> > >> > > > > > > > > the
> > >> > > > > > > > > >> > > > partition
> > >> > > > > > > > > >> > > > > level in GetReplicaLogInfoResponse to cover
> > >> > > > non-existing
> > >> > > > > > > > > >> partitions
> > >> > > > > > > > > >> > and
> > >> > > > > > > > > >> > > > no
> > >> > > > > > > > > >> > > > > authorization, etc?
> > >> > > > > > > > > >> > > > >
> > >> > > > > > > > > >> > > > > 52. The entry should matches => The entry
> > >> should
> > >> > > match
> > >> > > > > > > > > >> > > > >
> > >> > > > > > > > > >> > > > > 53. ElectLeadersRequest.DesiredLeaders:
> > Should
> > >> it
> > >> > be
> > >> > > > > > > nullable
> > >> > > > > > > > > >> since a
> > >> > > > > > > > > >> > > > user
> > >> > > > > > > > > >> > > > > may not specify DesiredLeaders?
> > >> > > > > > > > > >> > > > >
> > >> > > > > > > > > >> > > > > 54. Downgrade: Is that indeed possible? I
> > >> thought
> > >> > > > > earlier
> > >> > > > > > > you
> > >> > > > > > > > > said
> > >> > > > > > > > > >> > that
> > >> > > > > > > > > >> > > > > once the new version of the records are in
> > the
> > >> > > > metadata
> > >> > > > > > log,
> > >> > > > > > > > one
> > >> > > > > > > > > >> > can't
> > >> > > > > > > > > >> > > > > downgrade since the old broker doesn't know
> > >> how to
> > >> > > > parse
> > >> > > > > > the
> > >> > > > > > > > new
> > >> > > > > > > > > >> > > version
> > >> > > > > > > > > >> > > > of
> > >> > > > > > > > > >> > > > > the metadata records?
> > >> > > > > > > > > >> > > > >
> > >> > > > > > > > > >> > > >
> > >> > > > > > > > > >> > > > MetadataVersion downgrade is currently broken
> > >> but we
> > >> > > > have
> > >> > > > > > > fixing
> > >> > > > > > > > > it
> > >> > > > > > > > > >> on
> > >> > > > > > > > > >> > > our
> > >> > > > > > > > > >> > > > plate for Kafka 3.7.
> > >> > > > > > > > > >> > > >
> > >> > > > > > > > > >> > > > The way downgrade works is that "new
> features"
> > >> are
> > >> > > > > dropped,
> > >> > > > > > > > > leaving
> > >> > > > > > > > > >> > only
> > >> > > > > > > > > >> > > > the old ones.
> > >> > > > > > > > > >> > > >
> > >> > > > > > > > > >> > > > > 55. CleanShutdownFile: Should we add a
> > version
> > >> > field
> > >> > > > for
> > >> > > > > > > > future
> > >> > > > > > > > > >> > > > extension?
> > >> > > > > > > > > >> > > > >
> > >> > > > > > > > > >> > > > > 56. Config changes are public facing. Could
> > we
> > >> > have
> > >> > > a
> > >> > > > > > > separate
> > >> > > > > > > > > >> > section
> > >> > > > > > > > > >> > > to
> > >> > > > > > > > > >> > > > > document all the config changes?
> > >> > > > > > > > > >> > > >
> > >> > > > > > > > > >> > > > +1. A separate section for this would be
> good.
> > >> > > > > > > > > >> > > >
> > >> > > > > > > > > >> > > > best,
> > >> > > > > > > > > >> > > > Colin
> > >> > > > > > > > > >> > > >
> > >> > > > > > > > > >> > > > >
> > >> > > > > > > > > >> > > > > Thanks,
> > >> > > > > > > > > >> > > > >
> > >> > > > > > > > > >> > > > > Jun
> > >> > > > > > > > > >> > > > >
> > >> > > > > > > > > >> > > > > On Mon, Sep 25, 2023 at 4:29 PM Calvin Liu
> > >> > > > > > > > > >> > <caliu@confluent.io.invalid
> > >> > > > > > > > > >> > > >
> > >> > > > > > > > > >> > > > > wrote:
> > >> > > > > > > > > >> > > > >
> > >> > > > > > > > > >> > > > >> Hi Jun
> > >> > > > > > > > > >> > > > >> Thanks for the comments.
> > >> > > > > > > > > >> > > > >>
> > >> > > > > > > > > >> > > > >> 40. If we change to None, it is not
> > guaranteed
> > >> > for
> > >> > > no
> > >> > > > > > data
> > >> > > > > > > > > loss.
> > >> > > > > > > > > >> For
> > >> > > > > > > > > >> > > > users
> > >> > > > > > > > > >> > > > >> who are not able to validate the data with
> > >> > external
> > >> > > > > > > > resources,
> > >> > > > > > > > > >> > manual
> > >> > > > > > > > > >> > > > >> intervention does not give a better result
> > >> but a
> > >> > > loss
> > >> > > > > of
> > >> > > > > > > > > >> > availability.
> > >> > > > > > > > > >> > > > So
> > >> > > > > > > > > >> > > > >> practically speaking, the Balance mode
> would
> > >> be a
> > >> > > > > better
> > >> > > > > > > > > default
> > >> > > > > > > > > >> > > value.
> > >> > > > > > > > > >> > > > >>
> > >> > > > > > > > > >> > > > >> 41. No, it represents how we want to do
> the
> > >> > unclean
> > >> > > > > > leader
> > >> > > > > > > > > >> election.
> > >> > > > > > > > > >> > > If
> > >> > > > > > > > > >> > > > it
> > >> > > > > > > > > >> > > > >> is false, the unclean leader election will
> > be
> > >> the
> > >> > > old
> > >> > > > > > > random
> > >> > > > > > > > > way.
> > >> > > > > > > > > >> > > > >> Otherwise, the unclean recovery will be
> > used.
> > >> > > > > > > > > >> > > > >>
> > >> > > > > > > > > >> > > > >> 42. Good catch. Updated.
> > >> > > > > > > > > >> > > > >>
> > >> > > > > > > > > >> > > > >> 43. Only the first 20 topics will be
> served.
> > >> > Others
> > >> > > > > will
> > >> > > > > > be
> > >> > > > > > > > > >> returned
> > >> > > > > > > > > >> > > > with
> > >> > > > > > > > > >> > > > >> InvalidRequestError
> > >> > > > > > > > > >> > > > >>
> > >> > > > > > > > > >> > > > >> 44. The order matters. The desired leader
> > >> entries
> > >> > > > match
> > >> > > > > > > with
> > >> > > > > > > > > the
> > >> > > > > > > > > >> > topic
> > >> > > > > > > > > >> > > > >> partition list by the index.
> > >> > > > > > > > > >> > > > >>
> > >> > > > > > > > > >> > > > >> 45. Thanks! Updated.
> > >> > > > > > > > > >> > > > >>
> > >> > > > > > > > > >> > > > >> 46. Good advice! Updated.
> > >> > > > > > > > > >> > > > >>
> > >> > > > > > > > > >> > > > >> 47.1, updated the comment. Basically it
> will
> > >> > elect
> > >> > > > the
> > >> > > > > > > > replica
> > >> > > > > > > > > in
> > >> > > > > > > > > >> > the
> > >> > > > > > > > > >> > > > >> desiredLeader field to be the leader
> > >> > > > > > > > > >> > > > >>
> > >> > > > > > > > > >> > > > >> 47.2 We can let the admin client do the
> > >> > conversion.
> > >> > > > > Using
> > >> > > > > > > the
> > >> > > > > > > > > >> > > > desiredLeader
> > >> > > > > > > > > >> > > > >> field in the json format seems easier for
> > >> users.
> > >> > > > > > > > > >> > > > >>
> > >> > > > > > > > > >> > > > >> 48. Once the MV version is downgraded, all
> > the
> > >> > ELR
> > >> > > > > > related
> > >> > > > > > > > > fields
> > >> > > > > > > > > >> > will
> > >> > > > > > > > > >> > > > be
> > >> > > > > > > > > >> > > > >> removed on the next partition change. The
> > >> > > controller
> > >> > > > > will
> > >> > > > > > > > also
> > >> > > > > > > > > >> > ignore
> > >> > > > > > > > > >> > > > the
> > >> > > > > > > > > >> > > > >> ELR fields. Updated the KIP.
> > >> > > > > > > > > >> > > > >>
> > >> > > > > > > > > >> > > > >> 49. Yes, it would be deprecated/removed.
> > >> > > > > > > > > >> > > > >>
> > >> > > > > > > > > >> > > > >>
> > >> > > > > > > > > >> > > > >> On Mon, Sep 25, 2023 at 3:49 PM Jun Rao
> > >> > > > > > > > > <jun@confluent.io.invalid
> > >> > > > > > > > > >> >
> > >> > > > > > > > > >> > > > wrote:
> > >> > > > > > > > > >> > > > >>
> > >> > > > > > > > > >> > > > >> > Hi, Calvin,
> > >> > > > > > > > > >> > > > >> >
> > >> > > > > > > > > >> > > > >> > Thanks for the updated KIP. Made another
> > >> pass.
> > >> > A
> > >> > > > few
> > >> > > > > > more
> > >> > > > > > > > > >> comments
> > >> > > > > > > > > >> > > > below.
> > >> > > > > > > > > >> > > > >> >
> > >> > > > > > > > > >> > > > >> > 40. unclean.leader.election.enable.false
> > ->
> > >> > > > > > > > > >> > > > >> > unclean.recovery.strategy.Balanced: The
> > >> > Balanced
> > >> > > > mode
> > >> > > > > > > could
> > >> > > > > > > > > >> still
> > >> > > > > > > > > >> > > > lead to
> > >> > > > > > > > > >> > > > >> > data loss. So, I am wondering if
> > >> > > > > > > > > >> > > unclean.leader.election.enable.false
> > >> > > > > > > > > >> > > > >> > should map to None?
> > >> > > > > > > > > >> > > > >> >
> > >> > > > > > > > > >> > > > >> > 41. unclean.recovery.manager.enabled: I
> am
> > >> not
> > >> > > sure
> > >> > > > > why
> > >> > > > > > > we
> > >> > > > > > > > > >> > introduce
> > >> > > > > > > > > >> > > > this
> > >> > > > > > > > > >> > > > >> > additional config. Is it the same as
> > >> > > > > > > > > >> > unclean.recovery.strategy=None?
> > >> > > > > > > > > >> > > > >> >
> > >> > > > > > > > > >> > > > >> > 42.
> > >> > > > DescribeTopicResponse.TopicAuthorizedOperations:
> > >> > > > > > > Should
> > >> > > > > > > > > >> this
> > >> > > > > > > > > >> > be
> > >> > > > > > > > > >> > > at
> > >> > > > > > > > > >> > > > >> the
> > >> > > > > > > > > >> > > > >> > topic level?
> > >> > > > > > > > > >> > > > >> >
> > >> > > > > > > > > >> > > > >> > 43. "Limit: 20 topics max per request":
> > >> Could
> > >> > we
> > >> > > > > > describe
> > >> > > > > > > > > what
> > >> > > > > > > > > >> > > > happens if
> > >> > > > > > > > > >> > > > >> > the request includes more than 20
> topics?
> > >> > > > > > > > > >> > > > >> >
> > >> > > > > > > > > >> > > > >> > 44. ElectLeadersRequest.DesiredLeaders:
> > >> Could
> > >> > we
> > >> > > > > > describe
> > >> > > > > > > > > >> whether
> > >> > > > > > > > > >> > > the
> > >> > > > > > > > > >> > > > >> > ordering matters?
> > >> > > > > > > > > >> > > > >> >
> > >> > > > > > > > > >> > > > >> > 45. GetReplicaLogInfo.TopicPartitions:
> > >> "about":
> > >> > > > "The
> > >> > > > > > > topic
> > >> > > > > > > > > >> > > partitions
> > >> > > > > > > > > >> > > > to
> > >> > > > > > > > > >> > > > >> > elect leaders.": The description in
> > "about"
> > >> is
> > >> > > > > > incorrect.
> > >> > > > > > > > > >> > > > >> >
> > >> > > > > > > > > >> > > > >> > 46. GetReplicaLogInfoResponse: Should we
> > >> nest
> > >> > > > > > partitions
> > >> > > > > > > > > under
> > >> > > > > > > > > >> > > > topicId to
> > >> > > > > > > > > >> > > > >> > be consistent with other types of
> > responses?
> > >> > > > > > > > > >> > > > >> >
> > >> > > > > > > > > >> > > > >> > 47. kafka-leader-election.sh:
> > >> > > > > > > > > >> > > > >> > 47.1 Could we explain DESIGNATION?
> > >> > > > > > > > > >> > > > >> > 47.2 desiredLeader: Should it be a list
> to
> > >> > match
> > >> > > > the
> > >> > > > > > > field
> > >> > > > > > > > in
> > >> > > > > > > > > >> > > > >> > ElectLeadersRequest?
> > >> > > > > > > > > >> > > > >> >
> > >> > > > > > > > > >> > > > >> > 48. We could add a section on downgrade?
> > >> > > > > > > > > >> > > > >> >
> > >> > > > > > > > > >> > > > >> > 49. LastKnownLeader: This seems only
> > needed
> > >> in
> > >> > > the
> > >> > > > > > first
> > >> > > > > > > > > phase
> > >> > > > > > > > > >> of
> > >> > > > > > > > > >> > > > >> > delivering ELR. Will it be removed when
> > the
> > >> > > > complete
> > >> > > > > > KIP
> > >> > > > > > > is
> > >> > > > > > > > > >> > > delivered?
> > >> > > > > > > > > >> > > > >> >
> > >> > > > > > > > > >> > > > >> > Thanks,
> > >> > > > > > > > > >> > > > >> >
> > >> > > > > > > > > >> > > > >> > Jun
> > >> > > > > > > > > >> > > > >> >
> > >> > > > > > > > > >> > > > >> > On Tue, Sep 19, 2023 at 1:30 PM Colin
> > >> McCabe <
> > >> > > > > > > > > >> cmccabe@apache.org>
> > >> > > > > > > > > >> > > > wrote:
> > >> > > > > > > > > >> > > > >> >
> > >> > > > > > > > > >> > > > >> > > Hi Calvin,
> > >> > > > > > > > > >> > > > >> > >
> > >> > > > > > > > > >> > > > >> > > Thanks for the explanations. I like
> the
> > >> idea
> > >> > of
> > >> > > > > using
> > >> > > > > > > > none,
> > >> > > > > > > > > >> > > > balanced,
> > >> > > > > > > > > >> > > > >> > > aggressive. We also had an offline
> > >> discussion
> > >> > > > about
> > >> > > > > > why
> > >> > > > > > > > it
> > >> > > > > > > > > is
> > >> > > > > > > > > >> > good
> > >> > > > > > > > > >> > > > to
> > >> > > > > > > > > >> > > > >> > use a
> > >> > > > > > > > > >> > > > >> > > new config key (basically, so that we
> > can
> > >> > > > deprecate
> > >> > > > > > the
> > >> > > > > > > > old
> > >> > > > > > > > > >> one
> > >> > > > > > > > > >> > > > which
> > >> > > > > > > > > >> > > > >> had
> > >> > > > > > > > > >> > > > >> > > only false/true values in 4.0) With
> > these
> > >> > > > changes,
> > >> > > > > I
> > >> > > > > > am
> > >> > > > > > > > +1.
> > >> > > > > > > > > >> > > > >> > >
> > >> > > > > > > > > >> > > > >> > > best,
> > >> > > > > > > > > >> > > > >> > > Colin
> > >> > > > > > > > > >> > > > >> > >
> > >> > > > > > > > > >> > > > >> > > On Mon, Sep 18, 2023, at 15:54, Calvin
> > Liu
> > >> > > wrote:
> > >> > > > > > > > > >> > > > >> > > > Hi Colin,
> > >> > > > > > > > > >> > > > >> > > > Also, can we deprecate
> > >> > > > > > unclean.leader.election.enable
> > >> > > > > > > > in
> > >> > > > > > > > > >> 4.0?
> > >> > > > > > > > > >> > > > Before
> > >> > > > > > > > > >> > > > >> > > that,
> > >> > > > > > > > > >> > > > >> > > > we can have both the config
> > >> > > > > > unclean.recovery.strategy
> > >> > > > > > > > and
> > >> > > > > > > > > >> > > > >> > > > unclean.leader.election.enable
> > >> > > > > > > > > >> > > > >> > > > and using the
> unclean.recovery.Enabled
> > >> to
> > >> > > > > determine
> > >> > > > > > > > which
> > >> > > > > > > > > >> > config
> > >> > > > > > > > > >> > > > to
> > >> > > > > > > > > >> > > > >> use
> > >> > > > > > > > > >> > > > >> > > > during the unclean leader election.
> > >> > > > > > > > > >> > > > >> > > >
> > >> > > > > > > > > >> > > > >> > > > On Mon, Sep 18, 2023 at 3:51 PM
> Calvin
> > >> Liu
> > >> > <
> > >> > > > > > > > > >> > caliu@confluent.io>
> > >> > > > > > > > > >> > > > >> wrote:
> > >> > > > > > > > > >> > > > >> > > >
> > >> > > > > > > > > >> > > > >> > > >> Hi Colin,
> > >> > > > > > > > > >> > > > >> > > >> For the unclean.recovery.strategy
> > >> config
> > >> > > name,
> > >> > > > > how
> > >> > > > > > > > about
> > >> > > > > > > > > >> we
> > >> > > > > > > > > >> > use
> > >> > > > > > > > > >> > > > the
> > >> > > > > > > > > >> > > > >> > > >> following
> > >> > > > > > > > > >> > > > >> > > >> None. It basically means no unclean
> > >> > recovery
> > >> > > > > will
> > >> > > > > > be
> > >> > > > > > > > > >> > performed.
> > >> > > > > > > > > >> > > > >> > > >> Aggressive. It means availability
> > goes
> > >> > > first.
> > >> > > > > > > Whenever
> > >> > > > > > > > > the
> > >> > > > > > > > > >> > > > partition
> > >> > > > > > > > > >> > > > >> > > can't
> > >> > > > > > > > > >> > > > >> > > >> elect a durable replica, the
> > controller
> > >> > will
> > >> > > > try
> > >> > > > > > the
> > >> > > > > > > > > >> unclean
> > >> > > > > > > > > >> > > > >> recovery.
> > >> > > > > > > > > >> > > > >> > > >> Balanced. It is the balance point
> of
> > >> the
> > >> > > > > > > availability
> > >> > > > > > > > > >> > > > >> > first(Aggressive)
> > >> > > > > > > > > >> > > > >> > > >> and least availability(None). The
> > >> > controller
> > >> > > > > > > performs
> > >> > > > > > > > > >> unclean
> > >> > > > > > > > > >> > > > >> recovery
> > >> > > > > > > > > >> > > > >> > > when
> > >> > > > > > > > > >> > > > >> > > >> both ISR and ELR are empty.
> > >> > > > > > > > > >> > > > >> > > >>
> > >> > > > > > > > > >> > > > >> > > >>
> > >> > > > > > > > > >> > > > >> > > >> On Fri, Sep 15, 2023 at 11:42 AM
> > Calvin
> > >> > Liu
> > >> > > <
> > >> > > > > > > > > >> > > caliu@confluent.io>
> > >> > > > > > > > > >> > > > >> > wrote:
> > >> > > > > > > > > >> > > > >> > > >>
> > >> > > > > > > > > >> > > > >> > > >>> Hi Colin,
> > >> > > > > > > > > >> > > > >> > > >>>
> > >> > > > > > > > > >> > > > >> > > >>> > So, the proposal is that if
> > someone
> > >> > sets
> > >> > > > > > > > > >> > > > >> > > "unclean.leader.election.enable
> > >> > > > > > > > > >> > > > >> > > >>> = true"...
> > >> > > > > > > > > >> > > > >> > > >>>
> > >> > > > > > > > > >> > > > >> > > >>>
> > >> > > > > > > > > >> > > > >> > > >>> The idea is to use one of the
> > >> > > > > > > > > >> unclean.leader.election.enable
> > >> > > > > > > > > >> > > and
> > >> > > > > > > > > >> > > > >> > > >>> unclean.recovery.strategy based on
> > the
> > >> > > > > > > > > >> > > > unclean.recovery.Enabled. A
> > >> > > > > > > > > >> > > > >> > > possible
> > >> > > > > > > > > >> > > > >> > > >>> version can be
> > >> > > > > > > > > >> > > > >> > > >>>
> > >> > > > > > > > > >> > > > >> > > >>> If unclean.recovery.Enabled:
> > >> > > > > > > > > >> > > > >> > > >>>
> > >> > > > > > > > > >> > > > >> > > >>> {
> > >> > > > > > > > > >> > > > >> > > >>>
> > >> > > > > > > > > >> > > > >> > > >>> Check unclean.recovery.strategy.
> If
> > >> set,
> > >> > > use
> > >> > > > > it.
> > >> > > > > > > > > >> Otherwise,
> > >> > > > > > > > > >> > > > check
> > >> > > > > > > > > >> > > > >> > > >>> unclean.leader.election.enable and
> > >> > > translate
> > >> > > > it
> > >> > > > > > to
> > >> > > > > > > > > >> > > > >> > > >>> unclean.recovery.strategy.
> > >> > > > > > > > > >> > > > >> > > >>>
> > >> > > > > > > > > >> > > > >> > > >>> } else {
> > >> > > > > > > > > >> > > > >> > > >>>
> > >> > > > > > > > > >> > > > >> > > >>> Use unclean.leader.election.enable
> > >> > > > > > > > > >> > > > >> > > >>>
> > >> > > > > > > > > >> > > > >> > > >>> }
> > >> > > > > > > > > >> > > > >> > > >>>
> > >> > > > > > > > > >> > > > >> > > >>>
> > >> > > > > > > > > >> > > > >> > > >>> —--------
> > >> > > > > > > > > >> > > > >> > > >>>
> > >> > > > > > > > > >> > > > >> > > >>> >The configuration key should be
> > >> > > > > > > > > >> > > > >> "unclean.recovery.manager.enabled",
> > >> > > > > > > > > >> > > > >> > > >>> right?
> > >> > > > > > > > > >> > > > >> > > >>>
> > >> > > > > > > > > >> > > > >> > > >>>
> > >> > > > > > > > > >> > > > >> > > >>> I think we have two ways of
> > choosing a
> > >> > > leader
> > >> > > > > > > > > uncleanly,
> > >> > > > > > > > > >> > > unclean
> > >> > > > > > > > > >> > > > >> > leader
> > >> > > > > > > > > >> > > > >> > > >>> election and unclean recovery(log
> > >> > > inspection)
> > >> > > > > and
> > >> > > > > > > we
> > >> > > > > > > > > try
> > >> > > > > > > > > >> to
> > >> > > > > > > > > >> > > > switch
> > >> > > > > > > > > >> > > > >> > > between
> > >> > > > > > > > > >> > > > >> > > >>> them.
> > >> > > > > > > > > >> > > > >> > > >>>
> > >> > > > > > > > > >> > > > >> > > >>> Do you mean we want to develop two
> > >> ways
> > >> > of
> > >> > > > > > > performing
> > >> > > > > > > > > the
> > >> > > > > > > > > >> > > > unclean
> > >> > > > > > > > > >> > > > >> > > >>> recovery and one of them is using
> > >> > “unclean
> > >> > > > > > recovery
> > >> > > > > > > > > >> > manager”?
> > >> > > > > > > > > >> > > I
> > >> > > > > > > > > >> > > > >> guess
> > >> > > > > > > > > >> > > > >> > > we
> > >> > > > > > > > > >> > > > >> > > >>> haven’t discussed the second way.
> > >> > > > > > > > > >> > > > >> > > >>>
> > >> > > > > > > > > >> > > > >> > > >>>
> > >> > > > > > > > > >> > > > >> > > >>> —-------
> > >> > > > > > > > > >> > > > >> > > >>>
> > >> > > > > > > > > >> > > > >> > > >>> >How do these 4 levels of
> overrides
> > >> > > interact
> > >> > > > > with
> > >> > > > > > > > your
> > >> > > > > > > > > >> new
> > >> > > > > > > > > >> > > > >> > > >>> configurations?
> > >> > > > > > > > > >> > > > >> > > >>>
> > >> > > > > > > > > >> > > > >> > > >>>
> > >> > > > > > > > > >> > > > >> > > >>> I do notice in the Kraft
> controller
> > >> code,
> > >> > > the
> > >> > > > > > > method
> > >> > > > > > > > to
> > >> > > > > > > > > >> > check
> > >> > > > > > > > > >> > > > >> whether
> > >> > > > > > > > > >> > > > >> > > >>> perform unclean leader election is
> > >> hard
> > >> > > coded
> > >> > > > > to
> > >> > > > > > > > false
> > >> > > > > > > > > >> since
> > >> > > > > > > > > >> > > > >> > > >>>
> > >> > 2021(uncleanLeaderElectionEnabledForTopic).
> > >> > > > > Isn’t
> > >> > > > > > > it
> > >> > > > > > > > a
> > >> > > > > > > > > >> good
> > >> > > > > > > > > >> > > > chance
> > >> > > > > > > > > >> > > > >> to
> > >> > > > > > > > > >> > > > >> > > >>> completely deprecate the
> > >> > > > > > > > > unclean.leader.election.enable?
> > >> > > > > > > > > >> We
> > >> > > > > > > > > >> > > > don’t
> > >> > > > > > > > > >> > > > >> > even
> > >> > > > > > > > > >> > > > >> > > have
> > >> > > > > > > > > >> > > > >> > > >>> to worry about the config
> > conversion.
> > >> > > > > > > > > >> > > > >> > > >>>
> > >> > > > > > > > > >> > > > >> > > >>> On the other hand, whatever the
> > >> override
> > >> > > is,
> > >> > > > as
> > >> > > > > > > long
> > >> > > > > > > > as
> > >> > > > > > > > > >> the
> > >> > > > > > > > > >> > > > >> > controller
> > >> > > > > > > > > >> > > > >> > > >>> can have the final effective
> > >> > > > > > > > > >> unclean.leader.election.enable,
> > >> > > > > > > > > >> > > the
> > >> > > > > > > > > >> > > > >> > topic
> > >> > > > > > > > > >> > > > >> > > >>> level config
> > >> unclean.recovery.strategy,
> > >> > the
> > >> > > > > > cluster
> > >> > > > > > > > > level
> > >> > > > > > > > > >> > > config
> > >> > > > > > > > > >> > > > >> > > >>> unclean.recovery.Enabled, the
> > >> controller
> > >> > > can
> > >> > > > > > > > calculate
> > >> > > > > > > > > >> the
> > >> > > > > > > > > >> > > > correct
> > >> > > > > > > > > >> > > > >> > > methods
> > >> > > > > > > > > >> > > > >> > > >>> to use right?
> > >> > > > > > > > > >> > > > >> > > >>>
> > >> > > > > > > > > >> > > > >> > > >>>
> > >> > > > > > > > > >> > > > >> > > >>> On Fri, Sep 15, 2023 at 10:02 AM
> > Colin
> > >> > > > McCabe <
> > >> > > > > > > > > >> > > > cmccabe@apache.org>
> > >> > > > > > > > > >> > > > >> > > wrote:
> > >> > > > > > > > > >> > > > >> > > >>>
> > >> > > > > > > > > >> > > > >> > > >>>> On Thu, Sep 14, 2023, at 22:23,
> > >> Calvin
> > >> > Liu
> > >> > > > > > wrote:
> > >> > > > > > > > > >> > > > >> > > >>>> > Hi Colin
> > >> > > > > > > > > >> > > > >> > > >>>> > 1. I think using the new config
> > >> name
> > >> > is
> > >> > > > more
> > >> > > > > > > > clear.
> > >> > > > > > > > > >> > > > >> > > >>>> >        a. The unclean leader
> > >> election
> > >> > is
> > >> > > > > > > actually
> > >> > > > > > > > > >> removed
> > >> > > > > > > > > >> > > if
> > >> > > > > > > > > >> > > > >> > unclean
> > >> > > > > > > > > >> > > > >> > > >>>> > recovery is in use.
> > >> > > > > > > > > >> > > > >> > > >>>> >        b. Using multiple values
> > in
> > >> > > > > > > > > >> > > > >> unclean.leader.election.enable
> > >> > > > > > > > > >> > > > >> > is
> > >> > > > > > > > > >> > > > >> > > >>>> > confusing and it will be more
> > >> > confusing
> > >> > > > > after
> > >> > > > > > > > people
> > >> > > > > > > > > >> > forget
> > >> > > > > > > > > >> > > > >> about
> > >> > > > > > > > > >> > > > >> > > this
> > >> > > > > > > > > >> > > > >> > > >>>> > discussion.
> > >> > > > > > > > > >> > > > >> > > >>>>
> > >> > > > > > > > > >> > > > >> > > >>>> Hi Calvin,
> > >> > > > > > > > > >> > > > >> > > >>>>
> > >> > > > > > > > > >> > > > >> > > >>>> So, the proposal is that if
> someone
> > >> sets
> > >> > > > > > > > > >> > > > >> > > "unclean.leader.election.enable
> > >> > > > > > > > > >> > > > >> > > >>>> = true" but then sets one of your
> > new
> > >> > > > > > > > configurations,
> > >> > > > > > > > > >> the
> > >> > > > > > > > > >> > > > value of
> > >> > > > > > > > > >> > > > >> > > >>>> unclean.leader.election.enable is
> > >> > ignored?
> > >> > > > > That
> > >> > > > > > > > seems
> > >> > > > > > > > > >> less
> > >> > > > > > > > > >> > > > clear
> > >> > > > > > > > > >> > > > >> to
> > >> > > > > > > > > >> > > > >> > > me, not
> > >> > > > > > > > > >> > > > >> > > >>>> more. Just in general, having
> > >> multiple
> > >> > > > > > > configuration
> > >> > > > > > > > > >> keys
> > >> > > > > > > > > >> > to
> > >> > > > > > > > > >> > > > >> control
> > >> > > > > > > > > >> > > > >> > > the
> > >> > > > > > > > > >> > > > >> > > >>>> same thing confuses users.
> > Basically,
> > >> > they
> > >> > > > are
> > >> > > > > > > > sitting
> > >> > > > > > > > > >> at a
> > >> > > > > > > > > >> > > > giant
> > >> > > > > > > > > >> > > > >> > > control
> > >> > > > > > > > > >> > > > >> > > >>>> panel, and some of the levers do
> > >> > nothing.
> > >> > > > > > > > > >> > > > >> > > >>>>
> > >> > > > > > > > > >> > > > >> > > >>>> > 2. Sorry I forgot to mention in
> > the
> > >> > > > response
> > >> > > > > > > that
> > >> > > > > > > > I
> > >> > > > > > > > > >> did
> > >> > > > > > > > > >> > add
> > >> > > > > > > > > >> > > > the
> > >> > > > > > > > > >> > > > >> > > >>>> > unclean.recovery.Enabled flag.
> > >> > > > > > > > > >> > > > >> > > >>>>
> > >> > > > > > > > > >> > > > >> > > >>>> The configuration key should be
> > >> > > > > > > > > >> > > > >> "unclean.recovery.manager.enabled",
> > >> > > > > > > > > >> > > > >> > > >>>> right? Becuase we can do "unclean
> > >> > > recovery"
> > >> > > > > > > without
> > >> > > > > > > > > the
> > >> > > > > > > > > >> > > > manager.
> > >> > > > > > > > > >> > > > >> > > Disabling
> > >> > > > > > > > > >> > > > >> > > >>>> the manager just means we use a
> > >> > different
> > >> > > > > > > mechanism
> > >> > > > > > > > > for
> > >> > > > > > > > > >> > > > recovery.
> > >> > > > > > > > > >> > > > >> > > >>>>
> > >> > > > > > > > > >> > > > >> > > >>>> >        c. Maybe I
> underestimated
> > >> the
> > >> > > > > challenge
> > >> > > > > > > of
> > >> > > > > > > > > >> > replacing
> > >> > > > > > > > > >> > > > the
> > >> > > > > > > > > >> > > > >> > > >>>> config. Any
> > >> > > > > > > > > >> > > > >> > > >>>> > implementation problems ahead?
> > >> > > > > > > > > >> > > > >> > > >>>>
> > >> > > > > > > > > >> > > > >> > > >>>> There are four levels of
> overrides
> > >> for
> > >> > > > > > > > > >> > > > >> > unclean.leader.election.enable.
> > >> > > > > > > > > >> > > > >> > > >>>>
> > >> > > > > > > > > >> > > > >> > > >>>> 1. static configuration for node.
> > >> > > > > > > > > >> > > > >> > > >>>>     This goes in the
> configuration
> > >> file,
> > >> > > > > > typically
> > >> > > > > > > > > named
> > >> > > > > > > > > >> > > > >> > > >>>> server.properties
> > >> > > > > > > > > >> > > > >> > > >>>>
> > >> > > > > > > > > >> > > > >> > > >>>> 2. dynamic configuration for node
> > >> > default
> > >> > > > > > > > > >> > > > >> > > >>>>   ConfigResource(type=BROKER,
> > >> name="")
> > >> > > > > > > > > >> > > > >> > > >>>>
> > >> > > > > > > > > >> > > > >> > > >>>> 3. dynamic configuration for node
> > >> > > > > > > > > >> > > > >> > > >>>>   ConfigResource(type=BROKER,
> > >> > > > name=<controller
> > >> > > > > > > id>)
> > >> > > > > > > > > >> > > > >> > > >>>>
> > >> > > > > > > > > >> > > > >> > > >>>> 4. dynamic configuration for
> topic
> > >> > > > > > > > > >> > > > >> > > >>>>   ConfigResource(type=TOPIC,
> > >> > > > > name=<topic-name>)
> > >> > > > > > > > > >> > > > >> > > >>>>
> > >> > > > > > > > > >> > > > >> > > >>>> How do these 4 levels of
> overrides
> > >> > > interact
> > >> > > > > with
> > >> > > > > > > > your
> > >> > > > > > > > > >> new
> > >> > > > > > > > > >> > > > >> > > >>>> configurations? If the new
> > >> > configurations
> > >> > > > > > dominate
> > >> > > > > > > > > over
> > >> > > > > > > > > >> the
> > >> > > > > > > > > >> > > old
> > >> > > > > > > > > >> > > > >> > ones,
> > >> > > > > > > > > >> > > > >> > > it
> > >> > > > > > > > > >> > > > >> > > >>>> seems like this will get a lot
> more
> > >> > > > confusing
> > >> > > > > to
> > >> > > > > > > > > >> implement
> > >> > > > > > > > > >> > > (and
> > >> > > > > > > > > >> > > > >> also
> > >> > > > > > > > > >> > > > >> > > to
> > >> > > > > > > > > >> > > > >> > > >>>> use.)
> > >> > > > > > > > > >> > > > >> > > >>>>
> > >> > > > > > > > > >> > > > >> > > >>>> Again, I'd recommend just adding
> > some
> > >> > new
> > >> > > > > values
> > >> > > > > > > to
> > >> > > > > > > > > >> > > > >> > > >>>> unclean.leader.election.enable.
> > It's
> > >> > > simple
> > >> > > > > and
> > >> > > > > > > will
> > >> > > > > > > > > >> > prevent
> > >> > > > > > > > > >> > > > user
> > >> > > > > > > > > >> > > > >> > > confusion
> > >> > > > > > > > > >> > > > >> > > >>>> (as well as developer confusion.)
> > >> > > > > > > > > >> > > > >> > > >>>>
> > >> > > > > > > > > >> > > > >> > > >>>> best,
> > >> > > > > > > > > >> > > > >> > > >>>> Colin
> > >> > > > > > > > > >> > > > >> > > >>>>
> > >> > > > > > > > > >> > > > >> > > >>>>
> > >> > > > > > > > > >> > > > >> > > >>>> > 3. About the admin client, I
> > >> > mentioned 3
> > >> > > > > > changes
> > >> > > > > > > > in
> > >> > > > > > > > > >> the
> > >> > > > > > > > > >> > > > client.
> > >> > > > > > > > > >> > > > >> > > >>>> Anything
> > >> > > > > > > > > >> > > > >> > > >>>> > else I missed in the KIP?
> > >> > > > > > > > > >> > > > >> > > >>>> >       a. The client will switch
> > to
> > >> > using
> > >> > > > the
> > >> > > > > > new
> > >> > > > > > > > RPC
> > >> > > > > > > > > >> > > instead
> > >> > > > > > > > > >> > > > of
> > >> > > > > > > > > >> > > > >> > > >>>> > MetadataRequest for the topics.
> > >> > > > > > > > > >> > > > >> > > >>>> >       b. The TopicPartitionInfo
> > >> used
> > >> > in
> > >> > > > > > > > > >> TopicDescription
> > >> > > > > > > > > >> > > > needs
> > >> > > > > > > > > >> > > > >> to
> > >> > > > > > > > > >> > > > >> > > add
> > >> > > > > > > > > >> > > > >> > > >>>> new
> > >> > > > > > > > > >> > > > >> > > >>>> > fields related to the ELR.
> > >> > > > > > > > > >> > > > >> > > >>>> >       c. The outputs will add
> the
> > >> ELR
> > >> > > > > related
> > >> > > > > > > > > fields.
> > >> > > > > > > > > >> > > > >> > > >>>> >
> > >> > > > > > > > > >> > > > >> > > >>>> > On Thu, Sep 14, 2023 at 9:19 PM
> > >> Colin
> > >> > > > > McCabe <
> > >> > > > > > > > > >> > > > >> cmccabe@apache.org>
> > >> > > > > > > > > >> > > > >> > > >>>> wrote:
> > >> > > > > > > > > >> > > > >> > > >>>> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> Hi Calvin,
> > >> > > > > > > > > >> > > > >> > > >>>> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> Thanks for the changes.
> > >> > > > > > > > > >> > > > >> > > >>>> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> 1. Earlier I commented that
> > >> creating
> > >> > > > > > > > > >> > > > >> "unclean.recovery.strategy "
> > >> > > > > > > > > >> > > > >> > > is
> > >> > > > > > > > > >> > > > >> > > >>>> not
> > >> > > > > > > > > >> > > > >> > > >>>> >> necessary, and we can just
> reuse
> > >> the
> > >> > > > > existing
> > >> > > > > > > > > >> > > > >> > > >>>> >>
> "unclean.leader.election.enable"
> > >> > > > > > configuration
> > >> > > > > > > > key.
> > >> > > > > > > > > >> > Let's
> > >> > > > > > > > > >> > > > >> discuss
> > >> > > > > > > > > >> > > > >> > > >>>> that.
> > >> > > > > > > > > >> > > > >> > > >>>> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> 2.I also don't understand why
> > you
> > >> > > didn't
> > >> > > > > add
> > >> > > > > > a
> > >> > > > > > > > > >> > > > configuration to
> > >> > > > > > > > > >> > > > >> > > >>>> enable or
> > >> > > > > > > > > >> > > > >> > > >>>> >> disable the Unclean Recovery
> > >> Manager.
> > >> > > > This
> > >> > > > > > > seems
> > >> > > > > > > > > >> like a
> > >> > > > > > > > > >> > > very
> > >> > > > > > > > > >> > > > >> > simple
> > >> > > > > > > > > >> > > > >> > > >>>> way to
> > >> > > > > > > > > >> > > > >> > > >>>> >> handle the staging issue which
> > we
> > >> > > > > discussed.
> > >> > > > > > > The
> > >> > > > > > > > > URM
> > >> > > > > > > > > >> can
> > >> > > > > > > > > >> > > > just
> > >> > > > > > > > > >> > > > >> be
> > >> > > > > > > > > >> > > > >> > > >>>> turned off
> > >> > > > > > > > > >> > > > >> > > >>>> >> until it is production ready.
> > >> Let's
> > >> > > > discuss
> > >> > > > > > > this.
> > >> > > > > > > > > >> > > > >> > > >>>> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> 3. You still need to describe
> > the
> > >> > > changes
> > >> > > > > to
> > >> > > > > > > > > >> AdminClient
> > >> > > > > > > > > >> > > > that
> > >> > > > > > > > > >> > > > >> are
> > >> > > > > > > > > >> > > > >> > > >>>> needed
> > >> > > > > > > > > >> > > > >> > > >>>> >> to use DescribeTopicRequest.
> > >> > > > > > > > > >> > > > >> > > >>>> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> Keep at it. It's looking
> better.
> > >> :)
> > >> > > > > > > > > >> > > > >> > > >>>> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> best,
> > >> > > > > > > > > >> > > > >> > > >>>> >> Colin
> > >> > > > > > > > > >> > > > >> > > >>>> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> On Thu, Sep 14, 2023, at
> 11:03,
> > >> > Calvin
> > >> > > > Liu
> > >> > > > > > > wrote:
> > >> > > > > > > > > >> > > > >> > > >>>> >> > Hi Colin
> > >> > > > > > > > > >> > > > >> > > >>>> >> > Thanks for the comments!
> > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> > I did the following changes
> > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >    1.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >    Simplified the API spec
> > >> section
> > >> > to
> > >> > > > > only
> > >> > > > > > > > > include
> > >> > > > > > > > > >> the
> > >> > > > > > > > > >> > > > diff.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >    2.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >    Reordered the HWM
> > requirement
> > >> > > > section.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >    3.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >    Removed the URM
> > >> implementation
> > >> > > > details
> > >> > > > > > to
> > >> > > > > > > > keep
> > >> > > > > > > > > >> the
> > >> > > > > > > > > >> > > > >> necessary
> > >> > > > > > > > > >> > > > >> > > >>>> >> >    characteristics to
> perform
> > >> the
> > >> > > > unclean
> > >> > > > > > > > > recovery.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >    1.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >       When to perform the
> > >> unclean
> > >> > > > > recovery
> > >> > > > > > > > > >> > > > >> > > >>>> >> >       2.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >       Under different
> config,
> > >> how
> > >> > the
> > >> > > > > > unclean
> > >> > > > > > > > > >> recovery
> > >> > > > > > > > > >> > > > finds
> > >> > > > > > > > > >> > > > >> > the
> > >> > > > > > > > > >> > > > >> > > >>>> leader.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >       3.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >       How the config
> > >> > > > > > > > > unclean.leader.election.enable
> > >> > > > > > > > > >> > and
> > >> > > > > > > > > >> > > > >> > > >>>> >> >
> >  unclean.recovery.strategy
> > >> are
> > >> > > > > > converted
> > >> > > > > > > > > when
> > >> > > > > > > > > >> > users
> > >> > > > > > > > > >> > > > >> > > >>>> enable/disable
> > >> > > > > > > > > >> > > > >> > > >>>> >> the
> > >> > > > > > > > > >> > > > >> > > >>>> >> >       unclean recovery.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >       4.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >    More details about how we
> > >> change
> > >> > > > admin
> > >> > > > > > > > client.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >    5.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >    API limits on the
> > >> > > > > > GetReplicaLogInfoRequest
> > >> > > > > > > > and
> > >> > > > > > > > > >> > > > >> > > >>>> DescribeTopicRequest.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >    6.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >    Two metrics added
> > >> > > > > > > > > >> > > > >> > > >>>> >> >    1.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > >> > > > > > > > > >> > > >
> > >> > Kafka.controller.global_under_min_isr_partition_count
> > >> > > > > > > > > >> > > > >> > > >>>> >> >       2.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > >> > > > > > > > > >>  kafka.controller.unclean_recovery_finished_count
> > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> > On Wed, Sep 13, 2023 at
> > 10:46 AM
> > >> > > Colin
> > >> > > > > > > McCabe <
> > >> > > > > > > > > >> > > > >> > > cmccabe@apache.org>
> > >> > > > > > > > > >> > > > >> > > >>>> >> wrote:
> > >> > > > > > > > > >> > > > >> > > >>>> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> On Tue, Sep 12, 2023, at
> > 17:21,
> > >> > > Calvin
> > >> > > > > Liu
> > >> > > > > > > > > wrote:
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Hi Colin
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Thanks for the comments!
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> Hi Calvin,
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> Thanks again for the KIP.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> One meta-comment: it's
> > usually
> > >> > > better
> > >> > > > to
> > >> > > > > > > just
> > >> > > > > > > > > do a
> > >> > > > > > > > > >> > diff
> > >> > > > > > > > > >> > > > on a
> > >> > > > > > > > > >> > > > >> > > >>>> message
> > >> > > > > > > > > >> > > > >> > > >>>> >> spec
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> file or java file if you're
> > >> > > including
> > >> > > > > > > changes
> > >> > > > > > > > to
> > >> > > > > > > > > >> it
> > >> > > > > > > > > >> > in
> > >> > > > > > > > > >> > > > the
> > >> > > > > > > > > >> > > > >> > KIP.
> > >> > > > > > > > > >> > > > >> > > >>>> This is
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> easier to read than looking
> > for
> > >> > "new
> > >> > > > > > fields
> > >> > > > > > > > > begin"
> > >> > > > > > > > > >> > etc.
> > >> > > > > > > > > >> > > > in
> > >> > > > > > > > > >> > > > >> the
> > >> > > > > > > > > >> > > > >> > > >>>> text, and
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> gracefully handles the case
> > >> where
> > >> > > > > existing
> > >> > > > > > > > > fields
> > >> > > > > > > > > >> > were
> > >> > > > > > > > > >> > > > >> > changed.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Rewrite the Additional
> High
> > >> > > > Watermark
> > >> > > > > > > > > >> advancement
> > >> > > > > > > > > >> > > > >> > requirement
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> > There was feedback on
> this
> > >> > section
> > >> > > > > that
> > >> > > > > > > some
> > >> > > > > > > > > >> > readers
> > >> > > > > > > > > >> > > > may
> > >> > > > > > > > > >> > > > >> not
> > >> > > > > > > > > >> > > > >> > > be
> > >> > > > > > > > > >> > > > >> > > >>>> >> familiar
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> > with HWM and Ack=0,1,all
> > >> > requests.
> > >> > > > > This
> > >> > > > > > > can
> > >> > > > > > > > > help
> > >> > > > > > > > > >> > them
> > >> > > > > > > > > >> > > > >> > > understand
> > >> > > > > > > > > >> > > > >> > > >>>> the
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> > proposal. I will rewrite
> > this
> > >> > part
> > >> > > > for
> > >> > > > > > > more
> > >> > > > > > > > > >> > > > readability.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> To be clear, I wasn't
> > >> suggesting
> > >> > > > > dropping
> > >> > > > > > > > either
> > >> > > > > > > > > >> > > > section. I
> > >> > > > > > > > > >> > > > >> > > agree
> > >> > > > > > > > > >> > > > >> > > >>>> that
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> they add useful
> background. I
> > >> was
> > >> > > just
> > >> > > > > > > > > suggesting
> > >> > > > > > > > > >> > that
> > >> > > > > > > > > >> > > we
> > >> > > > > > > > > >> > > > >> > should
> > >> > > > > > > > > >> > > > >> > > >>>> discuss
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> the "acks" setting AFTER
> > >> > discussing
> > >> > > > the
> > >> > > > > > new
> > >> > > > > > > > high
> > >> > > > > > > > > >> > > > watermark
> > >> > > > > > > > > >> > > > >> > > >>>> advancement
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> conditions. We also should
> > >> discuss
> > >> > > > > acks=0.
> > >> > > > > > > > While
> > >> > > > > > > > > >> it
> > >> > > > > > > > > >> > > isn't
> > >> > > > > > > > > >> > > > >> > > >>>> conceptually
> > >> > > > > > > > > >> > > > >> > > >>>> >> much
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> different than acks=1 here,
> > its
> > >> > > > omission
> > >> > > > > > > from
> > >> > > > > > > > > this
> > >> > > > > > > > > >> > > > section
> > >> > > > > > > > > >> > > > >> is
> > >> > > > > > > > > >> > > > >> > > >>>> confusing.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Unclean recovery
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> > The plan is to replace
> the
> > >> > > > > > > > > >> > > > unclean.leader.election.enable
> > >> > > > > > > > > >> > > > >> > with
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> unclean.recovery.strategy.
> > If
> > >> > the
> > >> > > > > > Unclean
> > >> > > > > > > > > >> Recovery
> > >> > > > > > > > > >> > is
> > >> > > > > > > > > >> > > > >> > enabled
> > >> > > > > > > > > >> > > > >> > > >>>> then it
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> deals
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> > with the three options in
> > the
> > >> > > > > > > > > >> > > > unclean.recovery.strategy.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Let’s refine the Unclean
> > >> > Recovery.
> > >> > > > We
> > >> > > > > > have
> > >> > > > > > > > > >> already
> > >> > > > > > > > > >> > > > taken a
> > >> > > > > > > > > >> > > > >> > > lot of
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> > suggestions and I hope to
> > >> > enhance
> > >> > > > the
> > >> > > > > > > > > >> durability of
> > >> > > > > > > > > >> > > > Kafka
> > >> > > > > > > > > >> > > > >> to
> > >> > > > > > > > > >> > > > >> > > the
> > >> > > > > > > > > >> > > > >> > > >>>> next
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> level
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> > with this KIP.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> I am OK with doing the
> > unclean
> > >> > > leader
> > >> > > > > > > recovery
> > >> > > > > > > > > >> > > > improvements
> > >> > > > > > > > > >> > > > >> in
> > >> > > > > > > > > >> > > > >> > > >>>> this KIP.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> However, I think we need to
> > >> really
> > >> > > > work
> > >> > > > > on
> > >> > > > > > > the
> > >> > > > > > > > > >> > > > configuration
> > >> > > > > > > > > >> > > > >> > > >>>> settings.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> Configuration overrides are
> > >> often
> > >> > > > quite
> > >> > > > > > > messy.
> > >> > > > > > > > > For
> > >> > > > > > > > > >> > > > example,
> > >> > > > > > > > > >> > > > >> > the
> > >> > > > > > > > > >> > > > >> > > >>>> cases
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> where we have
> log.roll.hours
> > >> and
> > >> > > > > > > > > >> log.roll.segment.ms
> > >> > > > > > > > > >> > ,
> > >> > > > > > > > > >> > > > the
> > >> > > > > > > > > >> > > > >> > user
> > >> > > > > > > > > >> > > > >> > > >>>> has to
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> remember which one takes
> > >> > precedence,
> > >> > > > and
> > >> > > > > > it
> > >> > > > > > > is
> > >> > > > > > > > > not
> > >> > > > > > > > > >> > > > obvious.
> > >> > > > > > > > > >> > > > >> > So,
> > >> > > > > > > > > >> > > > >> > > >>>> rather
> > >> > > > > > > > > >> > > > >> > > >>>> >> than
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> creating a new
> configuration,
> > >> why
> > >> > > not
> > >> > > > > add
> > >> > > > > > > > > >> additional
> > >> > > > > > > > > >> > > > values
> > >> > > > > > > > > >> > > > >> to
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> "unclean.leader.election.enable"?
> > >> > I
> > >> > > > > think
> > >> > > > > > > this
> > >> > > > > > > > > >> will
> > >> > > > > > > > > >> > be
> > >> > > > > > > > > >> > > > >> simpler
> > >> > > > > > > > > >> > > > >> > > for
> > >> > > > > > > > > >> > > > >> > > >>>> >> people
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> to understand, and simpler
> in
> > >> the
> > >> > > code
> > >> > > > > as
> > >> > > > > > > > well.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> What if we continued to use
> > >> > > > > > > > > >> > > > "unclean.leader.election.enable"
> > >> > > > > > > > > >> > > > >> > but
> > >> > > > > > > > > >> > > > >> > > >>>> >> extended
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> it so that it took a
> string?
> > >> Then
> > >> > > the
> > >> > > > > > string
> > >> > > > > > > > > could
> > >> > > > > > > > > >> > have
> > >> > > > > > > > > >> > > > >> these
> > >> > > > > > > > > >> > > > >> > > >>>> values:
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> never
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>     never automatically do
> an
> > >> > > unclean
> > >> > > > > > leader
> > >> > > > > > > > > >> election
> > >> > > > > > > > > >> > > > under
> > >> > > > > > > > > >> > > > >> > any
> > >> > > > > > > > > >> > > > >> > > >>>> >> conditions
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> false / default
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>     only do an unclean
> leader
> > >> > > election
> > >> > > > > if
> > >> > > > > > > > there
> > >> > > > > > > > > >> may
> > >> > > > > > > > > >> > be
> > >> > > > > > > > > >> > > > >> > possible
> > >> > > > > > > > > >> > > > >> > > >>>> data
> > >> > > > > > > > > >> > > > >> > > >>>> >> loss
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> true / always
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>     always do an unclean
> > leader
> > >> > > > election
> > >> > > > > > if
> > >> > > > > > > we
> > >> > > > > > > > > >> can't
> > >> > > > > > > > > >> > > > >> > immediately
> > >> > > > > > > > > >> > > > >> > > >>>> elect a
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> leader
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> It's a bit awkward that
> false
> > >> maps
> > >> > > to
> > >> > > > > > > default
> > >> > > > > > > > > >> rather
> > >> > > > > > > > > >> > > > than to
> > >> > > > > > > > > >> > > > >> > > >>>> never. But
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> this awkwardness exists if
> we
> > >> use
> > >> > > two
> > >> > > > > > > > different
> > >> > > > > > > > > >> > > > >> configuration
> > >> > > > > > > > > >> > > > >> > > keys
> > >> > > > > > > > > >> > > > >> > > >>>> as
> > >> > > > > > > > > >> > > > >> > > >>>> >> well.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> The reason for the
> > awkwardness
> > >> is
> > >> > > that
> > >> > > > > we
> > >> > > > > > > > simply
> > >> > > > > > > > > >> > don't
> > >> > > > > > > > > >> > > > want
> > >> > > > > > > > > >> > > > >> > most
> > >> > > > > > > > > >> > > > >> > > >>>> of the
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> people currently setting
> > >> > > > > > > > > >> > > > >> unclean.leader.election.enable=false
> > >> > > > > > > > > >> > > > >> > to
> > >> > > > > > > > > >> > > > >> > > >>>> get the
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> "never" behavior. We have
> to
> > >> bite
> > >> > > that
> > >> > > > > > > bullet.
> > >> > > > > > > > > >> Better
> > >> > > > > > > > > >> > > to
> > >> > > > > > > > > >> > > > be
> > >> > > > > > > > > >> > > > >> > > clear
> > >> > > > > > > > > >> > > > >> > > >>>> and
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> explicit than hide it.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> Another thing that's a bit
> > >> awkward
> > >> > > is
> > >> > > > > > having
> > >> > > > > > > > two
> > >> > > > > > > > > >> > > > different
> > >> > > > > > > > > >> > > > >> > ways
> > >> > > > > > > > > >> > > > >> > > to
> > >> > > > > > > > > >> > > > >> > > >>>> do
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> unclean leader election
> > >> specified
> > >> > in
> > >> > > > the
> > >> > > > > > > KIP.
> > >> > > > > > > > > You
> > >> > > > > > > > > >> > > > descirbe
> > >> > > > > > > > > >> > > > >> two
> > >> > > > > > > > > >> > > > >> > > >>>> methods:
> > >> > > > > > > > > >> > > > >> > > >>>> >> the
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> simple "choose the last
> > leader"
> > >> > > > method,
> > >> > > > > > and
> > >> > > > > > > > the
> > >> > > > > > > > > >> > > "unclean
> > >> > > > > > > > > >> > > > >> > > recovery
> > >> > > > > > > > > >> > > > >> > > >>>> >> manager"
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> method. I understand why
> you
> > >> did
> > >> > it
> > >> > > > this
> > >> > > > > > way
> > >> > > > > > > > --
> > >> > > > > > > > > >> > "choose
> > >> > > > > > > > > >> > > > the
> > >> > > > > > > > > >> > > > >> > last
> > >> > > > > > > > > >> > > > >> > > >>>> >> leader" is
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> simple, and will help us
> > >> deliver
> > >> > an
> > >> > > > > > > > > implementation
> > >> > > > > > > > > >> > > > quickly,
> > >> > > > > > > > > >> > > > >> > > while
> > >> > > > > > > > > >> > > > >> > > >>>> the
> > >> > > > > > > > > >> > > > >> > > >>>> >> URM
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> is preferable in the long
> > >> term. My
> > >> > > > > > > suggestion
> > >> > > > > > > > > >> here is
> > >> > > > > > > > > >> > > to
> > >> > > > > > > > > >> > > > >> > > separate
> > >> > > > > > > > > >> > > > >> > > >>>> the
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> decision of HOW to do
> unclean
> > >> > leader
> > >> > > > > > > election
> > >> > > > > > > > > from
> > >> > > > > > > > > >> > the
> > >> > > > > > > > > >> > > > >> > decision
> > >> > > > > > > > > >> > > > >> > > of
> > >> > > > > > > > > >> > > > >> > > >>>> WHEN
> > >> > > > > > > > > >> > > > >> > > >>>> >> to
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> do it.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> So in other words, have
> > >> > > > > > > > > >> > > "unclean.leader.election.enable"
> > >> > > > > > > > > >> > > > >> > specify
> > >> > > > > > > > > >> > > > >> > > >>>> when we
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> do unclean leader election,
> > and
> > >> > > have a
> > >> > > > > new
> > >> > > > > > > > > >> > > configuration
> > >> > > > > > > > > >> > > > >> like
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> "unclean.recovery.manager.enable"
> > >> > to
> > >> > > > > > > determine
> > >> > > > > > > > > if
> > >> > > > > > > > > >> we
> > >> > > > > > > > > >> > > use
> > >> > > > > > > > > >> > > > the
> > >> > > > > > > > > >> > > > >> > > URM.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> Presumably the URM will
> take
> > >> some
> > >> > > time
> > >> > > > > to
> > >> > > > > > > get
> > >> > > > > > > > > >> fully
> > >> > > > > > > > > >> > > > stable,
> > >> > > > > > > > > >> > > > >> so
> > >> > > > > > > > > >> > > > >> > > >>>> this can
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> default to false for a
> while,
> > >> and
> > >> > we
> > >> > > > can
> > >> > > > > > > flip
> > >> > > > > > > > > the
> > >> > > > > > > > > >> > > > default to
> > >> > > > > > > > > >> > > > >> > > true
> > >> > > > > > > > > >> > > > >> > > >>>> when
> > >> > > > > > > > > >> > > > >> > > >>>> >> we
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> feel ready.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> The URM is somewhat
> > >> > under-described
> > >> > > > > here.
> > >> > > > > > I
> > >> > > > > > > > > think
> > >> > > > > > > > > >> we
> > >> > > > > > > > > >> > > > need a
> > >> > > > > > > > > >> > > > >> > few
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> configurations here for it.
> > For
> > >> > > > example,
> > >> > > > > > we
> > >> > > > > > > > > need a
> > >> > > > > > > > > >> > > > >> > > configuration to
> > >> > > > > > > > > >> > > > >> > > >>>> >> specify
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> how long it should wait
> for a
> > >> > broker
> > >> > > > to
> > >> > > > > > > > respond
> > >> > > > > > > > > to
> > >> > > > > > > > > >> > its
> > >> > > > > > > > > >> > > > RPCs
> > >> > > > > > > > > >> > > > >> > > before
> > >> > > > > > > > > >> > > > >> > > >>>> >> moving
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> on. We also need to
> > understand
> > >> how
> > >> > > the
> > >> > > > > URM
> > >> > > > > > > > > >> interacts
> > >> > > > > > > > > >> > > with
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > unclean.leader.election.enable=always. I
> > >> > > > > > > > assume
> > >> > > > > > > > > >> that
> > >> > > > > > > > > >> > > with
> > >> > > > > > > > > >> > > > >> > > "always"
> > >> > > > > > > > > >> > > > >> > > >>>> we
> > >> > > > > > > > > >> > > > >> > > >>>> >> will
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> just unconditionally use
> the
> > >> URM
> > >> > > > rather
> > >> > > > > > than
> > >> > > > > > > > > >> choosing
> > >> > > > > > > > > >> > > > >> > randomly.
> > >> > > > > > > > > >> > > > >> > > >>>> But this
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> should be spelled out in
> the
> > >> KIP.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> > DescribeTopicRequest
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >    1.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >    Yes, the plan is to
> > >> replace
> > >> > the
> > >> > > > > > > > > >> MetadataRequest
> > >> > > > > > > > > >> > > with
> > >> > > > > > > > > >> > > > >> the
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest
> for
> > >> the
> > >> > > > admin
> > >> > > > > > > > clients.
> > >> > > > > > > > > >> Will
> > >> > > > > > > > > >> > > > check
> > >> > > > > > > > > >> > > > >> > the
> > >> > > > > > > > > >> > > > >> > > >>>> details.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> Sounds good. But as I said,
> > you
> > >> > need
> > >> > > > to
> > >> > > > > > > > specify
> > >> > > > > > > > > >> how
> > >> > > > > > > > > >> > > > >> > AdminClient
> > >> > > > > > > > > >> > > > >> > > >>>> >> interacts
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> with the new request. This
> > will
> > >> > > > involve
> > >> > > > > > > adding
> > >> > > > > > > > > >> some
> > >> > > > > > > > > >> > > > fields
> > >> > > > > > > > > >> > > > >> to
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> TopicDescription.java. And
> > you
> > >> > need
> > >> > > to
> > >> > > > > > > specify
> > >> > > > > > > > > the
> > >> > > > > > > > > >> > > > changes
> > >> > > > > > > > > >> > > > >> to
> > >> > > > > > > > > >> > > > >> > > the
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> kafka-topics.sh command
> line
> > >> tool.
> > >> > > > > > Otherwise
> > >> > > > > > > > we
> > >> > > > > > > > > >> > cannot
> > >> > > > > > > > > >> > > > use
> > >> > > > > > > > > >> > > > >> the
> > >> > > > > > > > > >> > > > >> > > >>>> tool to
> > >> > > > > > > > > >> > > > >> > > >>>> >> see
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> the new information.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> The new requests,
> > >> > > DescribeTopicRequest
> > >> > > > > and
> > >> > > > > > > > > >> > > > >> > > >>>> GetReplicaLogInfoRequest,
> > >> > > > > > > > > >> > > > >> > > >>>> >> need
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> to have limits placed on
> them
> > >> so
> > >> > > that
> > >> > > > > > their
> > >> > > > > > > > size
> > >> > > > > > > > > >> > can't
> > >> > > > > > > > > >> > > be
> > >> > > > > > > > > >> > > > >> > > >>>> infinite. We
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> don't want to propagate the
> > >> > current
> > >> > > > > > problems
> > >> > > > > > > > of
> > >> > > > > > > > > >> > > > >> > MetadataRequest,
> > >> > > > > > > > > >> > > > >> > > >>>> where
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> clients can request massive
> > >> > > responses
> > >> > > > > that
> > >> > > > > > > can
> > >> > > > > > > > > >> mess
> > >> > > > > > > > > >> > up
> > >> > > > > > > > > >> > > > the
> > >> > > > > > > > > >> > > > >> JVM
> > >> > > > > > > > > >> > > > >> > > when
> > >> > > > > > > > > >> > > > >> > > >>>> >> handled.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> Adding limits is simple for
> > >> > > > > > > > > >> GetReplicaLogInfoRequest
> > >> > > > > > > > > >> > --
> > >> > > > > > > > > >> > > > we
> > >> > > > > > > > > >> > > > >> can
> > >> > > > > > > > > >> > > > >> > > >>>> just say
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> that only 2000 partitions
> at
> > a
> > >> > time
> > >> > > > can
> > >> > > > > be
> > >> > > > > > > > > >> requested.
> > >> > > > > > > > > >> > > For
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> DescribeTopicRequest we can
> > >> > probably
> > >> > > > > just
> > >> > > > > > > > limit
> > >> > > > > > > > > >> to 20
> > >> > > > > > > > > >> > > > topics
> > >> > > > > > > > > >> > > > >> > or
> > >> > > > > > > > > >> > > > >> > > >>>> >> something
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> like that, to avoid the
> > >> complexity
> > >> > > of
> > >> > > > > > doing
> > >> > > > > > > > > >> > pagination
> > >> > > > > > > > > >> > > in
> > >> > > > > > > > > >> > > > >> this
> > >> > > > > > > > > >> > > > >> > > KIP.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >    2.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >    I can let the broker
> > load
> > >> the
> > >> > > ELR
> > >> > > > > > info
> > >> > > > > > > so
> > >> > > > > > > > > >> that
> > >> > > > > > > > > >> > > they
> > >> > > > > > > > > >> > > > can
> > >> > > > > > > > > >> > > > >> > > serve
> > >> > > > > > > > > >> > > > >> > > >>>> the
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest
> as
> > >> well.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> Yes, it's fine to add to
> > >> > > > MetadataCache.
> > >> > > > > In
> > >> > > > > > > > fact,
> > >> > > > > > > > > >> > you'll
> > >> > > > > > > > > >> > > > be
> > >> > > > > > > > > >> > > > >> > > loading
> > >> > > > > > > > > >> > > > >> > > >>>> it
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> anyway once it's added to
> > >> > > > > PartitionImage.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >    3.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >    Yeah, it does not make
> > >> sense
> > >> > to
> > >> > > > > have
> > >> > > > > > > the
> > >> > > > > > > > > >> topic
> > >> > > > > > > > > >> > id
> > >> > > > > > > > > >> > > if
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest
> is
> > >> only
> > >> > > used
> > >> > > > > by
> > >> > > > > > > the
> > >> > > > > > > > > >> admin
> > >> > > > > > > > > >> > > > client.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> OK. That makes things
> > simpler.
> > >> We
> > >> > > can
> > >> > > > > > always
> > >> > > > > > > > > >> create a
> > >> > > > > > > > > >> > > new
> > >> > > > > > > > > >> > > > >> API
> > >> > > > > > > > > >> > > > >> > > later
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> (hopefully not in this
> KIP!)
> > to
> > >> > > query
> > >> > > > by
> > >> > > > > > > topic
> > >> > > > > > > > > ID.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Metrics
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> > As for overall cluster
> > health
> > >> > > > > metrics, I
> > >> > > > > > > > think
> > >> > > > > > > > > >> > > > >> under-min-ISR
> > >> > > > > > > > > >> > > > >> > > is
> > >> > > > > > > > > >> > > > >> > > >>>> still
> > >> > > > > > > > > >> > > > >> > > >>>> >> a
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> > useful one. ELR is more
> > like
> > >> a
> > >> > > > safety
> > >> > > > > > > belt.
> > >> > > > > > > > > When
> > >> > > > > > > > > >> > the
> > >> > > > > > > > > >> > > > ELR
> > >> > > > > > > > > >> > > > >> is
> > >> > > > > > > > > >> > > > >> > > >>>> used, the
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> > cluster availability has
> > >> already
> > >> > > > been
> > >> > > > > > > > > impacted.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Maybe we can have a
> metric
> > to
> > >> > > count
> > >> > > > > the
> > >> > > > > > > > > >> partitions
> > >> > > > > > > > > >> > > that
> > >> > > > > > > > > >> > > > >> > > sum(ISR,
> > >> > > > > > > > > >> > > > >> > > >>>> ELR)
> > >> > > > > > > > > >> > > > >> > > >>>> >> <
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> min
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> > ISR. What do you think?
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> How about:
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> A.  a metric for the totoal
> > >> number
> > >> > > of
> > >> > > > > > > > > >> under-min-isr
> > >> > > > > > > > > >> > > > >> > partitions?
> > >> > > > > > > > > >> > > > >> > > We
> > >> > > > > > > > > >> > > > >> > > >>>> don't
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> have that in Apache Kafka
> at
> > >> the
> > >> > > > moment.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> B. a metric for the number
> of
> > >> > > unclean
> > >> > > > > > leader
> > >> > > > > > > > > >> > elections
> > >> > > > > > > > > >> > > we
> > >> > > > > > > > > >> > > > >> did
> > >> > > > > > > > > >> > > > >> > > (for
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> simplicity, it can reset
> to 0
> > >> on
> > >> > > > > > controller
> > >> > > > > > > > > >> restart:
> > >> > > > > > > > > >> > we
> > >> > > > > > > > > >> > > > >> expect
> > >> > > > > > > > > >> > > > >> > > >>>> people to
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> monitor the change over
> time
> > >> > anyway)
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> best,
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> Colin
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> > Yeah, for the ongoing
> > unclean
> > >> > > > > > recoveries,
> > >> > > > > > > > the
> > >> > > > > > > > > >> > > > controller
> > >> > > > > > > > > >> > > > >> can
> > >> > > > > > > > > >> > > > >> > > >>>> keep an
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> > accurate count through
> > >> failover
> > >> > > > > because
> > >> > > > > > > > > >> partition
> > >> > > > > > > > > >> > > > >> > registration
> > >> > > > > > > > > >> > > > >> > > >>>> can
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> indicate
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> > whether a recovery is
> > needed.
> > >> > > > However,
> > >> > > > > > for
> > >> > > > > > > > the
> > >> > > > > > > > > >> > > happened
> > >> > > > > > > > > >> > > > >> > ones,
> > >> > > > > > > > > >> > > > >> > > >>>> unless
> > >> > > > > > > > > >> > > > >> > > >>>> >> we
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> > want to persist the
> number
> > >> > > > somewhere,
> > >> > > > > we
> > >> > > > > > > can
> > >> > > > > > > > > >> only
> > >> > > > > > > > > >> > > > figure
> > >> > > > > > > > > >> > > > >> it
> > >> > > > > > > > > >> > > > >> > > out
> > >> > > > > > > > > >> > > > >> > > >>>> from
> > >> > > > > > > > > >> > > > >> > > >>>> >> the
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> > log.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> > On Tue, Sep 12, 2023 at
> > >> 3:16 PM
> > >> > > > Colin
> > >> > > > > > > > McCabe <
> > >> > > > > > > > > >> > > > >> > > cmccabe@apache.org
> > >> > > > > > > > > >> > > > >> > > >>>> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> wrote:
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> Also, we should have
> > metrics
> > >> > that
> > >> > > > > show
> > >> > > > > > > what
> > >> > > > > > > > > is
> > >> > > > > > > > > >> > going
> > >> > > > > > > > > >> > > > on
> > >> > > > > > > > > >> > > > >> > with
> > >> > > > > > > > > >> > > > >> > > >>>> regard
> > >> > > > > > > > > >> > > > >> > > >>>> >> to
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> the
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> eligible replica set.
> I'm
> > >> not
> > >> > > sure
> > >> > > > > > > exactly
> > >> > > > > > > > > >> what to
> > >> > > > > > > > > >> > > > >> suggest,
> > >> > > > > > > > > >> > > > >> > > but
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> something
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> that could identify when
> > >> things
> > >> > > are
> > >> > > > > > going
> > >> > > > > > > > > >> wrong in
> > >> > > > > > > > > >> > > the
> > >> > > > > > > > > >> > > > >> > > clsuter.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> For example, maybe a
> > metric
> > >> for
> > >> > > > > > > partitions
> > >> > > > > > > > > >> > > containing
> > >> > > > > > > > > >> > > > >> > > replicas
> > >> > > > > > > > > >> > > > >> > > >>>> that
> > >> > > > > > > > > >> > > > >> > > >>>> >> are
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> ineligible to be leader?
> > >> That
> > >> > > would
> > >> > > > > > show
> > >> > > > > > > a
> > >> > > > > > > > > >> spike
> > >> > > > > > > > > >> > > when
> > >> > > > > > > > > >> > > > a
> > >> > > > > > > > > >> > > > >> > > broker
> > >> > > > > > > > > >> > > > >> > > >>>> had an
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> unclean restart.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> Ideally, we'd also have
> a
> > >> > metric
> > >> > > > that
> > >> > > > > > > > > indicates
> > >> > > > > > > > > >> > when
> > >> > > > > > > > > >> > > > an
> > >> > > > > > > > > >> > > > >> > > unclear
> > >> > > > > > > > > >> > > > >> > > >>>> >> leader
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> election or a recovery
> > >> > happened.
> > >> > > > > It's a
> > >> > > > > > > bit
> > >> > > > > > > > > >> tricky
> > >> > > > > > > > > >> > > > >> because
> > >> > > > > > > > > >> > > > >> > > the
> > >> > > > > > > > > >> > > > >> > > >>>> simple
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> thing, of tracking it
> per
> > >> > > > controller,
> > >> > > > > > may
> > >> > > > > > > > be
> > >> > > > > > > > > a
> > >> > > > > > > > > >> bit
> > >> > > > > > > > > >> > > > >> > confusing
> > >> > > > > > > > > >> > > > >> > > >>>> during
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> failovers.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> best,
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> Colin
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >>
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> On Tue, Sep 12, 2023, at
> > >> 14:25,
> > >> > > > Colin
> > >> > > > > > > > McCabe
> > >> > > > > > > > > >> > wrote:
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > Hi Calvin,
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > Thanks for the KIP. I
> > >> think
> > >> > > this
> > >> > > > > is a
> > >> > > > > > > > great
> > >> > > > > > > > > >> > > > >> improvement.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >> Additional High
> > Watermark
> > >> > > > advance
> > >> > > > > > > > > >> requirement
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > Typo: change "advance"
> > to
> > >> > > > > > "advancement"
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >> A bit recap of some
> key
> > >> > > > concepts.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > Typo: change "bit" to
> > >> "quick"
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >> Ack=1/all produce
> > >> request.
> > >> > It
> > >> > > > > > defines
> > >> > > > > > > > when
> > >> > > > > > > > > >> the
> > >> > > > > > > > > >> > > > Kafka
> > >> > > > > > > > > >> > > > >> > > server
> > >> > > > > > > > > >> > > > >> > > >>>> should
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> respond to the produce
> > >> request
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > I think this section
> > >> would be
> > >> > > > > clearer
> > >> > > > > > > if
> > >> > > > > > > > we
> > >> > > > > > > > > >> > talked
> > >> > > > > > > > > >> > > > >> about
> > >> > > > > > > > > >> > > > >> > > the
> > >> > > > > > > > > >> > > > >> > > >>>> new
> > >> > > > > > > > > >> > > > >> > > >>>> >> high
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > watermark advancement
> > >> > > requirement
> > >> > > > > > > first,
> > >> > > > > > > > > and
> > >> > > > > > > > > >> > THEN
> > >> > > > > > > > > >> > > > >> talked
> > >> > > > > > > > > >> > > > >> > > >>>> about its
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > impact on acks=0,
> > acks=1,
> > >> and
> > >> > > > > > > >  acks=all.
> > >> > > > > > > > > >> > > > acks=all
> > >> > > > > > > > > >> > > > >> is
> > >> > > > > > > > > >> > > > >> > of
> > >> > > > > > > > > >> > > > >> > > >>>> course
> > >> > > > > > > > > >> > > > >> > > >>>> >> the
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > main case we care
> about
> > >> here,
> > >> > > so
> > >> > > > it
> > >> > > > > > > would
> > >> > > > > > > > > be
> > >> > > > > > > > > >> > good
> > >> > > > > > > > > >> > > to
> > >> > > > > > > > > >> > > > >> lead
> > >> > > > > > > > > >> > > > >> > > with
> > >> > > > > > > > > >> > > > >> > > >>>> >> that,
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > rather than delving
> into
> > >> the
> > >> > > > > > > > technicalities
> > >> > > > > > > > > >> of
> > >> > > > > > > > > >> > > > acks=0/1
> > >> > > > > > > > > >> > > > >> > > first.
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >> Unclean recovery
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > So, here you are
> > >> introducing
> > >> > a
> > >> > > > new
> > >> > > > > > > > > >> > configuration,
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > unclean.recovery.strategy.
> > >> > The
> > >> > > > > > > difficult
> > >> > > > > > > > > >> thing
> > >> > > > > > > > > >> > > here
> > >> > > > > > > > > >> > > > is
> > >> > > > > > > > > >> > > > >> > that
> > >> > > > > > > > > >> > > > >> > > >>>> there
> > >> > > > > > > > > >> > > > >> > > >>>> >> is a
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > lot of overlap with
> > >> > > > > > > > > >> > > unclean.leader.election.enable.
> > >> > > > > > > > > >> > > > So
> > >> > > > > > > > > >> > > > >> we
> > >> > > > > > > > > >> > > > >> > > >>>> have 3
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > different settings for
> > >> > > > > > > > > >> > unclean.recovery.strategy,
> > >> > > > > > > > > >> > > > plus
> > >> > > > > > > > > >> > > > >> 2
> > >> > > > > > > > > >> > > > >> > > >>>> different
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > settings for
> > >> > > > > > > > > unclean.leader.election.enable,
> > >> > > > > > > > > >> > > giving
> > >> > > > > > > > > >> > > > a
> > >> > > > > > > > > >> > > > >> > cross
> > >> > > > > > > > > >> > > > >> > > >>>> >> product of
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > 6 different options.
> The
> > >> > > > following
> > >> > > > > > > > "unclean
> > >> > > > > > > > > >> > > recovery
> > >> > > > > > > > > >> > > > >> > > manager"
> > >> > > > > > > > > >> > > > >> > > >>>> >> section
> > >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > on
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> > >
> >
>


-- 
David Arthur

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Calvin Liu <ca...@confluent.io.INVALID>.
Hi Jun
-Good catch, yes, we don't need the -1 in the DescribeTopicRequest.
-No new value is added. The LeaderRecoveryState will still be set to 1 if
we have an unclean leader election. The unclean leader election includes
the old random way and the unclean recovery. During the unclean recovery,
the LeaderRecoveryState will not change until the controller decides to
update the records with the new leader.
Thanks

On Wed, Oct 11, 2023 at 9:02 AM Jun Rao <ju...@confluent.io.invalid> wrote:

> Hi, Calvin,
>
> Another thing. Currently, when there is an unclean leader election, we set
> the LeaderRecoveryState in PartitionRecord and PartitionChangeRecord to 1.
> With the KIP, will there be new values for LeaderRecoveryState? If not,
> when will LeaderRecoveryState be set to 1?
>
> Thanks,
>
> Jun
>
> On Tue, Oct 10, 2023 at 4:24 PM Jun Rao <ju...@confluent.io> wrote:
>
> > Hi, Calvin,
> >
> > One more comment.
> >
> > "The first partition to fetch details for. -1 means to fetch all
> > partitions." It seems that FirstPartitionId of 0 naturally means fetching
> > all partitions?
> >
> > Thanks,
> >
> > Jun
> >
> > On Tue, Oct 10, 2023 at 12:40 PM Calvin Liu <ca...@confluent.io.invalid>
> > wrote:
> >
> >> Hi Jun,
> >> Yeah, with the current Metadata request handling, we only return errors
> on
> >> the Topic level, like topic not found. It seems that querying a specific
> >> partition is not a valid use case. Will update.
> >> Thanks
> >>
> >> On Tue, Oct 10, 2023 at 11:55 AM Jun Rao <ju...@confluent.io.invalid>
> >> wrote:
> >>
> >> > Hi, Calvin,
> >> >
> >> > 60.  If the range query has errors for some of the partitions, do we
> >> expect
> >> > different responses when querying particular partitions?
> >> >
> >> > Thanks,
> >> >
> >> > Jun
> >> >
> >> > On Tue, Oct 10, 2023 at 10:50 AM Calvin Liu
> <caliu@confluent.io.invalid
> >> >
> >> > wrote:
> >> >
> >> > > Hi Jun
> >> > > 60. Yes, it is a good question. I was thinking the API could be
> >> flexible
> >> > to
> >> > > query the particular partitions if the range query has errors for
> >> some of
> >> > > the partitions. Not sure whether it is a valid assumption, what do
> you
> >> > > think?
> >> > >
> >> > > 61. Good point, I will update them to partition level with the same
> >> > limit.
> >> > >
> >> > > 62. Sure, will do.
> >> > >
> >> > > Thanks
> >> > >
> >> > > On Tue, Oct 10, 2023 at 10:12 AM Jun Rao <ju...@confluent.io.invalid>
> >> > wrote:
> >> > >
> >> > > > Hi, Calvin,
> >> > > >
> >> > > > A few more minor comments on your latest update.
> >> > > >
> >> > > > 60. DescribeTopicRequest: When will the Partitions field be used?
> It
> >> > > seems
> >> > > > that the FirstPartitionId field is enough for AdminClient usage.
> >> > > >
> >> > > > 61. Could we make the limit for DescribeTopicRequest,
> >> > > ElectLeadersRequest,
> >> > > > GetReplicaLogInfo consistent? Currently, ElectLeadersRequest's
> >> limit is
> >> > > at
> >> > > > topic level and GetReplicaLogInfo has a different partition level
> >> limit
> >> > > > from DescribeTopicRequest.
> >> > > >
> >> > > > 62. Should ElectLeadersRequest.DesiredLeaders be at the same level
> >> as
> >> > > > ElectLeadersRequest.TopicPartitions.Partitions? In the KIP, it
> looks
> >> > like
> >> > > > it's at the same level as ElectLeadersRequest.TopicPartitions.
> >> > > >
> >> > > > Thanks,
> >> > > >
> >> > > > Jun
> >> > > >
> >> > > > On Wed, Oct 4, 2023 at 3:55 PM Calvin Liu
> >> <ca...@confluent.io.invalid>
> >> > > > wrote:
> >> > > >
> >> > > > > Hi David,
> >> > > > > Thanks for the comments.
> >> > > > > ----
> >> > > > > I thought that a new snapshot with the downgraded MV is created
> in
> >> > this
> >> > > > > case. Isn’t it the case?
> >> > > > > Yes, you are right, a metadata delta will be generated after the
> >> MV
> >> > > > > downgrade. Then the user can start the software downgrade.
> >> > > > > -----
> >> > > > > Could you also elaborate a bit more on the reasoning behind
> adding
> >> > the
> >> > > > > limits to the admin RPCs? This is a new pattern in Kafka so it
> >> would
> >> > be
> >> > > > > good to clear on the motivation.
> >> > > > > Thanks to Colin for bringing it up. The current MetadataRequest
> >> does
> >> > > not
> >> > > > > have a limit on the number of topics to query in a single
> request.
> >> > > > Massive
> >> > > > > requests can mess up the JVM. We want to have some sort of
> >> throttle
> >> > on
> >> > > > the
> >> > > > > new APIs.
> >> > > > > -----
> >> > > > > Could you also explain how the client is supposed to handle the
> >> > > > > topics/partitions above the limit? I suppose that it will have
> to
> >> > retry
> >> > > > > those, correct?
> >> > > > > Corrent. For the official admin clients, it will split the large
> >> > > request
> >> > > > > into proper pieces and query one after another.
> >> > > > > -----
> >> > > > > My understanding is that the topics/partitions above the limit
> >> will
> >> > be
> >> > > > > failed with an invalid exception error. I wonder if this choice
> is
> >> > > > > judicious because the invalide request exception is usually
> >> fatal. It
> >> > > may
> >> > > > > be better to use an new and explicit error for this case.
> >> > > > >
> >> > > > > Thanks for bringing this up. How about "REQUEST_LIMIT_REACHED"?
> >> > > > > --------
> >> > > > > It seems that we still need to specify the changes to the admin
> >> api
> >> > to
> >> > > > > accommodate the new or updated apis. Do you plan to add them?
> >> > > > > Try to cover the following
> >> > > > > 1. The admin client will use the new DescribeTopicRequest to
> query
> >> > the
> >> > > > > topics
> >> > > > > 2. Mention the API limit and the new retriable error.
> >> > > > > 3. Output changes for the admin client when describing a topic
> >> (new
> >> > > > fields
> >> > > > > of ELR...)
> >> > > > > 4. Changes to data structures like TopicPartitionInfo to include
> >> the
> >> > > ELR.
> >> > > > > Anything else I missed?
> >> > > > >
> >> > > > > Thanks!
> >> > > > >
> >> > > > >
> >> > > > >
> >> > > > >
> >> > > > >
> >> > > > > On Wed, Oct 4, 2023 at 12:27 PM David Jacot <
> >> david.jacot@gmail.com>
> >> > > > wrote:
> >> > > > >
> >> > > > > > Hi Calvin,
> >> > > > > >
> >> > > > > > I thought that a new snapshot with the downgraded MV is
> created
> >> in
> >> > > this
> >> > > > > > case. Isn’t it the case?
> >> > > > > >
> >> > > > > > Could you also elaborate a bit more on the reasoning behind
> >> adding
> >> > > the
> >> > > > > > limits to the admin RPCs? This is a new pattern in Kafka so it
> >> > would
> >> > > be
> >> > > > > > good to clear on the motivation.
> >> > > > > >
> >> > > > > > Could you also explain how the client is supposed to handle
> the
> >> > > > > > topics/partitions above the limit? I suppose that it will have
> >> to
> >> > > retry
> >> > > > > > those, correct?
> >> > > > > >
> >> > > > > > My understanding is that the topics/partitions above the limit
> >> will
> >> > > be
> >> > > > > > failed with an invalid exception error. I wonder if this
> choice
> >> is
> >> > > > > > judicious because the invalide request exception is usually
> >> fatal.
> >> > It
> >> > > > may
> >> > > > > > be better to use an new and explicit error for this case.
> >> > > > > >
> >> > > > > > It seems that we still need to specify the changes to the
> admin
> >> api
> >> > > to
> >> > > > > > accommodate the new or updated apis. Do you plan to add them?
> >> > > > > >
> >> > > > > > Best,
> >> > > > > > David
> >> > > > > >
> >> > > > > > Le mer. 4 oct. 2023 à 20:39, Calvin Liu
> >> <caliu@confluent.io.invalid
> >> > >
> >> > > a
> >> > > > > > écrit :
> >> > > > > >
> >> > > > > > > Hi Jun,
> >> > > > > > > After the MV downgrade, the controller will write in the old
> >> > > version
> >> > > > of
> >> > > > > > the
> >> > > > > > > PartitionRecord/PartitionChangeRecord. If I understand
> >> correctly,
> >> > > it
> >> > > > is
> >> > > > > > > possible to downgrade the software version if the controller
> >> only
> >> > > has
> >> > > > > to
> >> > > > > > > handle old version records.
> >> > > > > > > However, the controller will not automatically rewrite the
> >> > > > > > PartitionRecord
> >> > > > > > > with the old version unless there is a partition update.
> Then,
> >> > the
> >> > > > user
> >> > > > > > may
> >> > > > > > > have to wait an unknown amount of time before the software
> >> > > downgrades
> >> > > > > > > unless they do a roll to force update every partition. If it
> >> > makes
> >> > > > > > sense, I
> >> > > > > > > can mention these steps to do a software downgrade.
> >> > > > > > > Thanks
> >> > > > > > >
> >> > > > > > > On Wed, Oct 4, 2023 at 11:20 AM Jun Rao
> >> <jun@confluent.io.invalid
> >> > >
> >> > > > > > wrote:
> >> > > > > > >
> >> > > > > > > > Hi, Calvin and Justine,
> >> > > > > > > >
> >> > > > > > > > Historically, when we change the record format in the log,
> >> we
> >> > > don't
> >> > > > > > > support
> >> > > > > > > > software version downgrading.
> >> > > > > > > >
> >> > > > > > > > For the record format change in the metadata log, have we
> >> > thought
> >> > > > > about
> >> > > > > > > > forcing the write of the latest metadata records with the
> >> old
> >> > > > version
> >> > > > > > > > during MV downgrading? This will in theory allow the old
> >> > version
> >> > > of
> >> > > > > the
> >> > > > > > > > software to obtain the latest metadata.
> >> > > > > > > >
> >> > > > > > > > Thanks,
> >> > > > > > > >
> >> > > > > > > > Jun
> >> > > > > > > >
> >> > > > > > > > On Wed, Oct 4, 2023 at 9:53 AM Justine Olshan
> >> > > > > > > <jolshan@confluent.io.invalid
> >> > > > > > > > >
> >> > > > > > > > wrote:
> >> > > > > > > >
> >> > > > > > > > > Sorry -- not MV but software version.
> >> > > > > > > > >
> >> > > > > > > > > On Wed, Oct 4, 2023 at 9:51 AM Justine Olshan <
> >> > > > > jolshan@confluent.io>
> >> > > > > > > > > wrote:
> >> > > > > > > > >
> >> > > > > > > > > > Catching up with this discussion.
> >> > > > > > > > > >
> >> > > > > > > > > > I was just curious -- have we had other instances
> where
> >> > > > > downgrading
> >> > > > > > > MV
> >> > > > > > > > is
> >> > > > > > > > > > not supported? I think Kafka typically tries to
> support
> >> > > > > downgrades,
> >> > > > > > > > and I
> >> > > > > > > > > > couldn't think of other examples.
> >> > > > > > > > > >
> >> > > > > > > > > > Thanks,
> >> > > > > > > > > > Justine
> >> > > > > > > > > >
> >> > > > > > > > > > On Wed, Oct 4, 2023 at 9:40 AM Calvin Liu
> >> > > > > > <caliu@confluent.io.invalid
> >> > > > > > > >
> >> > > > > > > > > > wrote:
> >> > > > > > > > > >
> >> > > > > > > > > >> Hi Jun,
> >> > > > > > > > > >> 54. Marked the software downgrading is not supported.
> >> As
> >> > the
> >> > > > old
> >> > > > > > > > > >> controller
> >> > > > > > > > > >> will not understand the new PartitionRecord and
> >> > > > > > > PartitionChangeRecord.
> >> > > > > > > > > >> Thanks!
> >> > > > > > > > > >>
> >> > > > > > > > > >> On Wed, Oct 4, 2023 at 9:12 AM Jun Rao
> >> > > > <jun@confluent.io.invalid
> >> > > > > >
> >> > > > > > > > > wrote:
> >> > > > > > > > > >>
> >> > > > > > > > > >> > Hi, Calvin,
> >> > > > > > > > > >> >
> >> > > > > > > > > >> > Thanks for the reply. Just one more comment.
> >> > > > > > > > > >> >
> >> > > > > > > > > >> > 54. It seems that downgrading MV is supported. Is
> >> > > > downgrading
> >> > > > > > the
> >> > > > > > > > > >> software
> >> > > > > > > > > >> > version supported? It would be useful to document
> >> that.
> >> > > > > > > > > >> >
> >> > > > > > > > > >> > Thanks,
> >> > > > > > > > > >> >
> >> > > > > > > > > >> > Jun
> >> > > > > > > > > >> >
> >> > > > > > > > > >> > On Tue, Oct 3, 2023 at 4:55 PM Artem Livshits
> >> > > > > > > > > >> > <al...@confluent.io.invalid> wrote:
> >> > > > > > > > > >> >
> >> > > > > > > > > >> > > Hi Colin,
> >> > > > > > > > > >> > >
> >> > > > > > > > > >> > > I think in your example "do_unclean_recovery"
> would
> >> > need
> >> > > > to
> >> > > > > do
> >> > > > > > > > > >> different
> >> > > > > > > > > >> > > things depending on the strategy.
> >> > > > > > > > > >> > >
> >> > > > > > > > > >> > > do_unclean_recovery() {
> >> > > > > > > > > >> > >    if (unclean.recovery.manager.enabled) {
> >> > > > > > > > > >> > >     if (strategy == Aggressive)
> >> > > > > > > > > >> > >       use
> >> > UncleanRecoveryManager(waitLastKnownERL=false)
> >> > > > //
> >> > > > > > > just
> >> > > > > > > > > >> inspect
> >> > > > > > > > > >> > > logs from whoever is available
> >> > > > > > > > > >> > >     else
> >> > > > > > > > > >> > >       use
> >> > UncleanRecoveryManager(waitLastKnownERL=true)
> >> > > > //
> >> > > > > > > must
> >> > > > > > > > > wait
> >> > > > > > > > > >> > for
> >> > > > > > > > > >> > > at least last known ELR
> >> > > > > > > > > >> > >   } else {
> >> > > > > > > > > >> > >     if (strategy == Aggressive)
> >> > > > > > > > > >> > >       choose the last known leader if that is
> >> > available,
> >> > > > or
> >> > > > > a
> >> > > > > > > > random
> >> > > > > > > > > >> > leader
> >> > > > > > > > > >> > > if not)
> >> > > > > > > > > >> > >     else
> >> > > > > > > > > >> > >       wait for last known leader to get back
> >> > > > > > > > > >> > >   }
> >> > > > > > > > > >> > > }
> >> > > > > > > > > >> > >
> >> > > > > > > > > >> > > The idea is that the Aggressive strategy would
> >> kick in
> >> > > as
> >> > > > > soon
> >> > > > > > > as
> >> > > > > > > > we
> >> > > > > > > > > >> lost
> >> > > > > > > > > >> > > the leader and would pick a leader from whoever
> is
> >> > > > > available;
> >> > > > > > > but
> >> > > > > > > > > the
> >> > > > > > > > > >> > > Balanced will only kick in when ELR is empty and
> >> will
> >> > > wait
> >> > > > > for
> >> > > > > > > the
> >> > > > > > > > > >> > brokers
> >> > > > > > > > > >> > > that likely have most data to be available.
> >> > > > > > > > > >> > >
> >> > > > > > > > > >> > > On Tue, Oct 3, 2023 at 3:04 PM Colin McCabe <
> >> > > > > > cmccabe@apache.org
> >> > > > > > > >
> >> > > > > > > > > >> wrote:
> >> > > > > > > > > >> > >
> >> > > > > > > > > >> > > > On Tue, Oct 3, 2023, at 10:49, Jun Rao wrote:
> >> > > > > > > > > >> > > > > Hi, Calvin,
> >> > > > > > > > > >> > > > >
> >> > > > > > > > > >> > > > > Thanks for the update KIP. A few more
> comments.
> >> > > > > > > > > >> > > > >
> >> > > > > > > > > >> > > > > 41. Why would a user choose the option to
> >> select a
> >> > > > > random
> >> > > > > > > > > replica
> >> > > > > > > > > >> as
> >> > > > > > > > > >> > > the
> >> > > > > > > > > >> > > > > leader instead of using
> >> > > > > > unclean.recovery.strateg=Aggressive?
> >> > > > > > > > It
> >> > > > > > > > > >> seems
> >> > > > > > > > > >> > > > that
> >> > > > > > > > > >> > > > > the latter is strictly better? If that's not
> >> the
> >> > > case,
> >> > > > > > could
> >> > > > > > > > we
> >> > > > > > > > > >> fold
> >> > > > > > > > > >> > > this
> >> > > > > > > > > >> > > > > option under unclean.recovery.strategy
> instead
> >> of
> >> > > > > > > introducing
> >> > > > > > > > a
> >> > > > > > > > > >> > > separate
> >> > > > > > > > > >> > > > > config?
> >> > > > > > > > > >> > > >
> >> > > > > > > > > >> > > > Hi Jun,
> >> > > > > > > > > >> > > >
> >> > > > > > > > > >> > > > I thought the flow of control was:
> >> > > > > > > > > >> > > >
> >> > > > > > > > > >> > > > If there is no leader for the partition {
> >> > > > > > > > > >> > > >   If (there are unfenced ELR members) {
> >> > > > > > > > > >> > > >     choose_an_unfenced_ELR_member
> >> > > > > > > > > >> > > >   } else if (there are fenced ELR members AND
> >> > > > > > > > > strategy=Aggressive) {
> >> > > > > > > > > >> > > >     do_unclean_recovery
> >> > > > > > > > > >> > > >   } else if (there are no ELR members AND
> >> strategy
> >> > !=
> >> > > > > None)
> >> > > > > > {
> >> > > > > > > > > >> > > >     do_unclean_recovery
> >> > > > > > > > > >> > > >   } else {
> >> > > > > > > > > >> > > >     do nothing about the missing leader
> >> > > > > > > > > >> > > >   }
> >> > > > > > > > > >> > > > }
> >> > > > > > > > > >> > > >
> >> > > > > > > > > >> > > > do_unclean_recovery() {
> >> > > > > > > > > >> > > >    if (unclean.recovery.manager.enabled) {
> >> > > > > > > > > >> > > >     use UncleanRecoveryManager
> >> > > > > > > > > >> > > >   } else {
> >> > > > > > > > > >> > > >     choose the last known leader if that is
> >> > available,
> >> > > > or
> >> > > > > a
> >> > > > > > > > random
> >> > > > > > > > > >> > leader
> >> > > > > > > > > >> > > > if not)
> >> > > > > > > > > >> > > >   }
> >> > > > > > > > > >> > > > }
> >> > > > > > > > > >> > > >
> >> > > > > > > > > >> > > > However, I think this could be clarified,
> >> especially
> >> > > the
> >> > > > > > > > behavior
> >> > > > > > > > > >> when
> >> > > > > > > > > >> > > > unclean.recovery.manager.enabled=false.
> >> Inuitively
> >> > the
> >> > > > > goal
> >> > > > > > > for
> >> > > > > > > > > >> > > > unclean.recovery.manager.enabled=false is to be
> >> "the
> >> > > > same
> >> > > > > as
> >> > > > > > > > now,
> >> > > > > > > > > >> > mostly"
> >> > > > > > > > > >> > > > but it's very underspecified in the KIP, I
> agree.
> >> > > > > > > > > >> > > >
> >> > > > > > > > > >> > > > >
> >> > > > > > > > > >> > > > > 50. ElectLeadersRequest: "If more than 20
> >> topics
> >> > are
> >> > > > > > > included,
> >> > > > > > > > > >> only
> >> > > > > > > > > >> > the
> >> > > > > > > > > >> > > > > first 20 will be served. Others will be
> >> returned
> >> > > with
> >> > > > > > > > > >> > DesiredLeaders."
> >> > > > > > > > > >> > > > Hmm,
> >> > > > > > > > > >> > > > > not sure that I understand this.
> >> > > ElectLeadersResponse
> >> > > > > > > doesn't
> >> > > > > > > > > >> have a
> >> > > > > > > > > >> > > > > DesiredLeaders field.
> >> > > > > > > > > >> > > > >
> >> > > > > > > > > >> > > > > 51. GetReplicaLogInfo: "If more than 2000
> >> > partitions
> >> > > > are
> >> > > > > > > > > included,
> >> > > > > > > > > >> > only
> >> > > > > > > > > >> > > > the
> >> > > > > > > > > >> > > > > first 2000 will be served" Do we return an
> >> error
> >> > for
> >> > > > the
> >> > > > > > > > > remaining
> >> > > > > > > > > >> > > > > partitions? Actually, should we include an
> >> > errorCode
> >> > > > > field
> >> > > > > > > at
> >> > > > > > > > > the
> >> > > > > > > > > >> > > > partition
> >> > > > > > > > > >> > > > > level in GetReplicaLogInfoResponse to cover
> >> > > > non-existing
> >> > > > > > > > > >> partitions
> >> > > > > > > > > >> > and
> >> > > > > > > > > >> > > > no
> >> > > > > > > > > >> > > > > authorization, etc?
> >> > > > > > > > > >> > > > >
> >> > > > > > > > > >> > > > > 52. The entry should matches => The entry
> >> should
> >> > > match
> >> > > > > > > > > >> > > > >
> >> > > > > > > > > >> > > > > 53. ElectLeadersRequest.DesiredLeaders:
> Should
> >> it
> >> > be
> >> > > > > > > nullable
> >> > > > > > > > > >> since a
> >> > > > > > > > > >> > > > user
> >> > > > > > > > > >> > > > > may not specify DesiredLeaders?
> >> > > > > > > > > >> > > > >
> >> > > > > > > > > >> > > > > 54. Downgrade: Is that indeed possible? I
> >> thought
> >> > > > > earlier
> >> > > > > > > you
> >> > > > > > > > > said
> >> > > > > > > > > >> > that
> >> > > > > > > > > >> > > > > once the new version of the records are in
> the
> >> > > > metadata
> >> > > > > > log,
> >> > > > > > > > one
> >> > > > > > > > > >> > can't
> >> > > > > > > > > >> > > > > downgrade since the old broker doesn't know
> >> how to
> >> > > > parse
> >> > > > > > the
> >> > > > > > > > new
> >> > > > > > > > > >> > > version
> >> > > > > > > > > >> > > > of
> >> > > > > > > > > >> > > > > the metadata records?
> >> > > > > > > > > >> > > > >
> >> > > > > > > > > >> > > >
> >> > > > > > > > > >> > > > MetadataVersion downgrade is currently broken
> >> but we
> >> > > > have
> >> > > > > > > fixing
> >> > > > > > > > > it
> >> > > > > > > > > >> on
> >> > > > > > > > > >> > > our
> >> > > > > > > > > >> > > > plate for Kafka 3.7.
> >> > > > > > > > > >> > > >
> >> > > > > > > > > >> > > > The way downgrade works is that "new features"
> >> are
> >> > > > > dropped,
> >> > > > > > > > > leaving
> >> > > > > > > > > >> > only
> >> > > > > > > > > >> > > > the old ones.
> >> > > > > > > > > >> > > >
> >> > > > > > > > > >> > > > > 55. CleanShutdownFile: Should we add a
> version
> >> > field
> >> > > > for
> >> > > > > > > > future
> >> > > > > > > > > >> > > > extension?
> >> > > > > > > > > >> > > > >
> >> > > > > > > > > >> > > > > 56. Config changes are public facing. Could
> we
> >> > have
> >> > > a
> >> > > > > > > separate
> >> > > > > > > > > >> > section
> >> > > > > > > > > >> > > to
> >> > > > > > > > > >> > > > > document all the config changes?
> >> > > > > > > > > >> > > >
> >> > > > > > > > > >> > > > +1. A separate section for this would be good.
> >> > > > > > > > > >> > > >
> >> > > > > > > > > >> > > > best,
> >> > > > > > > > > >> > > > Colin
> >> > > > > > > > > >> > > >
> >> > > > > > > > > >> > > > >
> >> > > > > > > > > >> > > > > Thanks,
> >> > > > > > > > > >> > > > >
> >> > > > > > > > > >> > > > > Jun
> >> > > > > > > > > >> > > > >
> >> > > > > > > > > >> > > > > On Mon, Sep 25, 2023 at 4:29 PM Calvin Liu
> >> > > > > > > > > >> > <caliu@confluent.io.invalid
> >> > > > > > > > > >> > > >
> >> > > > > > > > > >> > > > > wrote:
> >> > > > > > > > > >> > > > >
> >> > > > > > > > > >> > > > >> Hi Jun
> >> > > > > > > > > >> > > > >> Thanks for the comments.
> >> > > > > > > > > >> > > > >>
> >> > > > > > > > > >> > > > >> 40. If we change to None, it is not
> guaranteed
> >> > for
> >> > > no
> >> > > > > > data
> >> > > > > > > > > loss.
> >> > > > > > > > > >> For
> >> > > > > > > > > >> > > > users
> >> > > > > > > > > >> > > > >> who are not able to validate the data with
> >> > external
> >> > > > > > > > resources,
> >> > > > > > > > > >> > manual
> >> > > > > > > > > >> > > > >> intervention does not give a better result
> >> but a
> >> > > loss
> >> > > > > of
> >> > > > > > > > > >> > availability.
> >> > > > > > > > > >> > > > So
> >> > > > > > > > > >> > > > >> practically speaking, the Balance mode would
> >> be a
> >> > > > > better
> >> > > > > > > > > default
> >> > > > > > > > > >> > > value.
> >> > > > > > > > > >> > > > >>
> >> > > > > > > > > >> > > > >> 41. No, it represents how we want to do the
> >> > unclean
> >> > > > > > leader
> >> > > > > > > > > >> election.
> >> > > > > > > > > >> > > If
> >> > > > > > > > > >> > > > it
> >> > > > > > > > > >> > > > >> is false, the unclean leader election will
> be
> >> the
> >> > > old
> >> > > > > > > random
> >> > > > > > > > > way.
> >> > > > > > > > > >> > > > >> Otherwise, the unclean recovery will be
> used.
> >> > > > > > > > > >> > > > >>
> >> > > > > > > > > >> > > > >> 42. Good catch. Updated.
> >> > > > > > > > > >> > > > >>
> >> > > > > > > > > >> > > > >> 43. Only the first 20 topics will be served.
> >> > Others
> >> > > > > will
> >> > > > > > be
> >> > > > > > > > > >> returned
> >> > > > > > > > > >> > > > with
> >> > > > > > > > > >> > > > >> InvalidRequestError
> >> > > > > > > > > >> > > > >>
> >> > > > > > > > > >> > > > >> 44. The order matters. The desired leader
> >> entries
> >> > > > match
> >> > > > > > > with
> >> > > > > > > > > the
> >> > > > > > > > > >> > topic
> >> > > > > > > > > >> > > > >> partition list by the index.
> >> > > > > > > > > >> > > > >>
> >> > > > > > > > > >> > > > >> 45. Thanks! Updated.
> >> > > > > > > > > >> > > > >>
> >> > > > > > > > > >> > > > >> 46. Good advice! Updated.
> >> > > > > > > > > >> > > > >>
> >> > > > > > > > > >> > > > >> 47.1, updated the comment. Basically it will
> >> > elect
> >> > > > the
> >> > > > > > > > replica
> >> > > > > > > > > in
> >> > > > > > > > > >> > the
> >> > > > > > > > > >> > > > >> desiredLeader field to be the leader
> >> > > > > > > > > >> > > > >>
> >> > > > > > > > > >> > > > >> 47.2 We can let the admin client do the
> >> > conversion.
> >> > > > > Using
> >> > > > > > > the
> >> > > > > > > > > >> > > > desiredLeader
> >> > > > > > > > > >> > > > >> field in the json format seems easier for
> >> users.
> >> > > > > > > > > >> > > > >>
> >> > > > > > > > > >> > > > >> 48. Once the MV version is downgraded, all
> the
> >> > ELR
> >> > > > > > related
> >> > > > > > > > > fields
> >> > > > > > > > > >> > will
> >> > > > > > > > > >> > > > be
> >> > > > > > > > > >> > > > >> removed on the next partition change. The
> >> > > controller
> >> > > > > will
> >> > > > > > > > also
> >> > > > > > > > > >> > ignore
> >> > > > > > > > > >> > > > the
> >> > > > > > > > > >> > > > >> ELR fields. Updated the KIP.
> >> > > > > > > > > >> > > > >>
> >> > > > > > > > > >> > > > >> 49. Yes, it would be deprecated/removed.
> >> > > > > > > > > >> > > > >>
> >> > > > > > > > > >> > > > >>
> >> > > > > > > > > >> > > > >> On Mon, Sep 25, 2023 at 3:49 PM Jun Rao
> >> > > > > > > > > <jun@confluent.io.invalid
> >> > > > > > > > > >> >
> >> > > > > > > > > >> > > > wrote:
> >> > > > > > > > > >> > > > >>
> >> > > > > > > > > >> > > > >> > Hi, Calvin,
> >> > > > > > > > > >> > > > >> >
> >> > > > > > > > > >> > > > >> > Thanks for the updated KIP. Made another
> >> pass.
> >> > A
> >> > > > few
> >> > > > > > more
> >> > > > > > > > > >> comments
> >> > > > > > > > > >> > > > below.
> >> > > > > > > > > >> > > > >> >
> >> > > > > > > > > >> > > > >> > 40. unclean.leader.election.enable.false
> ->
> >> > > > > > > > > >> > > > >> > unclean.recovery.strategy.Balanced: The
> >> > Balanced
> >> > > > mode
> >> > > > > > > could
> >> > > > > > > > > >> still
> >> > > > > > > > > >> > > > lead to
> >> > > > > > > > > >> > > > >> > data loss. So, I am wondering if
> >> > > > > > > > > >> > > unclean.leader.election.enable.false
> >> > > > > > > > > >> > > > >> > should map to None?
> >> > > > > > > > > >> > > > >> >
> >> > > > > > > > > >> > > > >> > 41. unclean.recovery.manager.enabled: I am
> >> not
> >> > > sure
> >> > > > > why
> >> > > > > > > we
> >> > > > > > > > > >> > introduce
> >> > > > > > > > > >> > > > this
> >> > > > > > > > > >> > > > >> > additional config. Is it the same as
> >> > > > > > > > > >> > unclean.recovery.strategy=None?
> >> > > > > > > > > >> > > > >> >
> >> > > > > > > > > >> > > > >> > 42.
> >> > > > DescribeTopicResponse.TopicAuthorizedOperations:
> >> > > > > > > Should
> >> > > > > > > > > >> this
> >> > > > > > > > > >> > be
> >> > > > > > > > > >> > > at
> >> > > > > > > > > >> > > > >> the
> >> > > > > > > > > >> > > > >> > topic level?
> >> > > > > > > > > >> > > > >> >
> >> > > > > > > > > >> > > > >> > 43. "Limit: 20 topics max per request":
> >> Could
> >> > we
> >> > > > > > describe
> >> > > > > > > > > what
> >> > > > > > > > > >> > > > happens if
> >> > > > > > > > > >> > > > >> > the request includes more than 20 topics?
> >> > > > > > > > > >> > > > >> >
> >> > > > > > > > > >> > > > >> > 44. ElectLeadersRequest.DesiredLeaders:
> >> Could
> >> > we
> >> > > > > > describe
> >> > > > > > > > > >> whether
> >> > > > > > > > > >> > > the
> >> > > > > > > > > >> > > > >> > ordering matters?
> >> > > > > > > > > >> > > > >> >
> >> > > > > > > > > >> > > > >> > 45. GetReplicaLogInfo.TopicPartitions:
> >> "about":
> >> > > > "The
> >> > > > > > > topic
> >> > > > > > > > > >> > > partitions
> >> > > > > > > > > >> > > > to
> >> > > > > > > > > >> > > > >> > elect leaders.": The description in
> "about"
> >> is
> >> > > > > > incorrect.
> >> > > > > > > > > >> > > > >> >
> >> > > > > > > > > >> > > > >> > 46. GetReplicaLogInfoResponse: Should we
> >> nest
> >> > > > > > partitions
> >> > > > > > > > > under
> >> > > > > > > > > >> > > > topicId to
> >> > > > > > > > > >> > > > >> > be consistent with other types of
> responses?
> >> > > > > > > > > >> > > > >> >
> >> > > > > > > > > >> > > > >> > 47. kafka-leader-election.sh:
> >> > > > > > > > > >> > > > >> > 47.1 Could we explain DESIGNATION?
> >> > > > > > > > > >> > > > >> > 47.2 desiredLeader: Should it be a list to
> >> > match
> >> > > > the
> >> > > > > > > field
> >> > > > > > > > in
> >> > > > > > > > > >> > > > >> > ElectLeadersRequest?
> >> > > > > > > > > >> > > > >> >
> >> > > > > > > > > >> > > > >> > 48. We could add a section on downgrade?
> >> > > > > > > > > >> > > > >> >
> >> > > > > > > > > >> > > > >> > 49. LastKnownLeader: This seems only
> needed
> >> in
> >> > > the
> >> > > > > > first
> >> > > > > > > > > phase
> >> > > > > > > > > >> of
> >> > > > > > > > > >> > > > >> > delivering ELR. Will it be removed when
> the
> >> > > > complete
> >> > > > > > KIP
> >> > > > > > > is
> >> > > > > > > > > >> > > delivered?
> >> > > > > > > > > >> > > > >> >
> >> > > > > > > > > >> > > > >> > Thanks,
> >> > > > > > > > > >> > > > >> >
> >> > > > > > > > > >> > > > >> > Jun
> >> > > > > > > > > >> > > > >> >
> >> > > > > > > > > >> > > > >> > On Tue, Sep 19, 2023 at 1:30 PM Colin
> >> McCabe <
> >> > > > > > > > > >> cmccabe@apache.org>
> >> > > > > > > > > >> > > > wrote:
> >> > > > > > > > > >> > > > >> >
> >> > > > > > > > > >> > > > >> > > Hi Calvin,
> >> > > > > > > > > >> > > > >> > >
> >> > > > > > > > > >> > > > >> > > Thanks for the explanations. I like the
> >> idea
> >> > of
> >> > > > > using
> >> > > > > > > > none,
> >> > > > > > > > > >> > > > balanced,
> >> > > > > > > > > >> > > > >> > > aggressive. We also had an offline
> >> discussion
> >> > > > about
> >> > > > > > why
> >> > > > > > > > it
> >> > > > > > > > > is
> >> > > > > > > > > >> > good
> >> > > > > > > > > >> > > > to
> >> > > > > > > > > >> > > > >> > use a
> >> > > > > > > > > >> > > > >> > > new config key (basically, so that we
> can
> >> > > > deprecate
> >> > > > > > the
> >> > > > > > > > old
> >> > > > > > > > > >> one
> >> > > > > > > > > >> > > > which
> >> > > > > > > > > >> > > > >> had
> >> > > > > > > > > >> > > > >> > > only false/true values in 4.0) With
> these
> >> > > > changes,
> >> > > > > I
> >> > > > > > am
> >> > > > > > > > +1.
> >> > > > > > > > > >> > > > >> > >
> >> > > > > > > > > >> > > > >> > > best,
> >> > > > > > > > > >> > > > >> > > Colin
> >> > > > > > > > > >> > > > >> > >
> >> > > > > > > > > >> > > > >> > > On Mon, Sep 18, 2023, at 15:54, Calvin
> Liu
> >> > > wrote:
> >> > > > > > > > > >> > > > >> > > > Hi Colin,
> >> > > > > > > > > >> > > > >> > > > Also, can we deprecate
> >> > > > > > unclean.leader.election.enable
> >> > > > > > > > in
> >> > > > > > > > > >> 4.0?
> >> > > > > > > > > >> > > > Before
> >> > > > > > > > > >> > > > >> > > that,
> >> > > > > > > > > >> > > > >> > > > we can have both the config
> >> > > > > > unclean.recovery.strategy
> >> > > > > > > > and
> >> > > > > > > > > >> > > > >> > > > unclean.leader.election.enable
> >> > > > > > > > > >> > > > >> > > > and using the unclean.recovery.Enabled
> >> to
> >> > > > > determine
> >> > > > > > > > which
> >> > > > > > > > > >> > config
> >> > > > > > > > > >> > > > to
> >> > > > > > > > > >> > > > >> use
> >> > > > > > > > > >> > > > >> > > > during the unclean leader election.
> >> > > > > > > > > >> > > > >> > > >
> >> > > > > > > > > >> > > > >> > > > On Mon, Sep 18, 2023 at 3:51 PM Calvin
> >> Liu
> >> > <
> >> > > > > > > > > >> > caliu@confluent.io>
> >> > > > > > > > > >> > > > >> wrote:
> >> > > > > > > > > >> > > > >> > > >
> >> > > > > > > > > >> > > > >> > > >> Hi Colin,
> >> > > > > > > > > >> > > > >> > > >> For the unclean.recovery.strategy
> >> config
> >> > > name,
> >> > > > > how
> >> > > > > > > > about
> >> > > > > > > > > >> we
> >> > > > > > > > > >> > use
> >> > > > > > > > > >> > > > the
> >> > > > > > > > > >> > > > >> > > >> following
> >> > > > > > > > > >> > > > >> > > >> None. It basically means no unclean
> >> > recovery
> >> > > > > will
> >> > > > > > be
> >> > > > > > > > > >> > performed.
> >> > > > > > > > > >> > > > >> > > >> Aggressive. It means availability
> goes
> >> > > first.
> >> > > > > > > Whenever
> >> > > > > > > > > the
> >> > > > > > > > > >> > > > partition
> >> > > > > > > > > >> > > > >> > > can't
> >> > > > > > > > > >> > > > >> > > >> elect a durable replica, the
> controller
> >> > will
> >> > > > try
> >> > > > > > the
> >> > > > > > > > > >> unclean
> >> > > > > > > > > >> > > > >> recovery.
> >> > > > > > > > > >> > > > >> > > >> Balanced. It is the balance point of
> >> the
> >> > > > > > > availability
> >> > > > > > > > > >> > > > >> > first(Aggressive)
> >> > > > > > > > > >> > > > >> > > >> and least availability(None). The
> >> > controller
> >> > > > > > > performs
> >> > > > > > > > > >> unclean
> >> > > > > > > > > >> > > > >> recovery
> >> > > > > > > > > >> > > > >> > > when
> >> > > > > > > > > >> > > > >> > > >> both ISR and ELR are empty.
> >> > > > > > > > > >> > > > >> > > >>
> >> > > > > > > > > >> > > > >> > > >>
> >> > > > > > > > > >> > > > >> > > >> On Fri, Sep 15, 2023 at 11:42 AM
> Calvin
> >> > Liu
> >> > > <
> >> > > > > > > > > >> > > caliu@confluent.io>
> >> > > > > > > > > >> > > > >> > wrote:
> >> > > > > > > > > >> > > > >> > > >>
> >> > > > > > > > > >> > > > >> > > >>> Hi Colin,
> >> > > > > > > > > >> > > > >> > > >>>
> >> > > > > > > > > >> > > > >> > > >>> > So, the proposal is that if
> someone
> >> > sets
> >> > > > > > > > > >> > > > >> > > "unclean.leader.election.enable
> >> > > > > > > > > >> > > > >> > > >>> = true"...
> >> > > > > > > > > >> > > > >> > > >>>
> >> > > > > > > > > >> > > > >> > > >>>
> >> > > > > > > > > >> > > > >> > > >>> The idea is to use one of the
> >> > > > > > > > > >> unclean.leader.election.enable
> >> > > > > > > > > >> > > and
> >> > > > > > > > > >> > > > >> > > >>> unclean.recovery.strategy based on
> the
> >> > > > > > > > > >> > > > unclean.recovery.Enabled. A
> >> > > > > > > > > >> > > > >> > > possible
> >> > > > > > > > > >> > > > >> > > >>> version can be
> >> > > > > > > > > >> > > > >> > > >>>
> >> > > > > > > > > >> > > > >> > > >>> If unclean.recovery.Enabled:
> >> > > > > > > > > >> > > > >> > > >>>
> >> > > > > > > > > >> > > > >> > > >>> {
> >> > > > > > > > > >> > > > >> > > >>>
> >> > > > > > > > > >> > > > >> > > >>> Check unclean.recovery.strategy. If
> >> set,
> >> > > use
> >> > > > > it.
> >> > > > > > > > > >> Otherwise,
> >> > > > > > > > > >> > > > check
> >> > > > > > > > > >> > > > >> > > >>> unclean.leader.election.enable and
> >> > > translate
> >> > > > it
> >> > > > > > to
> >> > > > > > > > > >> > > > >> > > >>> unclean.recovery.strategy.
> >> > > > > > > > > >> > > > >> > > >>>
> >> > > > > > > > > >> > > > >> > > >>> } else {
> >> > > > > > > > > >> > > > >> > > >>>
> >> > > > > > > > > >> > > > >> > > >>> Use unclean.leader.election.enable
> >> > > > > > > > > >> > > > >> > > >>>
> >> > > > > > > > > >> > > > >> > > >>> }
> >> > > > > > > > > >> > > > >> > > >>>
> >> > > > > > > > > >> > > > >> > > >>>
> >> > > > > > > > > >> > > > >> > > >>> —--------
> >> > > > > > > > > >> > > > >> > > >>>
> >> > > > > > > > > >> > > > >> > > >>> >The configuration key should be
> >> > > > > > > > > >> > > > >> "unclean.recovery.manager.enabled",
> >> > > > > > > > > >> > > > >> > > >>> right?
> >> > > > > > > > > >> > > > >> > > >>>
> >> > > > > > > > > >> > > > >> > > >>>
> >> > > > > > > > > >> > > > >> > > >>> I think we have two ways of
> choosing a
> >> > > leader
> >> > > > > > > > > uncleanly,
> >> > > > > > > > > >> > > unclean
> >> > > > > > > > > >> > > > >> > leader
> >> > > > > > > > > >> > > > >> > > >>> election and unclean recovery(log
> >> > > inspection)
> >> > > > > and
> >> > > > > > > we
> >> > > > > > > > > try
> >> > > > > > > > > >> to
> >> > > > > > > > > >> > > > switch
> >> > > > > > > > > >> > > > >> > > between
> >> > > > > > > > > >> > > > >> > > >>> them.
> >> > > > > > > > > >> > > > >> > > >>>
> >> > > > > > > > > >> > > > >> > > >>> Do you mean we want to develop two
> >> ways
> >> > of
> >> > > > > > > performing
> >> > > > > > > > > the
> >> > > > > > > > > >> > > > unclean
> >> > > > > > > > > >> > > > >> > > >>> recovery and one of them is using
> >> > “unclean
> >> > > > > > recovery
> >> > > > > > > > > >> > manager”?
> >> > > > > > > > > >> > > I
> >> > > > > > > > > >> > > > >> guess
> >> > > > > > > > > >> > > > >> > > we
> >> > > > > > > > > >> > > > >> > > >>> haven’t discussed the second way.
> >> > > > > > > > > >> > > > >> > > >>>
> >> > > > > > > > > >> > > > >> > > >>>
> >> > > > > > > > > >> > > > >> > > >>> —-------
> >> > > > > > > > > >> > > > >> > > >>>
> >> > > > > > > > > >> > > > >> > > >>> >How do these 4 levels of overrides
> >> > > interact
> >> > > > > with
> >> > > > > > > > your
> >> > > > > > > > > >> new
> >> > > > > > > > > >> > > > >> > > >>> configurations?
> >> > > > > > > > > >> > > > >> > > >>>
> >> > > > > > > > > >> > > > >> > > >>>
> >> > > > > > > > > >> > > > >> > > >>> I do notice in the Kraft controller
> >> code,
> >> > > the
> >> > > > > > > method
> >> > > > > > > > to
> >> > > > > > > > > >> > check
> >> > > > > > > > > >> > > > >> whether
> >> > > > > > > > > >> > > > >> > > >>> perform unclean leader election is
> >> hard
> >> > > coded
> >> > > > > to
> >> > > > > > > > false
> >> > > > > > > > > >> since
> >> > > > > > > > > >> > > > >> > > >>>
> >> > 2021(uncleanLeaderElectionEnabledForTopic).
> >> > > > > Isn’t
> >> > > > > > > it
> >> > > > > > > > a
> >> > > > > > > > > >> good
> >> > > > > > > > > >> > > > chance
> >> > > > > > > > > >> > > > >> to
> >> > > > > > > > > >> > > > >> > > >>> completely deprecate the
> >> > > > > > > > > unclean.leader.election.enable?
> >> > > > > > > > > >> We
> >> > > > > > > > > >> > > > don’t
> >> > > > > > > > > >> > > > >> > even
> >> > > > > > > > > >> > > > >> > > have
> >> > > > > > > > > >> > > > >> > > >>> to worry about the config
> conversion.
> >> > > > > > > > > >> > > > >> > > >>>
> >> > > > > > > > > >> > > > >> > > >>> On the other hand, whatever the
> >> override
> >> > > is,
> >> > > > as
> >> > > > > > > long
> >> > > > > > > > as
> >> > > > > > > > > >> the
> >> > > > > > > > > >> > > > >> > controller
> >> > > > > > > > > >> > > > >> > > >>> can have the final effective
> >> > > > > > > > > >> unclean.leader.election.enable,
> >> > > > > > > > > >> > > the
> >> > > > > > > > > >> > > > >> > topic
> >> > > > > > > > > >> > > > >> > > >>> level config
> >> unclean.recovery.strategy,
> >> > the
> >> > > > > > cluster
> >> > > > > > > > > level
> >> > > > > > > > > >> > > config
> >> > > > > > > > > >> > > > >> > > >>> unclean.recovery.Enabled, the
> >> controller
> >> > > can
> >> > > > > > > > calculate
> >> > > > > > > > > >> the
> >> > > > > > > > > >> > > > correct
> >> > > > > > > > > >> > > > >> > > methods
> >> > > > > > > > > >> > > > >> > > >>> to use right?
> >> > > > > > > > > >> > > > >> > > >>>
> >> > > > > > > > > >> > > > >> > > >>>
> >> > > > > > > > > >> > > > >> > > >>> On Fri, Sep 15, 2023 at 10:02 AM
> Colin
> >> > > > McCabe <
> >> > > > > > > > > >> > > > cmccabe@apache.org>
> >> > > > > > > > > >> > > > >> > > wrote:
> >> > > > > > > > > >> > > > >> > > >>>
> >> > > > > > > > > >> > > > >> > > >>>> On Thu, Sep 14, 2023, at 22:23,
> >> Calvin
> >> > Liu
> >> > > > > > wrote:
> >> > > > > > > > > >> > > > >> > > >>>> > Hi Colin
> >> > > > > > > > > >> > > > >> > > >>>> > 1. I think using the new config
> >> name
> >> > is
> >> > > > more
> >> > > > > > > > clear.
> >> > > > > > > > > >> > > > >> > > >>>> >        a. The unclean leader
> >> election
> >> > is
> >> > > > > > > actually
> >> > > > > > > > > >> removed
> >> > > > > > > > > >> > > if
> >> > > > > > > > > >> > > > >> > unclean
> >> > > > > > > > > >> > > > >> > > >>>> > recovery is in use.
> >> > > > > > > > > >> > > > >> > > >>>> >        b. Using multiple values
> in
> >> > > > > > > > > >> > > > >> unclean.leader.election.enable
> >> > > > > > > > > >> > > > >> > is
> >> > > > > > > > > >> > > > >> > > >>>> > confusing and it will be more
> >> > confusing
> >> > > > > after
> >> > > > > > > > people
> >> > > > > > > > > >> > forget
> >> > > > > > > > > >> > > > >> about
> >> > > > > > > > > >> > > > >> > > this
> >> > > > > > > > > >> > > > >> > > >>>> > discussion.
> >> > > > > > > > > >> > > > >> > > >>>>
> >> > > > > > > > > >> > > > >> > > >>>> Hi Calvin,
> >> > > > > > > > > >> > > > >> > > >>>>
> >> > > > > > > > > >> > > > >> > > >>>> So, the proposal is that if someone
> >> sets
> >> > > > > > > > > >> > > > >> > > "unclean.leader.election.enable
> >> > > > > > > > > >> > > > >> > > >>>> = true" but then sets one of your
> new
> >> > > > > > > > configurations,
> >> > > > > > > > > >> the
> >> > > > > > > > > >> > > > value of
> >> > > > > > > > > >> > > > >> > > >>>> unclean.leader.election.enable is
> >> > ignored?
> >> > > > > That
> >> > > > > > > > seems
> >> > > > > > > > > >> less
> >> > > > > > > > > >> > > > clear
> >> > > > > > > > > >> > > > >> to
> >> > > > > > > > > >> > > > >> > > me, not
> >> > > > > > > > > >> > > > >> > > >>>> more. Just in general, having
> >> multiple
> >> > > > > > > configuration
> >> > > > > > > > > >> keys
> >> > > > > > > > > >> > to
> >> > > > > > > > > >> > > > >> control
> >> > > > > > > > > >> > > > >> > > the
> >> > > > > > > > > >> > > > >> > > >>>> same thing confuses users.
> Basically,
> >> > they
> >> > > > are
> >> > > > > > > > sitting
> >> > > > > > > > > >> at a
> >> > > > > > > > > >> > > > giant
> >> > > > > > > > > >> > > > >> > > control
> >> > > > > > > > > >> > > > >> > > >>>> panel, and some of the levers do
> >> > nothing.
> >> > > > > > > > > >> > > > >> > > >>>>
> >> > > > > > > > > >> > > > >> > > >>>> > 2. Sorry I forgot to mention in
> the
> >> > > > response
> >> > > > > > > that
> >> > > > > > > > I
> >> > > > > > > > > >> did
> >> > > > > > > > > >> > add
> >> > > > > > > > > >> > > > the
> >> > > > > > > > > >> > > > >> > > >>>> > unclean.recovery.Enabled flag.
> >> > > > > > > > > >> > > > >> > > >>>>
> >> > > > > > > > > >> > > > >> > > >>>> The configuration key should be
> >> > > > > > > > > >> > > > >> "unclean.recovery.manager.enabled",
> >> > > > > > > > > >> > > > >> > > >>>> right? Becuase we can do "unclean
> >> > > recovery"
> >> > > > > > > without
> >> > > > > > > > > the
> >> > > > > > > > > >> > > > manager.
> >> > > > > > > > > >> > > > >> > > Disabling
> >> > > > > > > > > >> > > > >> > > >>>> the manager just means we use a
> >> > different
> >> > > > > > > mechanism
> >> > > > > > > > > for
> >> > > > > > > > > >> > > > recovery.
> >> > > > > > > > > >> > > > >> > > >>>>
> >> > > > > > > > > >> > > > >> > > >>>> >        c. Maybe I underestimated
> >> the
> >> > > > > challenge
> >> > > > > > > of
> >> > > > > > > > > >> > replacing
> >> > > > > > > > > >> > > > the
> >> > > > > > > > > >> > > > >> > > >>>> config. Any
> >> > > > > > > > > >> > > > >> > > >>>> > implementation problems ahead?
> >> > > > > > > > > >> > > > >> > > >>>>
> >> > > > > > > > > >> > > > >> > > >>>> There are four levels of overrides
> >> for
> >> > > > > > > > > >> > > > >> > unclean.leader.election.enable.
> >> > > > > > > > > >> > > > >> > > >>>>
> >> > > > > > > > > >> > > > >> > > >>>> 1. static configuration for node.
> >> > > > > > > > > >> > > > >> > > >>>>     This goes in the configuration
> >> file,
> >> > > > > > typically
> >> > > > > > > > > named
> >> > > > > > > > > >> > > > >> > > >>>> server.properties
> >> > > > > > > > > >> > > > >> > > >>>>
> >> > > > > > > > > >> > > > >> > > >>>> 2. dynamic configuration for node
> >> > default
> >> > > > > > > > > >> > > > >> > > >>>>   ConfigResource(type=BROKER,
> >> name="")
> >> > > > > > > > > >> > > > >> > > >>>>
> >> > > > > > > > > >> > > > >> > > >>>> 3. dynamic configuration for node
> >> > > > > > > > > >> > > > >> > > >>>>   ConfigResource(type=BROKER,
> >> > > > name=<controller
> >> > > > > > > id>)
> >> > > > > > > > > >> > > > >> > > >>>>
> >> > > > > > > > > >> > > > >> > > >>>> 4. dynamic configuration for topic
> >> > > > > > > > > >> > > > >> > > >>>>   ConfigResource(type=TOPIC,
> >> > > > > name=<topic-name>)
> >> > > > > > > > > >> > > > >> > > >>>>
> >> > > > > > > > > >> > > > >> > > >>>> How do these 4 levels of overrides
> >> > > interact
> >> > > > > with
> >> > > > > > > > your
> >> > > > > > > > > >> new
> >> > > > > > > > > >> > > > >> > > >>>> configurations? If the new
> >> > configurations
> >> > > > > > dominate
> >> > > > > > > > > over
> >> > > > > > > > > >> the
> >> > > > > > > > > >> > > old
> >> > > > > > > > > >> > > > >> > ones,
> >> > > > > > > > > >> > > > >> > > it
> >> > > > > > > > > >> > > > >> > > >>>> seems like this will get a lot more
> >> > > > confusing
> >> > > > > to
> >> > > > > > > > > >> implement
> >> > > > > > > > > >> > > (and
> >> > > > > > > > > >> > > > >> also
> >> > > > > > > > > >> > > > >> > > to
> >> > > > > > > > > >> > > > >> > > >>>> use.)
> >> > > > > > > > > >> > > > >> > > >>>>
> >> > > > > > > > > >> > > > >> > > >>>> Again, I'd recommend just adding
> some
> >> > new
> >> > > > > values
> >> > > > > > > to
> >> > > > > > > > > >> > > > >> > > >>>> unclean.leader.election.enable.
> It's
> >> > > simple
> >> > > > > and
> >> > > > > > > will
> >> > > > > > > > > >> > prevent
> >> > > > > > > > > >> > > > user
> >> > > > > > > > > >> > > > >> > > confusion
> >> > > > > > > > > >> > > > >> > > >>>> (as well as developer confusion.)
> >> > > > > > > > > >> > > > >> > > >>>>
> >> > > > > > > > > >> > > > >> > > >>>> best,
> >> > > > > > > > > >> > > > >> > > >>>> Colin
> >> > > > > > > > > >> > > > >> > > >>>>
> >> > > > > > > > > >> > > > >> > > >>>>
> >> > > > > > > > > >> > > > >> > > >>>> > 3. About the admin client, I
> >> > mentioned 3
> >> > > > > > changes
> >> > > > > > > > in
> >> > > > > > > > > >> the
> >> > > > > > > > > >> > > > client.
> >> > > > > > > > > >> > > > >> > > >>>> Anything
> >> > > > > > > > > >> > > > >> > > >>>> > else I missed in the KIP?
> >> > > > > > > > > >> > > > >> > > >>>> >       a. The client will switch
> to
> >> > using
> >> > > > the
> >> > > > > > new
> >> > > > > > > > RPC
> >> > > > > > > > > >> > > instead
> >> > > > > > > > > >> > > > of
> >> > > > > > > > > >> > > > >> > > >>>> > MetadataRequest for the topics.
> >> > > > > > > > > >> > > > >> > > >>>> >       b. The TopicPartitionInfo
> >> used
> >> > in
> >> > > > > > > > > >> TopicDescription
> >> > > > > > > > > >> > > > needs
> >> > > > > > > > > >> > > > >> to
> >> > > > > > > > > >> > > > >> > > add
> >> > > > > > > > > >> > > > >> > > >>>> new
> >> > > > > > > > > >> > > > >> > > >>>> > fields related to the ELR.
> >> > > > > > > > > >> > > > >> > > >>>> >       c. The outputs will add the
> >> ELR
> >> > > > > related
> >> > > > > > > > > fields.
> >> > > > > > > > > >> > > > >> > > >>>> >
> >> > > > > > > > > >> > > > >> > > >>>> > On Thu, Sep 14, 2023 at 9:19 PM
> >> Colin
> >> > > > > McCabe <
> >> > > > > > > > > >> > > > >> cmccabe@apache.org>
> >> > > > > > > > > >> > > > >> > > >>>> wrote:
> >> > > > > > > > > >> > > > >> > > >>>> >
> >> > > > > > > > > >> > > > >> > > >>>> >> Hi Calvin,
> >> > > > > > > > > >> > > > >> > > >>>> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> Thanks for the changes.
> >> > > > > > > > > >> > > > >> > > >>>> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> 1. Earlier I commented that
> >> creating
> >> > > > > > > > > >> > > > >> "unclean.recovery.strategy "
> >> > > > > > > > > >> > > > >> > > is
> >> > > > > > > > > >> > > > >> > > >>>> not
> >> > > > > > > > > >> > > > >> > > >>>> >> necessary, and we can just reuse
> >> the
> >> > > > > existing
> >> > > > > > > > > >> > > > >> > > >>>> >> "unclean.leader.election.enable"
> >> > > > > > configuration
> >> > > > > > > > key.
> >> > > > > > > > > >> > Let's
> >> > > > > > > > > >> > > > >> discuss
> >> > > > > > > > > >> > > > >> > > >>>> that.
> >> > > > > > > > > >> > > > >> > > >>>> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> 2.I also don't understand why
> you
> >> > > didn't
> >> > > > > add
> >> > > > > > a
> >> > > > > > > > > >> > > > configuration to
> >> > > > > > > > > >> > > > >> > > >>>> enable or
> >> > > > > > > > > >> > > > >> > > >>>> >> disable the Unclean Recovery
> >> Manager.
> >> > > > This
> >> > > > > > > seems
> >> > > > > > > > > >> like a
> >> > > > > > > > > >> > > very
> >> > > > > > > > > >> > > > >> > simple
> >> > > > > > > > > >> > > > >> > > >>>> way to
> >> > > > > > > > > >> > > > >> > > >>>> >> handle the staging issue which
> we
> >> > > > > discussed.
> >> > > > > > > The
> >> > > > > > > > > URM
> >> > > > > > > > > >> can
> >> > > > > > > > > >> > > > just
> >> > > > > > > > > >> > > > >> be
> >> > > > > > > > > >> > > > >> > > >>>> turned off
> >> > > > > > > > > >> > > > >> > > >>>> >> until it is production ready.
> >> Let's
> >> > > > discuss
> >> > > > > > > this.
> >> > > > > > > > > >> > > > >> > > >>>> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> 3. You still need to describe
> the
> >> > > changes
> >> > > > > to
> >> > > > > > > > > >> AdminClient
> >> > > > > > > > > >> > > > that
> >> > > > > > > > > >> > > > >> are
> >> > > > > > > > > >> > > > >> > > >>>> needed
> >> > > > > > > > > >> > > > >> > > >>>> >> to use DescribeTopicRequest.
> >> > > > > > > > > >> > > > >> > > >>>> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> Keep at it. It's looking better.
> >> :)
> >> > > > > > > > > >> > > > >> > > >>>> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> best,
> >> > > > > > > > > >> > > > >> > > >>>> >> Colin
> >> > > > > > > > > >> > > > >> > > >>>> >>
> >> > > > > > > > > >> > > > >> > > >>>> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> On Thu, Sep 14, 2023, at 11:03,
> >> > Calvin
> >> > > > Liu
> >> > > > > > > wrote:
> >> > > > > > > > > >> > > > >> > > >>>> >> > Hi Colin
> >> > > > > > > > > >> > > > >> > > >>>> >> > Thanks for the comments!
> >> > > > > > > > > >> > > > >> > > >>>> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> > I did the following changes
> >> > > > > > > > > >> > > > >> > > >>>> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >    1.
> >> > > > > > > > > >> > > > >> > > >>>> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >    Simplified the API spec
> >> section
> >> > to
> >> > > > > only
> >> > > > > > > > > include
> >> > > > > > > > > >> the
> >> > > > > > > > > >> > > > diff.
> >> > > > > > > > > >> > > > >> > > >>>> >> >    2.
> >> > > > > > > > > >> > > > >> > > >>>> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >    Reordered the HWM
> requirement
> >> > > > section.
> >> > > > > > > > > >> > > > >> > > >>>> >> >    3.
> >> > > > > > > > > >> > > > >> > > >>>> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >    Removed the URM
> >> implementation
> >> > > > details
> >> > > > > > to
> >> > > > > > > > keep
> >> > > > > > > > > >> the
> >> > > > > > > > > >> > > > >> necessary
> >> > > > > > > > > >> > > > >> > > >>>> >> >    characteristics to perform
> >> the
> >> > > > unclean
> >> > > > > > > > > recovery.
> >> > > > > > > > > >> > > > >> > > >>>> >> >    1.
> >> > > > > > > > > >> > > > >> > > >>>> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >       When to perform the
> >> unclean
> >> > > > > recovery
> >> > > > > > > > > >> > > > >> > > >>>> >> >       2.
> >> > > > > > > > > >> > > > >> > > >>>> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >       Under different config,
> >> how
> >> > the
> >> > > > > > unclean
> >> > > > > > > > > >> recovery
> >> > > > > > > > > >> > > > finds
> >> > > > > > > > > >> > > > >> > the
> >> > > > > > > > > >> > > > >> > > >>>> leader.
> >> > > > > > > > > >> > > > >> > > >>>> >> >       3.
> >> > > > > > > > > >> > > > >> > > >>>> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >       How the config
> >> > > > > > > > > unclean.leader.election.enable
> >> > > > > > > > > >> > and
> >> > > > > > > > > >> > > > >> > > >>>> >> >
>  unclean.recovery.strategy
> >> are
> >> > > > > > converted
> >> > > > > > > > > when
> >> > > > > > > > > >> > users
> >> > > > > > > > > >> > > > >> > > >>>> enable/disable
> >> > > > > > > > > >> > > > >> > > >>>> >> the
> >> > > > > > > > > >> > > > >> > > >>>> >> >       unclean recovery.
> >> > > > > > > > > >> > > > >> > > >>>> >> >       4.
> >> > > > > > > > > >> > > > >> > > >>>> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >    More details about how we
> >> change
> >> > > > admin
> >> > > > > > > > client.
> >> > > > > > > > > >> > > > >> > > >>>> >> >    5.
> >> > > > > > > > > >> > > > >> > > >>>> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >    API limits on the
> >> > > > > > GetReplicaLogInfoRequest
> >> > > > > > > > and
> >> > > > > > > > > >> > > > >> > > >>>> DescribeTopicRequest.
> >> > > > > > > > > >> > > > >> > > >>>> >> >    6.
> >> > > > > > > > > >> > > > >> > > >>>> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >    Two metrics added
> >> > > > > > > > > >> > > > >> > > >>>> >> >    1.
> >> > > > > > > > > >> > > > >> > > >>>> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >
> >> > > > > > > > > >> > > >
> >> > Kafka.controller.global_under_min_isr_partition_count
> >> > > > > > > > > >> > > > >> > > >>>> >> >       2.
> >> > > > > > > > > >> > > > >> > > >>>> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >
> >> > > > > > > > > >>  kafka.controller.unclean_recovery_finished_count
> >> > > > > > > > > >> > > > >> > > >>>> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> > On Wed, Sep 13, 2023 at
> 10:46 AM
> >> > > Colin
> >> > > > > > > McCabe <
> >> > > > > > > > > >> > > > >> > > cmccabe@apache.org>
> >> > > > > > > > > >> > > > >> > > >>>> >> wrote:
> >> > > > > > > > > >> > > > >> > > >>>> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >> On Tue, Sep 12, 2023, at
> 17:21,
> >> > > Calvin
> >> > > > > Liu
> >> > > > > > > > > wrote:
> >> > > > > > > > > >> > > > >> > > >>>> >> >> > Hi Colin
> >> > > > > > > > > >> > > > >> > > >>>> >> >> > Thanks for the comments!
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> Hi Calvin,
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> Thanks again for the KIP.
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> One meta-comment: it's
> usually
> >> > > better
> >> > > > to
> >> > > > > > > just
> >> > > > > > > > > do a
> >> > > > > > > > > >> > diff
> >> > > > > > > > > >> > > > on a
> >> > > > > > > > > >> > > > >> > > >>>> message
> >> > > > > > > > > >> > > > >> > > >>>> >> spec
> >> > > > > > > > > >> > > > >> > > >>>> >> >> file or java file if you're
> >> > > including
> >> > > > > > > changes
> >> > > > > > > > to
> >> > > > > > > > > >> it
> >> > > > > > > > > >> > in
> >> > > > > > > > > >> > > > the
> >> > > > > > > > > >> > > > >> > KIP.
> >> > > > > > > > > >> > > > >> > > >>>> This is
> >> > > > > > > > > >> > > > >> > > >>>> >> >> easier to read than looking
> for
> >> > "new
> >> > > > > > fields
> >> > > > > > > > > begin"
> >> > > > > > > > > >> > etc.
> >> > > > > > > > > >> > > > in
> >> > > > > > > > > >> > > > >> the
> >> > > > > > > > > >> > > > >> > > >>>> text, and
> >> > > > > > > > > >> > > > >> > > >>>> >> >> gracefully handles the case
> >> where
> >> > > > > existing
> >> > > > > > > > > fields
> >> > > > > > > > > >> > were
> >> > > > > > > > > >> > > > >> > changed.
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> > Rewrite the Additional High
> >> > > > Watermark
> >> > > > > > > > > >> advancement
> >> > > > > > > > > >> > > > >> > requirement
> >> > > > > > > > > >> > > > >> > > >>>> >> >> > There was feedback on this
> >> > section
> >> > > > > that
> >> > > > > > > some
> >> > > > > > > > > >> > readers
> >> > > > > > > > > >> > > > may
> >> > > > > > > > > >> > > > >> not
> >> > > > > > > > > >> > > > >> > > be
> >> > > > > > > > > >> > > > >> > > >>>> >> familiar
> >> > > > > > > > > >> > > > >> > > >>>> >> >> > with HWM and Ack=0,1,all
> >> > requests.
> >> > > > > This
> >> > > > > > > can
> >> > > > > > > > > help
> >> > > > > > > > > >> > them
> >> > > > > > > > > >> > > > >> > > understand
> >> > > > > > > > > >> > > > >> > > >>>> the
> >> > > > > > > > > >> > > > >> > > >>>> >> >> > proposal. I will rewrite
> this
> >> > part
> >> > > > for
> >> > > > > > > more
> >> > > > > > > > > >> > > > readability.
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> To be clear, I wasn't
> >> suggesting
> >> > > > > dropping
> >> > > > > > > > either
> >> > > > > > > > > >> > > > section. I
> >> > > > > > > > > >> > > > >> > > agree
> >> > > > > > > > > >> > > > >> > > >>>> that
> >> > > > > > > > > >> > > > >> > > >>>> >> >> they add useful background. I
> >> was
> >> > > just
> >> > > > > > > > > suggesting
> >> > > > > > > > > >> > that
> >> > > > > > > > > >> > > we
> >> > > > > > > > > >> > > > >> > should
> >> > > > > > > > > >> > > > >> > > >>>> discuss
> >> > > > > > > > > >> > > > >> > > >>>> >> >> the "acks" setting AFTER
> >> > discussing
> >> > > > the
> >> > > > > > new
> >> > > > > > > > high
> >> > > > > > > > > >> > > > watermark
> >> > > > > > > > > >> > > > >> > > >>>> advancement
> >> > > > > > > > > >> > > > >> > > >>>> >> >> conditions. We also should
> >> discuss
> >> > > > > acks=0.
> >> > > > > > > > While
> >> > > > > > > > > >> it
> >> > > > > > > > > >> > > isn't
> >> > > > > > > > > >> > > > >> > > >>>> conceptually
> >> > > > > > > > > >> > > > >> > > >>>> >> much
> >> > > > > > > > > >> > > > >> > > >>>> >> >> different than acks=1 here,
> its
> >> > > > omission
> >> > > > > > > from
> >> > > > > > > > > this
> >> > > > > > > > > >> > > > section
> >> > > > > > > > > >> > > > >> is
> >> > > > > > > > > >> > > > >> > > >>>> confusing.
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> > Unclean recovery
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >> > The plan is to replace the
> >> > > > > > > > > >> > > > unclean.leader.election.enable
> >> > > > > > > > > >> > > > >> > with
> >> > > > > > > > > >> > > > >> > > >>>> >> >> > unclean.recovery.strategy.
> If
> >> > the
> >> > > > > > Unclean
> >> > > > > > > > > >> Recovery
> >> > > > > > > > > >> > is
> >> > > > > > > > > >> > > > >> > enabled
> >> > > > > > > > > >> > > > >> > > >>>> then it
> >> > > > > > > > > >> > > > >> > > >>>> >> >> deals
> >> > > > > > > > > >> > > > >> > > >>>> >> >> > with the three options in
> the
> >> > > > > > > > > >> > > > unclean.recovery.strategy.
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >> > Let’s refine the Unclean
> >> > Recovery.
> >> > > > We
> >> > > > > > have
> >> > > > > > > > > >> already
> >> > > > > > > > > >> > > > taken a
> >> > > > > > > > > >> > > > >> > > lot of
> >> > > > > > > > > >> > > > >> > > >>>> >> >> > suggestions and I hope to
> >> > enhance
> >> > > > the
> >> > > > > > > > > >> durability of
> >> > > > > > > > > >> > > > Kafka
> >> > > > > > > > > >> > > > >> to
> >> > > > > > > > > >> > > > >> > > the
> >> > > > > > > > > >> > > > >> > > >>>> next
> >> > > > > > > > > >> > > > >> > > >>>> >> >> level
> >> > > > > > > > > >> > > > >> > > >>>> >> >> > with this KIP.
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> I am OK with doing the
> unclean
> >> > > leader
> >> > > > > > > recovery
> >> > > > > > > > > >> > > > improvements
> >> > > > > > > > > >> > > > >> in
> >> > > > > > > > > >> > > > >> > > >>>> this KIP.
> >> > > > > > > > > >> > > > >> > > >>>> >> >> However, I think we need to
> >> really
> >> > > > work
> >> > > > > on
> >> > > > > > > the
> >> > > > > > > > > >> > > > configuration
> >> > > > > > > > > >> > > > >> > > >>>> settings.
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> Configuration overrides are
> >> often
> >> > > > quite
> >> > > > > > > messy.
> >> > > > > > > > > For
> >> > > > > > > > > >> > > > example,
> >> > > > > > > > > >> > > > >> > the
> >> > > > > > > > > >> > > > >> > > >>>> cases
> >> > > > > > > > > >> > > > >> > > >>>> >> >> where we have log.roll.hours
> >> and
> >> > > > > > > > > >> log.roll.segment.ms
> >> > > > > > > > > >> > ,
> >> > > > > > > > > >> > > > the
> >> > > > > > > > > >> > > > >> > user
> >> > > > > > > > > >> > > > >> > > >>>> has to
> >> > > > > > > > > >> > > > >> > > >>>> >> >> remember which one takes
> >> > precedence,
> >> > > > and
> >> > > > > > it
> >> > > > > > > is
> >> > > > > > > > > not
> >> > > > > > > > > >> > > > obvious.
> >> > > > > > > > > >> > > > >> > So,
> >> > > > > > > > > >> > > > >> > > >>>> rather
> >> > > > > > > > > >> > > > >> > > >>>> >> than
> >> > > > > > > > > >> > > > >> > > >>>> >> >> creating a new configuration,
> >> why
> >> > > not
> >> > > > > add
> >> > > > > > > > > >> additional
> >> > > > > > > > > >> > > > values
> >> > > > > > > > > >> > > > >> to
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> "unclean.leader.election.enable"?
> >> > I
> >> > > > > think
> >> > > > > > > this
> >> > > > > > > > > >> will
> >> > > > > > > > > >> > be
> >> > > > > > > > > >> > > > >> simpler
> >> > > > > > > > > >> > > > >> > > for
> >> > > > > > > > > >> > > > >> > > >>>> >> people
> >> > > > > > > > > >> > > > >> > > >>>> >> >> to understand, and simpler in
> >> the
> >> > > code
> >> > > > > as
> >> > > > > > > > well.
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> What if we continued to use
> >> > > > > > > > > >> > > > "unclean.leader.election.enable"
> >> > > > > > > > > >> > > > >> > but
> >> > > > > > > > > >> > > > >> > > >>>> >> extended
> >> > > > > > > > > >> > > > >> > > >>>> >> >> it so that it took a string?
> >> Then
> >> > > the
> >> > > > > > string
> >> > > > > > > > > could
> >> > > > > > > > > >> > have
> >> > > > > > > > > >> > > > >> these
> >> > > > > > > > > >> > > > >> > > >>>> values:
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> never
> >> > > > > > > > > >> > > > >> > > >>>> >> >>     never automatically do an
> >> > > unclean
> >> > > > > > leader
> >> > > > > > > > > >> election
> >> > > > > > > > > >> > > > under
> >> > > > > > > > > >> > > > >> > any
> >> > > > > > > > > >> > > > >> > > >>>> >> conditions
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> false / default
> >> > > > > > > > > >> > > > >> > > >>>> >> >>     only do an unclean leader
> >> > > election
> >> > > > > if
> >> > > > > > > > there
> >> > > > > > > > > >> may
> >> > > > > > > > > >> > be
> >> > > > > > > > > >> > > > >> > possible
> >> > > > > > > > > >> > > > >> > > >>>> data
> >> > > > > > > > > >> > > > >> > > >>>> >> loss
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> true / always
> >> > > > > > > > > >> > > > >> > > >>>> >> >>     always do an unclean
> leader
> >> > > > election
> >> > > > > > if
> >> > > > > > > we
> >> > > > > > > > > >> can't
> >> > > > > > > > > >> > > > >> > immediately
> >> > > > > > > > > >> > > > >> > > >>>> elect a
> >> > > > > > > > > >> > > > >> > > >>>> >> >> leader
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> It's a bit awkward that false
> >> maps
> >> > > to
> >> > > > > > > default
> >> > > > > > > > > >> rather
> >> > > > > > > > > >> > > > than to
> >> > > > > > > > > >> > > > >> > > >>>> never. But
> >> > > > > > > > > >> > > > >> > > >>>> >> >> this awkwardness exists if we
> >> use
> >> > > two
> >> > > > > > > > different
> >> > > > > > > > > >> > > > >> configuration
> >> > > > > > > > > >> > > > >> > > keys
> >> > > > > > > > > >> > > > >> > > >>>> as
> >> > > > > > > > > >> > > > >> > > >>>> >> well.
> >> > > > > > > > > >> > > > >> > > >>>> >> >> The reason for the
> awkwardness
> >> is
> >> > > that
> >> > > > > we
> >> > > > > > > > simply
> >> > > > > > > > > >> > don't
> >> > > > > > > > > >> > > > want
> >> > > > > > > > > >> > > > >> > most
> >> > > > > > > > > >> > > > >> > > >>>> of the
> >> > > > > > > > > >> > > > >> > > >>>> >> >> people currently setting
> >> > > > > > > > > >> > > > >> unclean.leader.election.enable=false
> >> > > > > > > > > >> > > > >> > to
> >> > > > > > > > > >> > > > >> > > >>>> get the
> >> > > > > > > > > >> > > > >> > > >>>> >> >> "never" behavior. We have to
> >> bite
> >> > > that
> >> > > > > > > bullet.
> >> > > > > > > > > >> Better
> >> > > > > > > > > >> > > to
> >> > > > > > > > > >> > > > be
> >> > > > > > > > > >> > > > >> > > clear
> >> > > > > > > > > >> > > > >> > > >>>> and
> >> > > > > > > > > >> > > > >> > > >>>> >> >> explicit than hide it.
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> Another thing that's a bit
> >> awkward
> >> > > is
> >> > > > > > having
> >> > > > > > > > two
> >> > > > > > > > > >> > > > different
> >> > > > > > > > > >> > > > >> > ways
> >> > > > > > > > > >> > > > >> > > to
> >> > > > > > > > > >> > > > >> > > >>>> do
> >> > > > > > > > > >> > > > >> > > >>>> >> >> unclean leader election
> >> specified
> >> > in
> >> > > > the
> >> > > > > > > KIP.
> >> > > > > > > > > You
> >> > > > > > > > > >> > > > descirbe
> >> > > > > > > > > >> > > > >> two
> >> > > > > > > > > >> > > > >> > > >>>> methods:
> >> > > > > > > > > >> > > > >> > > >>>> >> the
> >> > > > > > > > > >> > > > >> > > >>>> >> >> simple "choose the last
> leader"
> >> > > > method,
> >> > > > > > and
> >> > > > > > > > the
> >> > > > > > > > > >> > > "unclean
> >> > > > > > > > > >> > > > >> > > recovery
> >> > > > > > > > > >> > > > >> > > >>>> >> manager"
> >> > > > > > > > > >> > > > >> > > >>>> >> >> method. I understand why you
> >> did
> >> > it
> >> > > > this
> >> > > > > > way
> >> > > > > > > > --
> >> > > > > > > > > >> > "choose
> >> > > > > > > > > >> > > > the
> >> > > > > > > > > >> > > > >> > last
> >> > > > > > > > > >> > > > >> > > >>>> >> leader" is
> >> > > > > > > > > >> > > > >> > > >>>> >> >> simple, and will help us
> >> deliver
> >> > an
> >> > > > > > > > > implementation
> >> > > > > > > > > >> > > > quickly,
> >> > > > > > > > > >> > > > >> > > while
> >> > > > > > > > > >> > > > >> > > >>>> the
> >> > > > > > > > > >> > > > >> > > >>>> >> URM
> >> > > > > > > > > >> > > > >> > > >>>> >> >> is preferable in the long
> >> term. My
> >> > > > > > > suggestion
> >> > > > > > > > > >> here is
> >> > > > > > > > > >> > > to
> >> > > > > > > > > >> > > > >> > > separate
> >> > > > > > > > > >> > > > >> > > >>>> the
> >> > > > > > > > > >> > > > >> > > >>>> >> >> decision of HOW to do unclean
> >> > leader
> >> > > > > > > election
> >> > > > > > > > > from
> >> > > > > > > > > >> > the
> >> > > > > > > > > >> > > > >> > decision
> >> > > > > > > > > >> > > > >> > > of
> >> > > > > > > > > >> > > > >> > > >>>> WHEN
> >> > > > > > > > > >> > > > >> > > >>>> >> to
> >> > > > > > > > > >> > > > >> > > >>>> >> >> do it.
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> So in other words, have
> >> > > > > > > > > >> > > "unclean.leader.election.enable"
> >> > > > > > > > > >> > > > >> > specify
> >> > > > > > > > > >> > > > >> > > >>>> when we
> >> > > > > > > > > >> > > > >> > > >>>> >> >> do unclean leader election,
> and
> >> > > have a
> >> > > > > new
> >> > > > > > > > > >> > > configuration
> >> > > > > > > > > >> > > > >> like
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> "unclean.recovery.manager.enable"
> >> > to
> >> > > > > > > determine
> >> > > > > > > > > if
> >> > > > > > > > > >> we
> >> > > > > > > > > >> > > use
> >> > > > > > > > > >> > > > the
> >> > > > > > > > > >> > > > >> > > URM.
> >> > > > > > > > > >> > > > >> > > >>>> >> >> Presumably the URM will take
> >> some
> >> > > time
> >> > > > > to
> >> > > > > > > get
> >> > > > > > > > > >> fully
> >> > > > > > > > > >> > > > stable,
> >> > > > > > > > > >> > > > >> so
> >> > > > > > > > > >> > > > >> > > >>>> this can
> >> > > > > > > > > >> > > > >> > > >>>> >> >> default to false for a while,
> >> and
> >> > we
> >> > > > can
> >> > > > > > > flip
> >> > > > > > > > > the
> >> > > > > > > > > >> > > > default to
> >> > > > > > > > > >> > > > >> > > true
> >> > > > > > > > > >> > > > >> > > >>>> when
> >> > > > > > > > > >> > > > >> > > >>>> >> we
> >> > > > > > > > > >> > > > >> > > >>>> >> >> feel ready.
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> The URM is somewhat
> >> > under-described
> >> > > > > here.
> >> > > > > > I
> >> > > > > > > > > think
> >> > > > > > > > > >> we
> >> > > > > > > > > >> > > > need a
> >> > > > > > > > > >> > > > >> > few
> >> > > > > > > > > >> > > > >> > > >>>> >> >> configurations here for it.
> For
> >> > > > example,
> >> > > > > > we
> >> > > > > > > > > need a
> >> > > > > > > > > >> > > > >> > > configuration to
> >> > > > > > > > > >> > > > >> > > >>>> >> specify
> >> > > > > > > > > >> > > > >> > > >>>> >> >> how long it should wait for a
> >> > broker
> >> > > > to
> >> > > > > > > > respond
> >> > > > > > > > > to
> >> > > > > > > > > >> > its
> >> > > > > > > > > >> > > > RPCs
> >> > > > > > > > > >> > > > >> > > before
> >> > > > > > > > > >> > > > >> > > >>>> >> moving
> >> > > > > > > > > >> > > > >> > > >>>> >> >> on. We also need to
> understand
> >> how
> >> > > the
> >> > > > > URM
> >> > > > > > > > > >> interacts
> >> > > > > > > > > >> > > with
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > unclean.leader.election.enable=always. I
> >> > > > > > > > assume
> >> > > > > > > > > >> that
> >> > > > > > > > > >> > > with
> >> > > > > > > > > >> > > > >> > > "always"
> >> > > > > > > > > >> > > > >> > > >>>> we
> >> > > > > > > > > >> > > > >> > > >>>> >> will
> >> > > > > > > > > >> > > > >> > > >>>> >> >> just unconditionally use the
> >> URM
> >> > > > rather
> >> > > > > > than
> >> > > > > > > > > >> choosing
> >> > > > > > > > > >> > > > >> > randomly.
> >> > > > > > > > > >> > > > >> > > >>>> But this
> >> > > > > > > > > >> > > > >> > > >>>> >> >> should be spelled out in the
> >> KIP.
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >> > DescribeTopicRequest
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >    1.
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >    Yes, the plan is to
> >> replace
> >> > the
> >> > > > > > > > > >> MetadataRequest
> >> > > > > > > > > >> > > with
> >> > > > > > > > > >> > > > >> the
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest for
> >> the
> >> > > > admin
> >> > > > > > > > clients.
> >> > > > > > > > > >> Will
> >> > > > > > > > > >> > > > check
> >> > > > > > > > > >> > > > >> > the
> >> > > > > > > > > >> > > > >> > > >>>> details.
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> Sounds good. But as I said,
> you
> >> > need
> >> > > > to
> >> > > > > > > > specify
> >> > > > > > > > > >> how
> >> > > > > > > > > >> > > > >> > AdminClient
> >> > > > > > > > > >> > > > >> > > >>>> >> interacts
> >> > > > > > > > > >> > > > >> > > >>>> >> >> with the new request. This
> will
> >> > > > involve
> >> > > > > > > adding
> >> > > > > > > > > >> some
> >> > > > > > > > > >> > > > fields
> >> > > > > > > > > >> > > > >> to
> >> > > > > > > > > >> > > > >> > > >>>> >> >> TopicDescription.java. And
> you
> >> > need
> >> > > to
> >> > > > > > > specify
> >> > > > > > > > > the
> >> > > > > > > > > >> > > > changes
> >> > > > > > > > > >> > > > >> to
> >> > > > > > > > > >> > > > >> > > the
> >> > > > > > > > > >> > > > >> > > >>>> >> >> kafka-topics.sh command line
> >> tool.
> >> > > > > > Otherwise
> >> > > > > > > > we
> >> > > > > > > > > >> > cannot
> >> > > > > > > > > >> > > > use
> >> > > > > > > > > >> > > > >> the
> >> > > > > > > > > >> > > > >> > > >>>> tool to
> >> > > > > > > > > >> > > > >> > > >>>> >> see
> >> > > > > > > > > >> > > > >> > > >>>> >> >> the new information.
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> The new requests,
> >> > > DescribeTopicRequest
> >> > > > > and
> >> > > > > > > > > >> > > > >> > > >>>> GetReplicaLogInfoRequest,
> >> > > > > > > > > >> > > > >> > > >>>> >> need
> >> > > > > > > > > >> > > > >> > > >>>> >> >> to have limits placed on them
> >> so
> >> > > that
> >> > > > > > their
> >> > > > > > > > size
> >> > > > > > > > > >> > can't
> >> > > > > > > > > >> > > be
> >> > > > > > > > > >> > > > >> > > >>>> infinite. We
> >> > > > > > > > > >> > > > >> > > >>>> >> >> don't want to propagate the
> >> > current
> >> > > > > > problems
> >> > > > > > > > of
> >> > > > > > > > > >> > > > >> > MetadataRequest,
> >> > > > > > > > > >> > > > >> > > >>>> where
> >> > > > > > > > > >> > > > >> > > >>>> >> >> clients can request massive
> >> > > responses
> >> > > > > that
> >> > > > > > > can
> >> > > > > > > > > >> mess
> >> > > > > > > > > >> > up
> >> > > > > > > > > >> > > > the
> >> > > > > > > > > >> > > > >> JVM
> >> > > > > > > > > >> > > > >> > > when
> >> > > > > > > > > >> > > > >> > > >>>> >> handled.
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> Adding limits is simple for
> >> > > > > > > > > >> GetReplicaLogInfoRequest
> >> > > > > > > > > >> > --
> >> > > > > > > > > >> > > > we
> >> > > > > > > > > >> > > > >> can
> >> > > > > > > > > >> > > > >> > > >>>> just say
> >> > > > > > > > > >> > > > >> > > >>>> >> >> that only 2000 partitions at
> a
> >> > time
> >> > > > can
> >> > > > > be
> >> > > > > > > > > >> requested.
> >> > > > > > > > > >> > > For
> >> > > > > > > > > >> > > > >> > > >>>> >> >> DescribeTopicRequest we can
> >> > probably
> >> > > > > just
> >> > > > > > > > limit
> >> > > > > > > > > >> to 20
> >> > > > > > > > > >> > > > topics
> >> > > > > > > > > >> > > > >> > or
> >> > > > > > > > > >> > > > >> > > >>>> >> something
> >> > > > > > > > > >> > > > >> > > >>>> >> >> like that, to avoid the
> >> complexity
> >> > > of
> >> > > > > > doing
> >> > > > > > > > > >> > pagination
> >> > > > > > > > > >> > > in
> >> > > > > > > > > >> > > > >> this
> >> > > > > > > > > >> > > > >> > > KIP.
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >    2.
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >    I can let the broker
> load
> >> the
> >> > > ELR
> >> > > > > > info
> >> > > > > > > so
> >> > > > > > > > > >> that
> >> > > > > > > > > >> > > they
> >> > > > > > > > > >> > > > can
> >> > > > > > > > > >> > > > >> > > serve
> >> > > > > > > > > >> > > > >> > > >>>> the
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest as
> >> well.
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> Yes, it's fine to add to
> >> > > > MetadataCache.
> >> > > > > In
> >> > > > > > > > fact,
> >> > > > > > > > > >> > you'll
> >> > > > > > > > > >> > > > be
> >> > > > > > > > > >> > > > >> > > loading
> >> > > > > > > > > >> > > > >> > > >>>> it
> >> > > > > > > > > >> > > > >> > > >>>> >> >> anyway once it's added to
> >> > > > > PartitionImage.
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >    3.
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >    Yeah, it does not make
> >> sense
> >> > to
> >> > > > > have
> >> > > > > > > the
> >> > > > > > > > > >> topic
> >> > > > > > > > > >> > id
> >> > > > > > > > > >> > > if
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest is
> >> only
> >> > > used
> >> > > > > by
> >> > > > > > > the
> >> > > > > > > > > >> admin
> >> > > > > > > > > >> > > > client.
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> OK. That makes things
> simpler.
> >> We
> >> > > can
> >> > > > > > always
> >> > > > > > > > > >> create a
> >> > > > > > > > > >> > > new
> >> > > > > > > > > >> > > > >> API
> >> > > > > > > > > >> > > > >> > > later
> >> > > > > > > > > >> > > > >> > > >>>> >> >> (hopefully not in this KIP!)
> to
> >> > > query
> >> > > > by
> >> > > > > > > topic
> >> > > > > > > > > ID.
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >> > Metrics
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >> > As for overall cluster
> health
> >> > > > > metrics, I
> >> > > > > > > > think
> >> > > > > > > > > >> > > > >> under-min-ISR
> >> > > > > > > > > >> > > > >> > > is
> >> > > > > > > > > >> > > > >> > > >>>> still
> >> > > > > > > > > >> > > > >> > > >>>> >> a
> >> > > > > > > > > >> > > > >> > > >>>> >> >> > useful one. ELR is more
> like
> >> a
> >> > > > safety
> >> > > > > > > belt.
> >> > > > > > > > > When
> >> > > > > > > > > >> > the
> >> > > > > > > > > >> > > > ELR
> >> > > > > > > > > >> > > > >> is
> >> > > > > > > > > >> > > > >> > > >>>> used, the
> >> > > > > > > > > >> > > > >> > > >>>> >> >> > cluster availability has
> >> already
> >> > > > been
> >> > > > > > > > > impacted.
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >> > Maybe we can have a metric
> to
> >> > > count
> >> > > > > the
> >> > > > > > > > > >> partitions
> >> > > > > > > > > >> > > that
> >> > > > > > > > > >> > > > >> > > sum(ISR,
> >> > > > > > > > > >> > > > >> > > >>>> ELR)
> >> > > > > > > > > >> > > > >> > > >>>> >> <
> >> > > > > > > > > >> > > > >> > > >>>> >> >> min
> >> > > > > > > > > >> > > > >> > > >>>> >> >> > ISR. What do you think?
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> How about:
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> A.  a metric for the totoal
> >> number
> >> > > of
> >> > > > > > > > > >> under-min-isr
> >> > > > > > > > > >> > > > >> > partitions?
> >> > > > > > > > > >> > > > >> > > We
> >> > > > > > > > > >> > > > >> > > >>>> don't
> >> > > > > > > > > >> > > > >> > > >>>> >> >> have that in Apache Kafka at
> >> the
> >> > > > moment.
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> B. a metric for the number of
> >> > > unclean
> >> > > > > > leader
> >> > > > > > > > > >> > elections
> >> > > > > > > > > >> > > we
> >> > > > > > > > > >> > > > >> did
> >> > > > > > > > > >> > > > >> > > (for
> >> > > > > > > > > >> > > > >> > > >>>> >> >> simplicity, it can reset to 0
> >> on
> >> > > > > > controller
> >> > > > > > > > > >> restart:
> >> > > > > > > > > >> > we
> >> > > > > > > > > >> > > > >> expect
> >> > > > > > > > > >> > > > >> > > >>>> people to
> >> > > > > > > > > >> > > > >> > > >>>> >> >> monitor the change over time
> >> > anyway)
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> best,
> >> > > > > > > > > >> > > > >> > > >>>> >> >> Colin
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >> > Yeah, for the ongoing
> unclean
> >> > > > > > recoveries,
> >> > > > > > > > the
> >> > > > > > > > > >> > > > controller
> >> > > > > > > > > >> > > > >> can
> >> > > > > > > > > >> > > > >> > > >>>> keep an
> >> > > > > > > > > >> > > > >> > > >>>> >> >> > accurate count through
> >> failover
> >> > > > > because
> >> > > > > > > > > >> partition
> >> > > > > > > > > >> > > > >> > registration
> >> > > > > > > > > >> > > > >> > > >>>> can
> >> > > > > > > > > >> > > > >> > > >>>> >> >> indicate
> >> > > > > > > > > >> > > > >> > > >>>> >> >> > whether a recovery is
> needed.
> >> > > > However,
> >> > > > > > for
> >> > > > > > > > the
> >> > > > > > > > > >> > > happened
> >> > > > > > > > > >> > > > >> > ones,
> >> > > > > > > > > >> > > > >> > > >>>> unless
> >> > > > > > > > > >> > > > >> > > >>>> >> we
> >> > > > > > > > > >> > > > >> > > >>>> >> >> > want to persist the number
> >> > > > somewhere,
> >> > > > > we
> >> > > > > > > can
> >> > > > > > > > > >> only
> >> > > > > > > > > >> > > > figure
> >> > > > > > > > > >> > > > >> it
> >> > > > > > > > > >> > > > >> > > out
> >> > > > > > > > > >> > > > >> > > >>>> from
> >> > > > > > > > > >> > > > >> > > >>>> >> the
> >> > > > > > > > > >> > > > >> > > >>>> >> >> > log.
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >> > On Tue, Sep 12, 2023 at
> >> 3:16 PM
> >> > > > Colin
> >> > > > > > > > McCabe <
> >> > > > > > > > > >> > > > >> > > cmccabe@apache.org
> >> > > > > > > > > >> > > > >> > > >>>> >
> >> > > > > > > > > >> > > > >> > > >>>> >> wrote:
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> Also, we should have
> metrics
> >> > that
> >> > > > > show
> >> > > > > > > what
> >> > > > > > > > > is
> >> > > > > > > > > >> > going
> >> > > > > > > > > >> > > > on
> >> > > > > > > > > >> > > > >> > with
> >> > > > > > > > > >> > > > >> > > >>>> regard
> >> > > > > > > > > >> > > > >> > > >>>> >> to
> >> > > > > > > > > >> > > > >> > > >>>> >> >> the
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> eligible replica set. I'm
> >> not
> >> > > sure
> >> > > > > > > exactly
> >> > > > > > > > > >> what to
> >> > > > > > > > > >> > > > >> suggest,
> >> > > > > > > > > >> > > > >> > > but
> >> > > > > > > > > >> > > > >> > > >>>> >> >> something
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> that could identify when
> >> things
> >> > > are
> >> > > > > > going
> >> > > > > > > > > >> wrong in
> >> > > > > > > > > >> > > the
> >> > > > > > > > > >> > > > >> > > clsuter.
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> For example, maybe a
> metric
> >> for
> >> > > > > > > partitions
> >> > > > > > > > > >> > > containing
> >> > > > > > > > > >> > > > >> > > replicas
> >> > > > > > > > > >> > > > >> > > >>>> that
> >> > > > > > > > > >> > > > >> > > >>>> >> are
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> ineligible to be leader?
> >> That
> >> > > would
> >> > > > > > show
> >> > > > > > > a
> >> > > > > > > > > >> spike
> >> > > > > > > > > >> > > when
> >> > > > > > > > > >> > > > a
> >> > > > > > > > > >> > > > >> > > broker
> >> > > > > > > > > >> > > > >> > > >>>> had an
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> unclean restart.
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> Ideally, we'd also have a
> >> > metric
> >> > > > that
> >> > > > > > > > > indicates
> >> > > > > > > > > >> > when
> >> > > > > > > > > >> > > > an
> >> > > > > > > > > >> > > > >> > > unclear
> >> > > > > > > > > >> > > > >> > > >>>> >> leader
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> election or a recovery
> >> > happened.
> >> > > > > It's a
> >> > > > > > > bit
> >> > > > > > > > > >> tricky
> >> > > > > > > > > >> > > > >> because
> >> > > > > > > > > >> > > > >> > > the
> >> > > > > > > > > >> > > > >> > > >>>> simple
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> thing, of tracking it per
> >> > > > controller,
> >> > > > > > may
> >> > > > > > > > be
> >> > > > > > > > > a
> >> > > > > > > > > >> bit
> >> > > > > > > > > >> > > > >> > confusing
> >> > > > > > > > > >> > > > >> > > >>>> during
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> failovers.
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> best,
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> Colin
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >>
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> On Tue, Sep 12, 2023, at
> >> 14:25,
> >> > > > Colin
> >> > > > > > > > McCabe
> >> > > > > > > > > >> > wrote:
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > Hi Calvin,
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > Thanks for the KIP. I
> >> think
> >> > > this
> >> > > > > is a
> >> > > > > > > > great
> >> > > > > > > > > >> > > > >> improvement.
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >> Additional High
> Watermark
> >> > > > advance
> >> > > > > > > > > >> requirement
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > Typo: change "advance"
> to
> >> > > > > > "advancement"
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >> A bit recap of some key
> >> > > > concepts.
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > Typo: change "bit" to
> >> "quick"
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >> Ack=1/all produce
> >> request.
> >> > It
> >> > > > > > defines
> >> > > > > > > > when
> >> > > > > > > > > >> the
> >> > > > > > > > > >> > > > Kafka
> >> > > > > > > > > >> > > > >> > > server
> >> > > > > > > > > >> > > > >> > > >>>> should
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> respond to the produce
> >> request
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > I think this section
> >> would be
> >> > > > > clearer
> >> > > > > > > if
> >> > > > > > > > we
> >> > > > > > > > > >> > talked
> >> > > > > > > > > >> > > > >> about
> >> > > > > > > > > >> > > > >> > > the
> >> > > > > > > > > >> > > > >> > > >>>> new
> >> > > > > > > > > >> > > > >> > > >>>> >> high
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > watermark advancement
> >> > > requirement
> >> > > > > > > first,
> >> > > > > > > > > and
> >> > > > > > > > > >> > THEN
> >> > > > > > > > > >> > > > >> talked
> >> > > > > > > > > >> > > > >> > > >>>> about its
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > impact on acks=0,
> acks=1,
> >> and
> >> > > > > > > >  acks=all.
> >> > > > > > > > > >> > > > acks=all
> >> > > > > > > > > >> > > > >> is
> >> > > > > > > > > >> > > > >> > of
> >> > > > > > > > > >> > > > >> > > >>>> course
> >> > > > > > > > > >> > > > >> > > >>>> >> the
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > main case we care about
> >> here,
> >> > > so
> >> > > > it
> >> > > > > > > would
> >> > > > > > > > > be
> >> > > > > > > > > >> > good
> >> > > > > > > > > >> > > to
> >> > > > > > > > > >> > > > >> lead
> >> > > > > > > > > >> > > > >> > > with
> >> > > > > > > > > >> > > > >> > > >>>> >> that,
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > rather than delving into
> >> the
> >> > > > > > > > technicalities
> >> > > > > > > > > >> of
> >> > > > > > > > > >> > > > acks=0/1
> >> > > > > > > > > >> > > > >> > > first.
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >> Unclean recovery
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > So, here you are
> >> introducing
> >> > a
> >> > > > new
> >> > > > > > > > > >> > configuration,
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> unclean.recovery.strategy.
> >> > The
> >> > > > > > > difficult
> >> > > > > > > > > >> thing
> >> > > > > > > > > >> > > here
> >> > > > > > > > > >> > > > is
> >> > > > > > > > > >> > > > >> > that
> >> > > > > > > > > >> > > > >> > > >>>> there
> >> > > > > > > > > >> > > > >> > > >>>> >> is a
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > lot of overlap with
> >> > > > > > > > > >> > > unclean.leader.election.enable.
> >> > > > > > > > > >> > > > So
> >> > > > > > > > > >> > > > >> we
> >> > > > > > > > > >> > > > >> > > >>>> have 3
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > different settings for
> >> > > > > > > > > >> > unclean.recovery.strategy,
> >> > > > > > > > > >> > > > plus
> >> > > > > > > > > >> > > > >> 2
> >> > > > > > > > > >> > > > >> > > >>>> different
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > settings for
> >> > > > > > > > > unclean.leader.election.enable,
> >> > > > > > > > > >> > > giving
> >> > > > > > > > > >> > > > a
> >> > > > > > > > > >> > > > >> > cross
> >> > > > > > > > > >> > > > >> > > >>>> >> product of
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > 6 different options. The
> >> > > > following
> >> > > > > > > > "unclean
> >> > > > > > > > > >> > > recovery
> >> > > > > > > > > >> > > > >> > > manager"
> >> > > > > > > > > >> > > > >> > > >>>> >> section
> >> > > > > > > > > >> > > > >> > > >>>> >> >> >> > on
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

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

Another thing. Currently, when there is an unclean leader election, we set
the LeaderRecoveryState in PartitionRecord and PartitionChangeRecord to 1.
With the KIP, will there be new values for LeaderRecoveryState? If not,
when will LeaderRecoveryState be set to 1?

Thanks,

Jun

On Tue, Oct 10, 2023 at 4:24 PM Jun Rao <ju...@confluent.io> wrote:

> Hi, Calvin,
>
> One more comment.
>
> "The first partition to fetch details for. -1 means to fetch all
> partitions." It seems that FirstPartitionId of 0 naturally means fetching
> all partitions?
>
> Thanks,
>
> Jun
>
> On Tue, Oct 10, 2023 at 12:40 PM Calvin Liu <ca...@confluent.io.invalid>
> wrote:
>
>> Hi Jun,
>> Yeah, with the current Metadata request handling, we only return errors on
>> the Topic level, like topic not found. It seems that querying a specific
>> partition is not a valid use case. Will update.
>> Thanks
>>
>> On Tue, Oct 10, 2023 at 11:55 AM Jun Rao <ju...@confluent.io.invalid>
>> wrote:
>>
>> > Hi, Calvin,
>> >
>> > 60.  If the range query has errors for some of the partitions, do we
>> expect
>> > different responses when querying particular partitions?
>> >
>> > Thanks,
>> >
>> > Jun
>> >
>> > On Tue, Oct 10, 2023 at 10:50 AM Calvin Liu <caliu@confluent.io.invalid
>> >
>> > wrote:
>> >
>> > > Hi Jun
>> > > 60. Yes, it is a good question. I was thinking the API could be
>> flexible
>> > to
>> > > query the particular partitions if the range query has errors for
>> some of
>> > > the partitions. Not sure whether it is a valid assumption, what do you
>> > > think?
>> > >
>> > > 61. Good point, I will update them to partition level with the same
>> > limit.
>> > >
>> > > 62. Sure, will do.
>> > >
>> > > Thanks
>> > >
>> > > On Tue, Oct 10, 2023 at 10:12 AM Jun Rao <ju...@confluent.io.invalid>
>> > wrote:
>> > >
>> > > > Hi, Calvin,
>> > > >
>> > > > A few more minor comments on your latest update.
>> > > >
>> > > > 60. DescribeTopicRequest: When will the Partitions field be used? It
>> > > seems
>> > > > that the FirstPartitionId field is enough for AdminClient usage.
>> > > >
>> > > > 61. Could we make the limit for DescribeTopicRequest,
>> > > ElectLeadersRequest,
>> > > > GetReplicaLogInfo consistent? Currently, ElectLeadersRequest's
>> limit is
>> > > at
>> > > > topic level and GetReplicaLogInfo has a different partition level
>> limit
>> > > > from DescribeTopicRequest.
>> > > >
>> > > > 62. Should ElectLeadersRequest.DesiredLeaders be at the same level
>> as
>> > > > ElectLeadersRequest.TopicPartitions.Partitions? In the KIP, it looks
>> > like
>> > > > it's at the same level as ElectLeadersRequest.TopicPartitions.
>> > > >
>> > > > Thanks,
>> > > >
>> > > > Jun
>> > > >
>> > > > On Wed, Oct 4, 2023 at 3:55 PM Calvin Liu
>> <ca...@confluent.io.invalid>
>> > > > wrote:
>> > > >
>> > > > > Hi David,
>> > > > > Thanks for the comments.
>> > > > > ----
>> > > > > I thought that a new snapshot with the downgraded MV is created in
>> > this
>> > > > > case. Isn’t it the case?
>> > > > > Yes, you are right, a metadata delta will be generated after the
>> MV
>> > > > > downgrade. Then the user can start the software downgrade.
>> > > > > -----
>> > > > > Could you also elaborate a bit more on the reasoning behind adding
>> > the
>> > > > > limits to the admin RPCs? This is a new pattern in Kafka so it
>> would
>> > be
>> > > > > good to clear on the motivation.
>> > > > > Thanks to Colin for bringing it up. The current MetadataRequest
>> does
>> > > not
>> > > > > have a limit on the number of topics to query in a single request.
>> > > > Massive
>> > > > > requests can mess up the JVM. We want to have some sort of
>> throttle
>> > on
>> > > > the
>> > > > > new APIs.
>> > > > > -----
>> > > > > Could you also explain how the client is supposed to handle the
>> > > > > topics/partitions above the limit? I suppose that it will have to
>> > retry
>> > > > > those, correct?
>> > > > > Corrent. For the official admin clients, it will split the large
>> > > request
>> > > > > into proper pieces and query one after another.
>> > > > > -----
>> > > > > My understanding is that the topics/partitions above the limit
>> will
>> > be
>> > > > > failed with an invalid exception error. I wonder if this choice is
>> > > > > judicious because the invalide request exception is usually
>> fatal. It
>> > > may
>> > > > > be better to use an new and explicit error for this case.
>> > > > >
>> > > > > Thanks for bringing this up. How about "REQUEST_LIMIT_REACHED"?
>> > > > > --------
>> > > > > It seems that we still need to specify the changes to the admin
>> api
>> > to
>> > > > > accommodate the new or updated apis. Do you plan to add them?
>> > > > > Try to cover the following
>> > > > > 1. The admin client will use the new DescribeTopicRequest to query
>> > the
>> > > > > topics
>> > > > > 2. Mention the API limit and the new retriable error.
>> > > > > 3. Output changes for the admin client when describing a topic
>> (new
>> > > > fields
>> > > > > of ELR...)
>> > > > > 4. Changes to data structures like TopicPartitionInfo to include
>> the
>> > > ELR.
>> > > > > Anything else I missed?
>> > > > >
>> > > > > Thanks!
>> > > > >
>> > > > >
>> > > > >
>> > > > >
>> > > > >
>> > > > > On Wed, Oct 4, 2023 at 12:27 PM David Jacot <
>> david.jacot@gmail.com>
>> > > > wrote:
>> > > > >
>> > > > > > Hi Calvin,
>> > > > > >
>> > > > > > I thought that a new snapshot with the downgraded MV is created
>> in
>> > > this
>> > > > > > case. Isn’t it the case?
>> > > > > >
>> > > > > > Could you also elaborate a bit more on the reasoning behind
>> adding
>> > > the
>> > > > > > limits to the admin RPCs? This is a new pattern in Kafka so it
>> > would
>> > > be
>> > > > > > good to clear on the motivation.
>> > > > > >
>> > > > > > Could you also explain how the client is supposed to handle the
>> > > > > > topics/partitions above the limit? I suppose that it will have
>> to
>> > > retry
>> > > > > > those, correct?
>> > > > > >
>> > > > > > My understanding is that the topics/partitions above the limit
>> will
>> > > be
>> > > > > > failed with an invalid exception error. I wonder if this choice
>> is
>> > > > > > judicious because the invalide request exception is usually
>> fatal.
>> > It
>> > > > may
>> > > > > > be better to use an new and explicit error for this case.
>> > > > > >
>> > > > > > It seems that we still need to specify the changes to the admin
>> api
>> > > to
>> > > > > > accommodate the new or updated apis. Do you plan to add them?
>> > > > > >
>> > > > > > Best,
>> > > > > > David
>> > > > > >
>> > > > > > Le mer. 4 oct. 2023 à 20:39, Calvin Liu
>> <caliu@confluent.io.invalid
>> > >
>> > > a
>> > > > > > écrit :
>> > > > > >
>> > > > > > > Hi Jun,
>> > > > > > > After the MV downgrade, the controller will write in the old
>> > > version
>> > > > of
>> > > > > > the
>> > > > > > > PartitionRecord/PartitionChangeRecord. If I understand
>> correctly,
>> > > it
>> > > > is
>> > > > > > > possible to downgrade the software version if the controller
>> only
>> > > has
>> > > > > to
>> > > > > > > handle old version records.
>> > > > > > > However, the controller will not automatically rewrite the
>> > > > > > PartitionRecord
>> > > > > > > with the old version unless there is a partition update. Then,
>> > the
>> > > > user
>> > > > > > may
>> > > > > > > have to wait an unknown amount of time before the software
>> > > downgrades
>> > > > > > > unless they do a roll to force update every partition. If it
>> > makes
>> > > > > > sense, I
>> > > > > > > can mention these steps to do a software downgrade.
>> > > > > > > Thanks
>> > > > > > >
>> > > > > > > On Wed, Oct 4, 2023 at 11:20 AM Jun Rao
>> <jun@confluent.io.invalid
>> > >
>> > > > > > wrote:
>> > > > > > >
>> > > > > > > > Hi, Calvin and Justine,
>> > > > > > > >
>> > > > > > > > Historically, when we change the record format in the log,
>> we
>> > > don't
>> > > > > > > support
>> > > > > > > > software version downgrading.
>> > > > > > > >
>> > > > > > > > For the record format change in the metadata log, have we
>> > thought
>> > > > > about
>> > > > > > > > forcing the write of the latest metadata records with the
>> old
>> > > > version
>> > > > > > > > during MV downgrading? This will in theory allow the old
>> > version
>> > > of
>> > > > > the
>> > > > > > > > software to obtain the latest metadata.
>> > > > > > > >
>> > > > > > > > Thanks,
>> > > > > > > >
>> > > > > > > > Jun
>> > > > > > > >
>> > > > > > > > On Wed, Oct 4, 2023 at 9:53 AM Justine Olshan
>> > > > > > > <jolshan@confluent.io.invalid
>> > > > > > > > >
>> > > > > > > > wrote:
>> > > > > > > >
>> > > > > > > > > Sorry -- not MV but software version.
>> > > > > > > > >
>> > > > > > > > > On Wed, Oct 4, 2023 at 9:51 AM Justine Olshan <
>> > > > > jolshan@confluent.io>
>> > > > > > > > > wrote:
>> > > > > > > > >
>> > > > > > > > > > Catching up with this discussion.
>> > > > > > > > > >
>> > > > > > > > > > I was just curious -- have we had other instances where
>> > > > > downgrading
>> > > > > > > MV
>> > > > > > > > is
>> > > > > > > > > > not supported? I think Kafka typically tries to support
>> > > > > downgrades,
>> > > > > > > > and I
>> > > > > > > > > > couldn't think of other examples.
>> > > > > > > > > >
>> > > > > > > > > > Thanks,
>> > > > > > > > > > Justine
>> > > > > > > > > >
>> > > > > > > > > > On Wed, Oct 4, 2023 at 9:40 AM Calvin Liu
>> > > > > > <caliu@confluent.io.invalid
>> > > > > > > >
>> > > > > > > > > > wrote:
>> > > > > > > > > >
>> > > > > > > > > >> Hi Jun,
>> > > > > > > > > >> 54. Marked the software downgrading is not supported.
>> As
>> > the
>> > > > old
>> > > > > > > > > >> controller
>> > > > > > > > > >> will not understand the new PartitionRecord and
>> > > > > > > PartitionChangeRecord.
>> > > > > > > > > >> Thanks!
>> > > > > > > > > >>
>> > > > > > > > > >> On Wed, Oct 4, 2023 at 9:12 AM Jun Rao
>> > > > <jun@confluent.io.invalid
>> > > > > >
>> > > > > > > > > wrote:
>> > > > > > > > > >>
>> > > > > > > > > >> > Hi, Calvin,
>> > > > > > > > > >> >
>> > > > > > > > > >> > Thanks for the reply. Just one more comment.
>> > > > > > > > > >> >
>> > > > > > > > > >> > 54. It seems that downgrading MV is supported. Is
>> > > > downgrading
>> > > > > > the
>> > > > > > > > > >> software
>> > > > > > > > > >> > version supported? It would be useful to document
>> that.
>> > > > > > > > > >> >
>> > > > > > > > > >> > Thanks,
>> > > > > > > > > >> >
>> > > > > > > > > >> > Jun
>> > > > > > > > > >> >
>> > > > > > > > > >> > On Tue, Oct 3, 2023 at 4:55 PM Artem Livshits
>> > > > > > > > > >> > <al...@confluent.io.invalid> wrote:
>> > > > > > > > > >> >
>> > > > > > > > > >> > > Hi Colin,
>> > > > > > > > > >> > >
>> > > > > > > > > >> > > I think in your example "do_unclean_recovery" would
>> > need
>> > > > to
>> > > > > do
>> > > > > > > > > >> different
>> > > > > > > > > >> > > things depending on the strategy.
>> > > > > > > > > >> > >
>> > > > > > > > > >> > > do_unclean_recovery() {
>> > > > > > > > > >> > >    if (unclean.recovery.manager.enabled) {
>> > > > > > > > > >> > >     if (strategy == Aggressive)
>> > > > > > > > > >> > >       use
>> > UncleanRecoveryManager(waitLastKnownERL=false)
>> > > > //
>> > > > > > > just
>> > > > > > > > > >> inspect
>> > > > > > > > > >> > > logs from whoever is available
>> > > > > > > > > >> > >     else
>> > > > > > > > > >> > >       use
>> > UncleanRecoveryManager(waitLastKnownERL=true)
>> > > > //
>> > > > > > > must
>> > > > > > > > > wait
>> > > > > > > > > >> > for
>> > > > > > > > > >> > > at least last known ELR
>> > > > > > > > > >> > >   } else {
>> > > > > > > > > >> > >     if (strategy == Aggressive)
>> > > > > > > > > >> > >       choose the last known leader if that is
>> > available,
>> > > > or
>> > > > > a
>> > > > > > > > random
>> > > > > > > > > >> > leader
>> > > > > > > > > >> > > if not)
>> > > > > > > > > >> > >     else
>> > > > > > > > > >> > >       wait for last known leader to get back
>> > > > > > > > > >> > >   }
>> > > > > > > > > >> > > }
>> > > > > > > > > >> > >
>> > > > > > > > > >> > > The idea is that the Aggressive strategy would
>> kick in
>> > > as
>> > > > > soon
>> > > > > > > as
>> > > > > > > > we
>> > > > > > > > > >> lost
>> > > > > > > > > >> > > the leader and would pick a leader from whoever is
>> > > > > available;
>> > > > > > > but
>> > > > > > > > > the
>> > > > > > > > > >> > > Balanced will only kick in when ELR is empty and
>> will
>> > > wait
>> > > > > for
>> > > > > > > the
>> > > > > > > > > >> > brokers
>> > > > > > > > > >> > > that likely have most data to be available.
>> > > > > > > > > >> > >
>> > > > > > > > > >> > > On Tue, Oct 3, 2023 at 3:04 PM Colin McCabe <
>> > > > > > cmccabe@apache.org
>> > > > > > > >
>> > > > > > > > > >> wrote:
>> > > > > > > > > >> > >
>> > > > > > > > > >> > > > On Tue, Oct 3, 2023, at 10:49, Jun Rao wrote:
>> > > > > > > > > >> > > > > Hi, Calvin,
>> > > > > > > > > >> > > > >
>> > > > > > > > > >> > > > > Thanks for the update KIP. A few more comments.
>> > > > > > > > > >> > > > >
>> > > > > > > > > >> > > > > 41. Why would a user choose the option to
>> select a
>> > > > > random
>> > > > > > > > > replica
>> > > > > > > > > >> as
>> > > > > > > > > >> > > the
>> > > > > > > > > >> > > > > leader instead of using
>> > > > > > unclean.recovery.strateg=Aggressive?
>> > > > > > > > It
>> > > > > > > > > >> seems
>> > > > > > > > > >> > > > that
>> > > > > > > > > >> > > > > the latter is strictly better? If that's not
>> the
>> > > case,
>> > > > > > could
>> > > > > > > > we
>> > > > > > > > > >> fold
>> > > > > > > > > >> > > this
>> > > > > > > > > >> > > > > option under unclean.recovery.strategy instead
>> of
>> > > > > > > introducing
>> > > > > > > > a
>> > > > > > > > > >> > > separate
>> > > > > > > > > >> > > > > config?
>> > > > > > > > > >> > > >
>> > > > > > > > > >> > > > Hi Jun,
>> > > > > > > > > >> > > >
>> > > > > > > > > >> > > > I thought the flow of control was:
>> > > > > > > > > >> > > >
>> > > > > > > > > >> > > > If there is no leader for the partition {
>> > > > > > > > > >> > > >   If (there are unfenced ELR members) {
>> > > > > > > > > >> > > >     choose_an_unfenced_ELR_member
>> > > > > > > > > >> > > >   } else if (there are fenced ELR members AND
>> > > > > > > > > strategy=Aggressive) {
>> > > > > > > > > >> > > >     do_unclean_recovery
>> > > > > > > > > >> > > >   } else if (there are no ELR members AND
>> strategy
>> > !=
>> > > > > None)
>> > > > > > {
>> > > > > > > > > >> > > >     do_unclean_recovery
>> > > > > > > > > >> > > >   } else {
>> > > > > > > > > >> > > >     do nothing about the missing leader
>> > > > > > > > > >> > > >   }
>> > > > > > > > > >> > > > }
>> > > > > > > > > >> > > >
>> > > > > > > > > >> > > > do_unclean_recovery() {
>> > > > > > > > > >> > > >    if (unclean.recovery.manager.enabled) {
>> > > > > > > > > >> > > >     use UncleanRecoveryManager
>> > > > > > > > > >> > > >   } else {
>> > > > > > > > > >> > > >     choose the last known leader if that is
>> > available,
>> > > > or
>> > > > > a
>> > > > > > > > random
>> > > > > > > > > >> > leader
>> > > > > > > > > >> > > > if not)
>> > > > > > > > > >> > > >   }
>> > > > > > > > > >> > > > }
>> > > > > > > > > >> > > >
>> > > > > > > > > >> > > > However, I think this could be clarified,
>> especially
>> > > the
>> > > > > > > > behavior
>> > > > > > > > > >> when
>> > > > > > > > > >> > > > unclean.recovery.manager.enabled=false.
>> Inuitively
>> > the
>> > > > > goal
>> > > > > > > for
>> > > > > > > > > >> > > > unclean.recovery.manager.enabled=false is to be
>> "the
>> > > > same
>> > > > > as
>> > > > > > > > now,
>> > > > > > > > > >> > mostly"
>> > > > > > > > > >> > > > but it's very underspecified in the KIP, I agree.
>> > > > > > > > > >> > > >
>> > > > > > > > > >> > > > >
>> > > > > > > > > >> > > > > 50. ElectLeadersRequest: "If more than 20
>> topics
>> > are
>> > > > > > > included,
>> > > > > > > > > >> only
>> > > > > > > > > >> > the
>> > > > > > > > > >> > > > > first 20 will be served. Others will be
>> returned
>> > > with
>> > > > > > > > > >> > DesiredLeaders."
>> > > > > > > > > >> > > > Hmm,
>> > > > > > > > > >> > > > > not sure that I understand this.
>> > > ElectLeadersResponse
>> > > > > > > doesn't
>> > > > > > > > > >> have a
>> > > > > > > > > >> > > > > DesiredLeaders field.
>> > > > > > > > > >> > > > >
>> > > > > > > > > >> > > > > 51. GetReplicaLogInfo: "If more than 2000
>> > partitions
>> > > > are
>> > > > > > > > > included,
>> > > > > > > > > >> > only
>> > > > > > > > > >> > > > the
>> > > > > > > > > >> > > > > first 2000 will be served" Do we return an
>> error
>> > for
>> > > > the
>> > > > > > > > > remaining
>> > > > > > > > > >> > > > > partitions? Actually, should we include an
>> > errorCode
>> > > > > field
>> > > > > > > at
>> > > > > > > > > the
>> > > > > > > > > >> > > > partition
>> > > > > > > > > >> > > > > level in GetReplicaLogInfoResponse to cover
>> > > > non-existing
>> > > > > > > > > >> partitions
>> > > > > > > > > >> > and
>> > > > > > > > > >> > > > no
>> > > > > > > > > >> > > > > authorization, etc?
>> > > > > > > > > >> > > > >
>> > > > > > > > > >> > > > > 52. The entry should matches => The entry
>> should
>> > > match
>> > > > > > > > > >> > > > >
>> > > > > > > > > >> > > > > 53. ElectLeadersRequest.DesiredLeaders: Should
>> it
>> > be
>> > > > > > > nullable
>> > > > > > > > > >> since a
>> > > > > > > > > >> > > > user
>> > > > > > > > > >> > > > > may not specify DesiredLeaders?
>> > > > > > > > > >> > > > >
>> > > > > > > > > >> > > > > 54. Downgrade: Is that indeed possible? I
>> thought
>> > > > > earlier
>> > > > > > > you
>> > > > > > > > > said
>> > > > > > > > > >> > that
>> > > > > > > > > >> > > > > once the new version of the records are in the
>> > > > metadata
>> > > > > > log,
>> > > > > > > > one
>> > > > > > > > > >> > can't
>> > > > > > > > > >> > > > > downgrade since the old broker doesn't know
>> how to
>> > > > parse
>> > > > > > the
>> > > > > > > > new
>> > > > > > > > > >> > > version
>> > > > > > > > > >> > > > of
>> > > > > > > > > >> > > > > the metadata records?
>> > > > > > > > > >> > > > >
>> > > > > > > > > >> > > >
>> > > > > > > > > >> > > > MetadataVersion downgrade is currently broken
>> but we
>> > > > have
>> > > > > > > fixing
>> > > > > > > > > it
>> > > > > > > > > >> on
>> > > > > > > > > >> > > our
>> > > > > > > > > >> > > > plate for Kafka 3.7.
>> > > > > > > > > >> > > >
>> > > > > > > > > >> > > > The way downgrade works is that "new features"
>> are
>> > > > > dropped,
>> > > > > > > > > leaving
>> > > > > > > > > >> > only
>> > > > > > > > > >> > > > the old ones.
>> > > > > > > > > >> > > >
>> > > > > > > > > >> > > > > 55. CleanShutdownFile: Should we add a version
>> > field
>> > > > for
>> > > > > > > > future
>> > > > > > > > > >> > > > extension?
>> > > > > > > > > >> > > > >
>> > > > > > > > > >> > > > > 56. Config changes are public facing. Could we
>> > have
>> > > a
>> > > > > > > separate
>> > > > > > > > > >> > section
>> > > > > > > > > >> > > to
>> > > > > > > > > >> > > > > document all the config changes?
>> > > > > > > > > >> > > >
>> > > > > > > > > >> > > > +1. A separate section for this would be good.
>> > > > > > > > > >> > > >
>> > > > > > > > > >> > > > best,
>> > > > > > > > > >> > > > Colin
>> > > > > > > > > >> > > >
>> > > > > > > > > >> > > > >
>> > > > > > > > > >> > > > > Thanks,
>> > > > > > > > > >> > > > >
>> > > > > > > > > >> > > > > Jun
>> > > > > > > > > >> > > > >
>> > > > > > > > > >> > > > > On Mon, Sep 25, 2023 at 4:29 PM Calvin Liu
>> > > > > > > > > >> > <caliu@confluent.io.invalid
>> > > > > > > > > >> > > >
>> > > > > > > > > >> > > > > wrote:
>> > > > > > > > > >> > > > >
>> > > > > > > > > >> > > > >> Hi Jun
>> > > > > > > > > >> > > > >> Thanks for the comments.
>> > > > > > > > > >> > > > >>
>> > > > > > > > > >> > > > >> 40. If we change to None, it is not guaranteed
>> > for
>> > > no
>> > > > > > data
>> > > > > > > > > loss.
>> > > > > > > > > >> For
>> > > > > > > > > >> > > > users
>> > > > > > > > > >> > > > >> who are not able to validate the data with
>> > external
>> > > > > > > > resources,
>> > > > > > > > > >> > manual
>> > > > > > > > > >> > > > >> intervention does not give a better result
>> but a
>> > > loss
>> > > > > of
>> > > > > > > > > >> > availability.
>> > > > > > > > > >> > > > So
>> > > > > > > > > >> > > > >> practically speaking, the Balance mode would
>> be a
>> > > > > better
>> > > > > > > > > default
>> > > > > > > > > >> > > value.
>> > > > > > > > > >> > > > >>
>> > > > > > > > > >> > > > >> 41. No, it represents how we want to do the
>> > unclean
>> > > > > > leader
>> > > > > > > > > >> election.
>> > > > > > > > > >> > > If
>> > > > > > > > > >> > > > it
>> > > > > > > > > >> > > > >> is false, the unclean leader election will be
>> the
>> > > old
>> > > > > > > random
>> > > > > > > > > way.
>> > > > > > > > > >> > > > >> Otherwise, the unclean recovery will be used.
>> > > > > > > > > >> > > > >>
>> > > > > > > > > >> > > > >> 42. Good catch. Updated.
>> > > > > > > > > >> > > > >>
>> > > > > > > > > >> > > > >> 43. Only the first 20 topics will be served.
>> > Others
>> > > > > will
>> > > > > > be
>> > > > > > > > > >> returned
>> > > > > > > > > >> > > > with
>> > > > > > > > > >> > > > >> InvalidRequestError
>> > > > > > > > > >> > > > >>
>> > > > > > > > > >> > > > >> 44. The order matters. The desired leader
>> entries
>> > > > match
>> > > > > > > with
>> > > > > > > > > the
>> > > > > > > > > >> > topic
>> > > > > > > > > >> > > > >> partition list by the index.
>> > > > > > > > > >> > > > >>
>> > > > > > > > > >> > > > >> 45. Thanks! Updated.
>> > > > > > > > > >> > > > >>
>> > > > > > > > > >> > > > >> 46. Good advice! Updated.
>> > > > > > > > > >> > > > >>
>> > > > > > > > > >> > > > >> 47.1, updated the comment. Basically it will
>> > elect
>> > > > the
>> > > > > > > > replica
>> > > > > > > > > in
>> > > > > > > > > >> > the
>> > > > > > > > > >> > > > >> desiredLeader field to be the leader
>> > > > > > > > > >> > > > >>
>> > > > > > > > > >> > > > >> 47.2 We can let the admin client do the
>> > conversion.
>> > > > > Using
>> > > > > > > the
>> > > > > > > > > >> > > > desiredLeader
>> > > > > > > > > >> > > > >> field in the json format seems easier for
>> users.
>> > > > > > > > > >> > > > >>
>> > > > > > > > > >> > > > >> 48. Once the MV version is downgraded, all the
>> > ELR
>> > > > > > related
>> > > > > > > > > fields
>> > > > > > > > > >> > will
>> > > > > > > > > >> > > > be
>> > > > > > > > > >> > > > >> removed on the next partition change. The
>> > > controller
>> > > > > will
>> > > > > > > > also
>> > > > > > > > > >> > ignore
>> > > > > > > > > >> > > > the
>> > > > > > > > > >> > > > >> ELR fields. Updated the KIP.
>> > > > > > > > > >> > > > >>
>> > > > > > > > > >> > > > >> 49. Yes, it would be deprecated/removed.
>> > > > > > > > > >> > > > >>
>> > > > > > > > > >> > > > >>
>> > > > > > > > > >> > > > >> On Mon, Sep 25, 2023 at 3:49 PM Jun Rao
>> > > > > > > > > <jun@confluent.io.invalid
>> > > > > > > > > >> >
>> > > > > > > > > >> > > > wrote:
>> > > > > > > > > >> > > > >>
>> > > > > > > > > >> > > > >> > Hi, Calvin,
>> > > > > > > > > >> > > > >> >
>> > > > > > > > > >> > > > >> > Thanks for the updated KIP. Made another
>> pass.
>> > A
>> > > > few
>> > > > > > more
>> > > > > > > > > >> comments
>> > > > > > > > > >> > > > below.
>> > > > > > > > > >> > > > >> >
>> > > > > > > > > >> > > > >> > 40. unclean.leader.election.enable.false ->
>> > > > > > > > > >> > > > >> > unclean.recovery.strategy.Balanced: The
>> > Balanced
>> > > > mode
>> > > > > > > could
>> > > > > > > > > >> still
>> > > > > > > > > >> > > > lead to
>> > > > > > > > > >> > > > >> > data loss. So, I am wondering if
>> > > > > > > > > >> > > unclean.leader.election.enable.false
>> > > > > > > > > >> > > > >> > should map to None?
>> > > > > > > > > >> > > > >> >
>> > > > > > > > > >> > > > >> > 41. unclean.recovery.manager.enabled: I am
>> not
>> > > sure
>> > > > > why
>> > > > > > > we
>> > > > > > > > > >> > introduce
>> > > > > > > > > >> > > > this
>> > > > > > > > > >> > > > >> > additional config. Is it the same as
>> > > > > > > > > >> > unclean.recovery.strategy=None?
>> > > > > > > > > >> > > > >> >
>> > > > > > > > > >> > > > >> > 42.
>> > > > DescribeTopicResponse.TopicAuthorizedOperations:
>> > > > > > > Should
>> > > > > > > > > >> this
>> > > > > > > > > >> > be
>> > > > > > > > > >> > > at
>> > > > > > > > > >> > > > >> the
>> > > > > > > > > >> > > > >> > topic level?
>> > > > > > > > > >> > > > >> >
>> > > > > > > > > >> > > > >> > 43. "Limit: 20 topics max per request":
>> Could
>> > we
>> > > > > > describe
>> > > > > > > > > what
>> > > > > > > > > >> > > > happens if
>> > > > > > > > > >> > > > >> > the request includes more than 20 topics?
>> > > > > > > > > >> > > > >> >
>> > > > > > > > > >> > > > >> > 44. ElectLeadersRequest.DesiredLeaders:
>> Could
>> > we
>> > > > > > describe
>> > > > > > > > > >> whether
>> > > > > > > > > >> > > the
>> > > > > > > > > >> > > > >> > ordering matters?
>> > > > > > > > > >> > > > >> >
>> > > > > > > > > >> > > > >> > 45. GetReplicaLogInfo.TopicPartitions:
>> "about":
>> > > > "The
>> > > > > > > topic
>> > > > > > > > > >> > > partitions
>> > > > > > > > > >> > > > to
>> > > > > > > > > >> > > > >> > elect leaders.": The description in "about"
>> is
>> > > > > > incorrect.
>> > > > > > > > > >> > > > >> >
>> > > > > > > > > >> > > > >> > 46. GetReplicaLogInfoResponse: Should we
>> nest
>> > > > > > partitions
>> > > > > > > > > under
>> > > > > > > > > >> > > > topicId to
>> > > > > > > > > >> > > > >> > be consistent with other types of responses?
>> > > > > > > > > >> > > > >> >
>> > > > > > > > > >> > > > >> > 47. kafka-leader-election.sh:
>> > > > > > > > > >> > > > >> > 47.1 Could we explain DESIGNATION?
>> > > > > > > > > >> > > > >> > 47.2 desiredLeader: Should it be a list to
>> > match
>> > > > the
>> > > > > > > field
>> > > > > > > > in
>> > > > > > > > > >> > > > >> > ElectLeadersRequest?
>> > > > > > > > > >> > > > >> >
>> > > > > > > > > >> > > > >> > 48. We could add a section on downgrade?
>> > > > > > > > > >> > > > >> >
>> > > > > > > > > >> > > > >> > 49. LastKnownLeader: This seems only needed
>> in
>> > > the
>> > > > > > first
>> > > > > > > > > phase
>> > > > > > > > > >> of
>> > > > > > > > > >> > > > >> > delivering ELR. Will it be removed when the
>> > > > complete
>> > > > > > KIP
>> > > > > > > is
>> > > > > > > > > >> > > delivered?
>> > > > > > > > > >> > > > >> >
>> > > > > > > > > >> > > > >> > Thanks,
>> > > > > > > > > >> > > > >> >
>> > > > > > > > > >> > > > >> > Jun
>> > > > > > > > > >> > > > >> >
>> > > > > > > > > >> > > > >> > On Tue, Sep 19, 2023 at 1:30 PM Colin
>> McCabe <
>> > > > > > > > > >> cmccabe@apache.org>
>> > > > > > > > > >> > > > wrote:
>> > > > > > > > > >> > > > >> >
>> > > > > > > > > >> > > > >> > > Hi Calvin,
>> > > > > > > > > >> > > > >> > >
>> > > > > > > > > >> > > > >> > > Thanks for the explanations. I like the
>> idea
>> > of
>> > > > > using
>> > > > > > > > none,
>> > > > > > > > > >> > > > balanced,
>> > > > > > > > > >> > > > >> > > aggressive. We also had an offline
>> discussion
>> > > > about
>> > > > > > why
>> > > > > > > > it
>> > > > > > > > > is
>> > > > > > > > > >> > good
>> > > > > > > > > >> > > > to
>> > > > > > > > > >> > > > >> > use a
>> > > > > > > > > >> > > > >> > > new config key (basically, so that we can
>> > > > deprecate
>> > > > > > the
>> > > > > > > > old
>> > > > > > > > > >> one
>> > > > > > > > > >> > > > which
>> > > > > > > > > >> > > > >> had
>> > > > > > > > > >> > > > >> > > only false/true values in 4.0) With these
>> > > > changes,
>> > > > > I
>> > > > > > am
>> > > > > > > > +1.
>> > > > > > > > > >> > > > >> > >
>> > > > > > > > > >> > > > >> > > best,
>> > > > > > > > > >> > > > >> > > Colin
>> > > > > > > > > >> > > > >> > >
>> > > > > > > > > >> > > > >> > > On Mon, Sep 18, 2023, at 15:54, Calvin Liu
>> > > wrote:
>> > > > > > > > > >> > > > >> > > > Hi Colin,
>> > > > > > > > > >> > > > >> > > > Also, can we deprecate
>> > > > > > unclean.leader.election.enable
>> > > > > > > > in
>> > > > > > > > > >> 4.0?
>> > > > > > > > > >> > > > Before
>> > > > > > > > > >> > > > >> > > that,
>> > > > > > > > > >> > > > >> > > > we can have both the config
>> > > > > > unclean.recovery.strategy
>> > > > > > > > and
>> > > > > > > > > >> > > > >> > > > unclean.leader.election.enable
>> > > > > > > > > >> > > > >> > > > and using the unclean.recovery.Enabled
>> to
>> > > > > determine
>> > > > > > > > which
>> > > > > > > > > >> > config
>> > > > > > > > > >> > > > to
>> > > > > > > > > >> > > > >> use
>> > > > > > > > > >> > > > >> > > > during the unclean leader election.
>> > > > > > > > > >> > > > >> > > >
>> > > > > > > > > >> > > > >> > > > On Mon, Sep 18, 2023 at 3:51 PM Calvin
>> Liu
>> > <
>> > > > > > > > > >> > caliu@confluent.io>
>> > > > > > > > > >> > > > >> wrote:
>> > > > > > > > > >> > > > >> > > >
>> > > > > > > > > >> > > > >> > > >> Hi Colin,
>> > > > > > > > > >> > > > >> > > >> For the unclean.recovery.strategy
>> config
>> > > name,
>> > > > > how
>> > > > > > > > about
>> > > > > > > > > >> we
>> > > > > > > > > >> > use
>> > > > > > > > > >> > > > the
>> > > > > > > > > >> > > > >> > > >> following
>> > > > > > > > > >> > > > >> > > >> None. It basically means no unclean
>> > recovery
>> > > > > will
>> > > > > > be
>> > > > > > > > > >> > performed.
>> > > > > > > > > >> > > > >> > > >> Aggressive. It means availability goes
>> > > first.
>> > > > > > > Whenever
>> > > > > > > > > the
>> > > > > > > > > >> > > > partition
>> > > > > > > > > >> > > > >> > > can't
>> > > > > > > > > >> > > > >> > > >> elect a durable replica, the controller
>> > will
>> > > > try
>> > > > > > the
>> > > > > > > > > >> unclean
>> > > > > > > > > >> > > > >> recovery.
>> > > > > > > > > >> > > > >> > > >> Balanced. It is the balance point of
>> the
>> > > > > > > availability
>> > > > > > > > > >> > > > >> > first(Aggressive)
>> > > > > > > > > >> > > > >> > > >> and least availability(None). The
>> > controller
>> > > > > > > performs
>> > > > > > > > > >> unclean
>> > > > > > > > > >> > > > >> recovery
>> > > > > > > > > >> > > > >> > > when
>> > > > > > > > > >> > > > >> > > >> both ISR and ELR are empty.
>> > > > > > > > > >> > > > >> > > >>
>> > > > > > > > > >> > > > >> > > >>
>> > > > > > > > > >> > > > >> > > >> On Fri, Sep 15, 2023 at 11:42 AM Calvin
>> > Liu
>> > > <
>> > > > > > > > > >> > > caliu@confluent.io>
>> > > > > > > > > >> > > > >> > wrote:
>> > > > > > > > > >> > > > >> > > >>
>> > > > > > > > > >> > > > >> > > >>> Hi Colin,
>> > > > > > > > > >> > > > >> > > >>>
>> > > > > > > > > >> > > > >> > > >>> > So, the proposal is that if someone
>> > sets
>> > > > > > > > > >> > > > >> > > "unclean.leader.election.enable
>> > > > > > > > > >> > > > >> > > >>> = true"...
>> > > > > > > > > >> > > > >> > > >>>
>> > > > > > > > > >> > > > >> > > >>>
>> > > > > > > > > >> > > > >> > > >>> The idea is to use one of the
>> > > > > > > > > >> unclean.leader.election.enable
>> > > > > > > > > >> > > and
>> > > > > > > > > >> > > > >> > > >>> unclean.recovery.strategy based on the
>> > > > > > > > > >> > > > unclean.recovery.Enabled. A
>> > > > > > > > > >> > > > >> > > possible
>> > > > > > > > > >> > > > >> > > >>> version can be
>> > > > > > > > > >> > > > >> > > >>>
>> > > > > > > > > >> > > > >> > > >>> If unclean.recovery.Enabled:
>> > > > > > > > > >> > > > >> > > >>>
>> > > > > > > > > >> > > > >> > > >>> {
>> > > > > > > > > >> > > > >> > > >>>
>> > > > > > > > > >> > > > >> > > >>> Check unclean.recovery.strategy. If
>> set,
>> > > use
>> > > > > it.
>> > > > > > > > > >> Otherwise,
>> > > > > > > > > >> > > > check
>> > > > > > > > > >> > > > >> > > >>> unclean.leader.election.enable and
>> > > translate
>> > > > it
>> > > > > > to
>> > > > > > > > > >> > > > >> > > >>> unclean.recovery.strategy.
>> > > > > > > > > >> > > > >> > > >>>
>> > > > > > > > > >> > > > >> > > >>> } else {
>> > > > > > > > > >> > > > >> > > >>>
>> > > > > > > > > >> > > > >> > > >>> Use unclean.leader.election.enable
>> > > > > > > > > >> > > > >> > > >>>
>> > > > > > > > > >> > > > >> > > >>> }
>> > > > > > > > > >> > > > >> > > >>>
>> > > > > > > > > >> > > > >> > > >>>
>> > > > > > > > > >> > > > >> > > >>> —--------
>> > > > > > > > > >> > > > >> > > >>>
>> > > > > > > > > >> > > > >> > > >>> >The configuration key should be
>> > > > > > > > > >> > > > >> "unclean.recovery.manager.enabled",
>> > > > > > > > > >> > > > >> > > >>> right?
>> > > > > > > > > >> > > > >> > > >>>
>> > > > > > > > > >> > > > >> > > >>>
>> > > > > > > > > >> > > > >> > > >>> I think we have two ways of choosing a
>> > > leader
>> > > > > > > > > uncleanly,
>> > > > > > > > > >> > > unclean
>> > > > > > > > > >> > > > >> > leader
>> > > > > > > > > >> > > > >> > > >>> election and unclean recovery(log
>> > > inspection)
>> > > > > and
>> > > > > > > we
>> > > > > > > > > try
>> > > > > > > > > >> to
>> > > > > > > > > >> > > > switch
>> > > > > > > > > >> > > > >> > > between
>> > > > > > > > > >> > > > >> > > >>> them.
>> > > > > > > > > >> > > > >> > > >>>
>> > > > > > > > > >> > > > >> > > >>> Do you mean we want to develop two
>> ways
>> > of
>> > > > > > > performing
>> > > > > > > > > the
>> > > > > > > > > >> > > > unclean
>> > > > > > > > > >> > > > >> > > >>> recovery and one of them is using
>> > “unclean
>> > > > > > recovery
>> > > > > > > > > >> > manager”?
>> > > > > > > > > >> > > I
>> > > > > > > > > >> > > > >> guess
>> > > > > > > > > >> > > > >> > > we
>> > > > > > > > > >> > > > >> > > >>> haven’t discussed the second way.
>> > > > > > > > > >> > > > >> > > >>>
>> > > > > > > > > >> > > > >> > > >>>
>> > > > > > > > > >> > > > >> > > >>> —-------
>> > > > > > > > > >> > > > >> > > >>>
>> > > > > > > > > >> > > > >> > > >>> >How do these 4 levels of overrides
>> > > interact
>> > > > > with
>> > > > > > > > your
>> > > > > > > > > >> new
>> > > > > > > > > >> > > > >> > > >>> configurations?
>> > > > > > > > > >> > > > >> > > >>>
>> > > > > > > > > >> > > > >> > > >>>
>> > > > > > > > > >> > > > >> > > >>> I do notice in the Kraft controller
>> code,
>> > > the
>> > > > > > > method
>> > > > > > > > to
>> > > > > > > > > >> > check
>> > > > > > > > > >> > > > >> whether
>> > > > > > > > > >> > > > >> > > >>> perform unclean leader election is
>> hard
>> > > coded
>> > > > > to
>> > > > > > > > false
>> > > > > > > > > >> since
>> > > > > > > > > >> > > > >> > > >>>
>> > 2021(uncleanLeaderElectionEnabledForTopic).
>> > > > > Isn’t
>> > > > > > > it
>> > > > > > > > a
>> > > > > > > > > >> good
>> > > > > > > > > >> > > > chance
>> > > > > > > > > >> > > > >> to
>> > > > > > > > > >> > > > >> > > >>> completely deprecate the
>> > > > > > > > > unclean.leader.election.enable?
>> > > > > > > > > >> We
>> > > > > > > > > >> > > > don’t
>> > > > > > > > > >> > > > >> > even
>> > > > > > > > > >> > > > >> > > have
>> > > > > > > > > >> > > > >> > > >>> to worry about the config conversion.
>> > > > > > > > > >> > > > >> > > >>>
>> > > > > > > > > >> > > > >> > > >>> On the other hand, whatever the
>> override
>> > > is,
>> > > > as
>> > > > > > > long
>> > > > > > > > as
>> > > > > > > > > >> the
>> > > > > > > > > >> > > > >> > controller
>> > > > > > > > > >> > > > >> > > >>> can have the final effective
>> > > > > > > > > >> unclean.leader.election.enable,
>> > > > > > > > > >> > > the
>> > > > > > > > > >> > > > >> > topic
>> > > > > > > > > >> > > > >> > > >>> level config
>> unclean.recovery.strategy,
>> > the
>> > > > > > cluster
>> > > > > > > > > level
>> > > > > > > > > >> > > config
>> > > > > > > > > >> > > > >> > > >>> unclean.recovery.Enabled, the
>> controller
>> > > can
>> > > > > > > > calculate
>> > > > > > > > > >> the
>> > > > > > > > > >> > > > correct
>> > > > > > > > > >> > > > >> > > methods
>> > > > > > > > > >> > > > >> > > >>> to use right?
>> > > > > > > > > >> > > > >> > > >>>
>> > > > > > > > > >> > > > >> > > >>>
>> > > > > > > > > >> > > > >> > > >>> On Fri, Sep 15, 2023 at 10:02 AM Colin
>> > > > McCabe <
>> > > > > > > > > >> > > > cmccabe@apache.org>
>> > > > > > > > > >> > > > >> > > wrote:
>> > > > > > > > > >> > > > >> > > >>>
>> > > > > > > > > >> > > > >> > > >>>> On Thu, Sep 14, 2023, at 22:23,
>> Calvin
>> > Liu
>> > > > > > wrote:
>> > > > > > > > > >> > > > >> > > >>>> > Hi Colin
>> > > > > > > > > >> > > > >> > > >>>> > 1. I think using the new config
>> name
>> > is
>> > > > more
>> > > > > > > > clear.
>> > > > > > > > > >> > > > >> > > >>>> >        a. The unclean leader
>> election
>> > is
>> > > > > > > actually
>> > > > > > > > > >> removed
>> > > > > > > > > >> > > if
>> > > > > > > > > >> > > > >> > unclean
>> > > > > > > > > >> > > > >> > > >>>> > recovery is in use.
>> > > > > > > > > >> > > > >> > > >>>> >        b. Using multiple values in
>> > > > > > > > > >> > > > >> unclean.leader.election.enable
>> > > > > > > > > >> > > > >> > is
>> > > > > > > > > >> > > > >> > > >>>> > confusing and it will be more
>> > confusing
>> > > > > after
>> > > > > > > > people
>> > > > > > > > > >> > forget
>> > > > > > > > > >> > > > >> about
>> > > > > > > > > >> > > > >> > > this
>> > > > > > > > > >> > > > >> > > >>>> > discussion.
>> > > > > > > > > >> > > > >> > > >>>>
>> > > > > > > > > >> > > > >> > > >>>> Hi Calvin,
>> > > > > > > > > >> > > > >> > > >>>>
>> > > > > > > > > >> > > > >> > > >>>> So, the proposal is that if someone
>> sets
>> > > > > > > > > >> > > > >> > > "unclean.leader.election.enable
>> > > > > > > > > >> > > > >> > > >>>> = true" but then sets one of your new
>> > > > > > > > configurations,
>> > > > > > > > > >> the
>> > > > > > > > > >> > > > value of
>> > > > > > > > > >> > > > >> > > >>>> unclean.leader.election.enable is
>> > ignored?
>> > > > > That
>> > > > > > > > seems
>> > > > > > > > > >> less
>> > > > > > > > > >> > > > clear
>> > > > > > > > > >> > > > >> to
>> > > > > > > > > >> > > > >> > > me, not
>> > > > > > > > > >> > > > >> > > >>>> more. Just in general, having
>> multiple
>> > > > > > > configuration
>> > > > > > > > > >> keys
>> > > > > > > > > >> > to
>> > > > > > > > > >> > > > >> control
>> > > > > > > > > >> > > > >> > > the
>> > > > > > > > > >> > > > >> > > >>>> same thing confuses users. Basically,
>> > they
>> > > > are
>> > > > > > > > sitting
>> > > > > > > > > >> at a
>> > > > > > > > > >> > > > giant
>> > > > > > > > > >> > > > >> > > control
>> > > > > > > > > >> > > > >> > > >>>> panel, and some of the levers do
>> > nothing.
>> > > > > > > > > >> > > > >> > > >>>>
>> > > > > > > > > >> > > > >> > > >>>> > 2. Sorry I forgot to mention in the
>> > > > response
>> > > > > > > that
>> > > > > > > > I
>> > > > > > > > > >> did
>> > > > > > > > > >> > add
>> > > > > > > > > >> > > > the
>> > > > > > > > > >> > > > >> > > >>>> > unclean.recovery.Enabled flag.
>> > > > > > > > > >> > > > >> > > >>>>
>> > > > > > > > > >> > > > >> > > >>>> The configuration key should be
>> > > > > > > > > >> > > > >> "unclean.recovery.manager.enabled",
>> > > > > > > > > >> > > > >> > > >>>> right? Becuase we can do "unclean
>> > > recovery"
>> > > > > > > without
>> > > > > > > > > the
>> > > > > > > > > >> > > > manager.
>> > > > > > > > > >> > > > >> > > Disabling
>> > > > > > > > > >> > > > >> > > >>>> the manager just means we use a
>> > different
>> > > > > > > mechanism
>> > > > > > > > > for
>> > > > > > > > > >> > > > recovery.
>> > > > > > > > > >> > > > >> > > >>>>
>> > > > > > > > > >> > > > >> > > >>>> >        c. Maybe I underestimated
>> the
>> > > > > challenge
>> > > > > > > of
>> > > > > > > > > >> > replacing
>> > > > > > > > > >> > > > the
>> > > > > > > > > >> > > > >> > > >>>> config. Any
>> > > > > > > > > >> > > > >> > > >>>> > implementation problems ahead?
>> > > > > > > > > >> > > > >> > > >>>>
>> > > > > > > > > >> > > > >> > > >>>> There are four levels of overrides
>> for
>> > > > > > > > > >> > > > >> > unclean.leader.election.enable.
>> > > > > > > > > >> > > > >> > > >>>>
>> > > > > > > > > >> > > > >> > > >>>> 1. static configuration for node.
>> > > > > > > > > >> > > > >> > > >>>>     This goes in the configuration
>> file,
>> > > > > > typically
>> > > > > > > > > named
>> > > > > > > > > >> > > > >> > > >>>> server.properties
>> > > > > > > > > >> > > > >> > > >>>>
>> > > > > > > > > >> > > > >> > > >>>> 2. dynamic configuration for node
>> > default
>> > > > > > > > > >> > > > >> > > >>>>   ConfigResource(type=BROKER,
>> name="")
>> > > > > > > > > >> > > > >> > > >>>>
>> > > > > > > > > >> > > > >> > > >>>> 3. dynamic configuration for node
>> > > > > > > > > >> > > > >> > > >>>>   ConfigResource(type=BROKER,
>> > > > name=<controller
>> > > > > > > id>)
>> > > > > > > > > >> > > > >> > > >>>>
>> > > > > > > > > >> > > > >> > > >>>> 4. dynamic configuration for topic
>> > > > > > > > > >> > > > >> > > >>>>   ConfigResource(type=TOPIC,
>> > > > > name=<topic-name>)
>> > > > > > > > > >> > > > >> > > >>>>
>> > > > > > > > > >> > > > >> > > >>>> How do these 4 levels of overrides
>> > > interact
>> > > > > with
>> > > > > > > > your
>> > > > > > > > > >> new
>> > > > > > > > > >> > > > >> > > >>>> configurations? If the new
>> > configurations
>> > > > > > dominate
>> > > > > > > > > over
>> > > > > > > > > >> the
>> > > > > > > > > >> > > old
>> > > > > > > > > >> > > > >> > ones,
>> > > > > > > > > >> > > > >> > > it
>> > > > > > > > > >> > > > >> > > >>>> seems like this will get a lot more
>> > > > confusing
>> > > > > to
>> > > > > > > > > >> implement
>> > > > > > > > > >> > > (and
>> > > > > > > > > >> > > > >> also
>> > > > > > > > > >> > > > >> > > to
>> > > > > > > > > >> > > > >> > > >>>> use.)
>> > > > > > > > > >> > > > >> > > >>>>
>> > > > > > > > > >> > > > >> > > >>>> Again, I'd recommend just adding some
>> > new
>> > > > > values
>> > > > > > > to
>> > > > > > > > > >> > > > >> > > >>>> unclean.leader.election.enable. It's
>> > > simple
>> > > > > and
>> > > > > > > will
>> > > > > > > > > >> > prevent
>> > > > > > > > > >> > > > user
>> > > > > > > > > >> > > > >> > > confusion
>> > > > > > > > > >> > > > >> > > >>>> (as well as developer confusion.)
>> > > > > > > > > >> > > > >> > > >>>>
>> > > > > > > > > >> > > > >> > > >>>> best,
>> > > > > > > > > >> > > > >> > > >>>> Colin
>> > > > > > > > > >> > > > >> > > >>>>
>> > > > > > > > > >> > > > >> > > >>>>
>> > > > > > > > > >> > > > >> > > >>>> > 3. About the admin client, I
>> > mentioned 3
>> > > > > > changes
>> > > > > > > > in
>> > > > > > > > > >> the
>> > > > > > > > > >> > > > client.
>> > > > > > > > > >> > > > >> > > >>>> Anything
>> > > > > > > > > >> > > > >> > > >>>> > else I missed in the KIP?
>> > > > > > > > > >> > > > >> > > >>>> >       a. The client will switch to
>> > using
>> > > > the
>> > > > > > new
>> > > > > > > > RPC
>> > > > > > > > > >> > > instead
>> > > > > > > > > >> > > > of
>> > > > > > > > > >> > > > >> > > >>>> > MetadataRequest for the topics.
>> > > > > > > > > >> > > > >> > > >>>> >       b. The TopicPartitionInfo
>> used
>> > in
>> > > > > > > > > >> TopicDescription
>> > > > > > > > > >> > > > needs
>> > > > > > > > > >> > > > >> to
>> > > > > > > > > >> > > > >> > > add
>> > > > > > > > > >> > > > >> > > >>>> new
>> > > > > > > > > >> > > > >> > > >>>> > fields related to the ELR.
>> > > > > > > > > >> > > > >> > > >>>> >       c. The outputs will add the
>> ELR
>> > > > > related
>> > > > > > > > > fields.
>> > > > > > > > > >> > > > >> > > >>>> >
>> > > > > > > > > >> > > > >> > > >>>> > On Thu, Sep 14, 2023 at 9:19 PM
>> Colin
>> > > > > McCabe <
>> > > > > > > > > >> > > > >> cmccabe@apache.org>
>> > > > > > > > > >> > > > >> > > >>>> wrote:
>> > > > > > > > > >> > > > >> > > >>>> >
>> > > > > > > > > >> > > > >> > > >>>> >> Hi Calvin,
>> > > > > > > > > >> > > > >> > > >>>> >>
>> > > > > > > > > >> > > > >> > > >>>> >> Thanks for the changes.
>> > > > > > > > > >> > > > >> > > >>>> >>
>> > > > > > > > > >> > > > >> > > >>>> >> 1. Earlier I commented that
>> creating
>> > > > > > > > > >> > > > >> "unclean.recovery.strategy "
>> > > > > > > > > >> > > > >> > > is
>> > > > > > > > > >> > > > >> > > >>>> not
>> > > > > > > > > >> > > > >> > > >>>> >> necessary, and we can just reuse
>> the
>> > > > > existing
>> > > > > > > > > >> > > > >> > > >>>> >> "unclean.leader.election.enable"
>> > > > > > configuration
>> > > > > > > > key.
>> > > > > > > > > >> > Let's
>> > > > > > > > > >> > > > >> discuss
>> > > > > > > > > >> > > > >> > > >>>> that.
>> > > > > > > > > >> > > > >> > > >>>> >>
>> > > > > > > > > >> > > > >> > > >>>> >> 2.I also don't understand why you
>> > > didn't
>> > > > > add
>> > > > > > a
>> > > > > > > > > >> > > > configuration to
>> > > > > > > > > >> > > > >> > > >>>> enable or
>> > > > > > > > > >> > > > >> > > >>>> >> disable the Unclean Recovery
>> Manager.
>> > > > This
>> > > > > > > seems
>> > > > > > > > > >> like a
>> > > > > > > > > >> > > very
>> > > > > > > > > >> > > > >> > simple
>> > > > > > > > > >> > > > >> > > >>>> way to
>> > > > > > > > > >> > > > >> > > >>>> >> handle the staging issue which we
>> > > > > discussed.
>> > > > > > > The
>> > > > > > > > > URM
>> > > > > > > > > >> can
>> > > > > > > > > >> > > > just
>> > > > > > > > > >> > > > >> be
>> > > > > > > > > >> > > > >> > > >>>> turned off
>> > > > > > > > > >> > > > >> > > >>>> >> until it is production ready.
>> Let's
>> > > > discuss
>> > > > > > > this.
>> > > > > > > > > >> > > > >> > > >>>> >>
>> > > > > > > > > >> > > > >> > > >>>> >> 3. You still need to describe the
>> > > changes
>> > > > > to
>> > > > > > > > > >> AdminClient
>> > > > > > > > > >> > > > that
>> > > > > > > > > >> > > > >> are
>> > > > > > > > > >> > > > >> > > >>>> needed
>> > > > > > > > > >> > > > >> > > >>>> >> to use DescribeTopicRequest.
>> > > > > > > > > >> > > > >> > > >>>> >>
>> > > > > > > > > >> > > > >> > > >>>> >> Keep at it. It's looking better.
>> :)
>> > > > > > > > > >> > > > >> > > >>>> >>
>> > > > > > > > > >> > > > >> > > >>>> >> best,
>> > > > > > > > > >> > > > >> > > >>>> >> Colin
>> > > > > > > > > >> > > > >> > > >>>> >>
>> > > > > > > > > >> > > > >> > > >>>> >>
>> > > > > > > > > >> > > > >> > > >>>> >> On Thu, Sep 14, 2023, at 11:03,
>> > Calvin
>> > > > Liu
>> > > > > > > wrote:
>> > > > > > > > > >> > > > >> > > >>>> >> > Hi Colin
>> > > > > > > > > >> > > > >> > > >>>> >> > Thanks for the comments!
>> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> > I did the following changes
>> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >    1.
>> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >    Simplified the API spec
>> section
>> > to
>> > > > > only
>> > > > > > > > > include
>> > > > > > > > > >> the
>> > > > > > > > > >> > > > diff.
>> > > > > > > > > >> > > > >> > > >>>> >> >    2.
>> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >    Reordered the HWM requirement
>> > > > section.
>> > > > > > > > > >> > > > >> > > >>>> >> >    3.
>> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >    Removed the URM
>> implementation
>> > > > details
>> > > > > > to
>> > > > > > > > keep
>> > > > > > > > > >> the
>> > > > > > > > > >> > > > >> necessary
>> > > > > > > > > >> > > > >> > > >>>> >> >    characteristics to perform
>> the
>> > > > unclean
>> > > > > > > > > recovery.
>> > > > > > > > > >> > > > >> > > >>>> >> >    1.
>> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >       When to perform the
>> unclean
>> > > > > recovery
>> > > > > > > > > >> > > > >> > > >>>> >> >       2.
>> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >       Under different config,
>> how
>> > the
>> > > > > > unclean
>> > > > > > > > > >> recovery
>> > > > > > > > > >> > > > finds
>> > > > > > > > > >> > > > >> > the
>> > > > > > > > > >> > > > >> > > >>>> leader.
>> > > > > > > > > >> > > > >> > > >>>> >> >       3.
>> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >       How the config
>> > > > > > > > > unclean.leader.election.enable
>> > > > > > > > > >> > and
>> > > > > > > > > >> > > > >> > > >>>> >> >       unclean.recovery.strategy
>> are
>> > > > > > converted
>> > > > > > > > > when
>> > > > > > > > > >> > users
>> > > > > > > > > >> > > > >> > > >>>> enable/disable
>> > > > > > > > > >> > > > >> > > >>>> >> the
>> > > > > > > > > >> > > > >> > > >>>> >> >       unclean recovery.
>> > > > > > > > > >> > > > >> > > >>>> >> >       4.
>> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >    More details about how we
>> change
>> > > > admin
>> > > > > > > > client.
>> > > > > > > > > >> > > > >> > > >>>> >> >    5.
>> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >    API limits on the
>> > > > > > GetReplicaLogInfoRequest
>> > > > > > > > and
>> > > > > > > > > >> > > > >> > > >>>> DescribeTopicRequest.
>> > > > > > > > > >> > > > >> > > >>>> >> >    6.
>> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >    Two metrics added
>> > > > > > > > > >> > > > >> > > >>>> >> >    1.
>> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > > > > > > > >> > > >
>> > Kafka.controller.global_under_min_isr_partition_count
>> > > > > > > > > >> > > > >> > > >>>> >> >       2.
>> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > > > > > > > >>  kafka.controller.unclean_recovery_finished_count
>> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> > On Wed, Sep 13, 2023 at 10:46 AM
>> > > Colin
>> > > > > > > McCabe <
>> > > > > > > > > >> > > > >> > > cmccabe@apache.org>
>> > > > > > > > > >> > > > >> > > >>>> >> wrote:
>> > > > > > > > > >> > > > >> > > >>>> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >> On Tue, Sep 12, 2023, at 17:21,
>> > > Calvin
>> > > > > Liu
>> > > > > > > > > wrote:
>> > > > > > > > > >> > > > >> > > >>>> >> >> > Hi Colin
>> > > > > > > > > >> > > > >> > > >>>> >> >> > Thanks for the comments!
>> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> Hi Calvin,
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> Thanks again for the KIP.
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> One meta-comment: it's usually
>> > > better
>> > > > to
>> > > > > > > just
>> > > > > > > > > do a
>> > > > > > > > > >> > diff
>> > > > > > > > > >> > > > on a
>> > > > > > > > > >> > > > >> > > >>>> message
>> > > > > > > > > >> > > > >> > > >>>> >> spec
>> > > > > > > > > >> > > > >> > > >>>> >> >> file or java file if you're
>> > > including
>> > > > > > > changes
>> > > > > > > > to
>> > > > > > > > > >> it
>> > > > > > > > > >> > in
>> > > > > > > > > >> > > > the
>> > > > > > > > > >> > > > >> > KIP.
>> > > > > > > > > >> > > > >> > > >>>> This is
>> > > > > > > > > >> > > > >> > > >>>> >> >> easier to read than looking for
>> > "new
>> > > > > > fields
>> > > > > > > > > begin"
>> > > > > > > > > >> > etc.
>> > > > > > > > > >> > > > in
>> > > > > > > > > >> > > > >> the
>> > > > > > > > > >> > > > >> > > >>>> text, and
>> > > > > > > > > >> > > > >> > > >>>> >> >> gracefully handles the case
>> where
>> > > > > existing
>> > > > > > > > > fields
>> > > > > > > > > >> > were
>> > > > > > > > > >> > > > >> > changed.
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> > Rewrite the Additional High
>> > > > Watermark
>> > > > > > > > > >> advancement
>> > > > > > > > > >> > > > >> > requirement
>> > > > > > > > > >> > > > >> > > >>>> >> >> > There was feedback on this
>> > section
>> > > > > that
>> > > > > > > some
>> > > > > > > > > >> > readers
>> > > > > > > > > >> > > > may
>> > > > > > > > > >> > > > >> not
>> > > > > > > > > >> > > > >> > > be
>> > > > > > > > > >> > > > >> > > >>>> >> familiar
>> > > > > > > > > >> > > > >> > > >>>> >> >> > with HWM and Ack=0,1,all
>> > requests.
>> > > > > This
>> > > > > > > can
>> > > > > > > > > help
>> > > > > > > > > >> > them
>> > > > > > > > > >> > > > >> > > understand
>> > > > > > > > > >> > > > >> > > >>>> the
>> > > > > > > > > >> > > > >> > > >>>> >> >> > proposal. I will rewrite this
>> > part
>> > > > for
>> > > > > > > more
>> > > > > > > > > >> > > > readability.
>> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> To be clear, I wasn't
>> suggesting
>> > > > > dropping
>> > > > > > > > either
>> > > > > > > > > >> > > > section. I
>> > > > > > > > > >> > > > >> > > agree
>> > > > > > > > > >> > > > >> > > >>>> that
>> > > > > > > > > >> > > > >> > > >>>> >> >> they add useful background. I
>> was
>> > > just
>> > > > > > > > > suggesting
>> > > > > > > > > >> > that
>> > > > > > > > > >> > > we
>> > > > > > > > > >> > > > >> > should
>> > > > > > > > > >> > > > >> > > >>>> discuss
>> > > > > > > > > >> > > > >> > > >>>> >> >> the "acks" setting AFTER
>> > discussing
>> > > > the
>> > > > > > new
>> > > > > > > > high
>> > > > > > > > > >> > > > watermark
>> > > > > > > > > >> > > > >> > > >>>> advancement
>> > > > > > > > > >> > > > >> > > >>>> >> >> conditions. We also should
>> discuss
>> > > > > acks=0.
>> > > > > > > > While
>> > > > > > > > > >> it
>> > > > > > > > > >> > > isn't
>> > > > > > > > > >> > > > >> > > >>>> conceptually
>> > > > > > > > > >> > > > >> > > >>>> >> much
>> > > > > > > > > >> > > > >> > > >>>> >> >> different than acks=1 here, its
>> > > > omission
>> > > > > > > from
>> > > > > > > > > this
>> > > > > > > > > >> > > > section
>> > > > > > > > > >> > > > >> is
>> > > > > > > > > >> > > > >> > > >>>> confusing.
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> > Unclean recovery
>> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >> > The plan is to replace the
>> > > > > > > > > >> > > > unclean.leader.election.enable
>> > > > > > > > > >> > > > >> > with
>> > > > > > > > > >> > > > >> > > >>>> >> >> > unclean.recovery.strategy. If
>> > the
>> > > > > > Unclean
>> > > > > > > > > >> Recovery
>> > > > > > > > > >> > is
>> > > > > > > > > >> > > > >> > enabled
>> > > > > > > > > >> > > > >> > > >>>> then it
>> > > > > > > > > >> > > > >> > > >>>> >> >> deals
>> > > > > > > > > >> > > > >> > > >>>> >> >> > with the three options in the
>> > > > > > > > > >> > > > unclean.recovery.strategy.
>> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >> > Let’s refine the Unclean
>> > Recovery.
>> > > > We
>> > > > > > have
>> > > > > > > > > >> already
>> > > > > > > > > >> > > > taken a
>> > > > > > > > > >> > > > >> > > lot of
>> > > > > > > > > >> > > > >> > > >>>> >> >> > suggestions and I hope to
>> > enhance
>> > > > the
>> > > > > > > > > >> durability of
>> > > > > > > > > >> > > > Kafka
>> > > > > > > > > >> > > > >> to
>> > > > > > > > > >> > > > >> > > the
>> > > > > > > > > >> > > > >> > > >>>> next
>> > > > > > > > > >> > > > >> > > >>>> >> >> level
>> > > > > > > > > >> > > > >> > > >>>> >> >> > with this KIP.
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> I am OK with doing the unclean
>> > > leader
>> > > > > > > recovery
>> > > > > > > > > >> > > > improvements
>> > > > > > > > > >> > > > >> in
>> > > > > > > > > >> > > > >> > > >>>> this KIP.
>> > > > > > > > > >> > > > >> > > >>>> >> >> However, I think we need to
>> really
>> > > > work
>> > > > > on
>> > > > > > > the
>> > > > > > > > > >> > > > configuration
>> > > > > > > > > >> > > > >> > > >>>> settings.
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> Configuration overrides are
>> often
>> > > > quite
>> > > > > > > messy.
>> > > > > > > > > For
>> > > > > > > > > >> > > > example,
>> > > > > > > > > >> > > > >> > the
>> > > > > > > > > >> > > > >> > > >>>> cases
>> > > > > > > > > >> > > > >> > > >>>> >> >> where we have log.roll.hours
>> and
>> > > > > > > > > >> log.roll.segment.ms
>> > > > > > > > > >> > ,
>> > > > > > > > > >> > > > the
>> > > > > > > > > >> > > > >> > user
>> > > > > > > > > >> > > > >> > > >>>> has to
>> > > > > > > > > >> > > > >> > > >>>> >> >> remember which one takes
>> > precedence,
>> > > > and
>> > > > > > it
>> > > > > > > is
>> > > > > > > > > not
>> > > > > > > > > >> > > > obvious.
>> > > > > > > > > >> > > > >> > So,
>> > > > > > > > > >> > > > >> > > >>>> rather
>> > > > > > > > > >> > > > >> > > >>>> >> than
>> > > > > > > > > >> > > > >> > > >>>> >> >> creating a new configuration,
>> why
>> > > not
>> > > > > add
>> > > > > > > > > >> additional
>> > > > > > > > > >> > > > values
>> > > > > > > > > >> > > > >> to
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> "unclean.leader.election.enable"?
>> > I
>> > > > > think
>> > > > > > > this
>> > > > > > > > > >> will
>> > > > > > > > > >> > be
>> > > > > > > > > >> > > > >> simpler
>> > > > > > > > > >> > > > >> > > for
>> > > > > > > > > >> > > > >> > > >>>> >> people
>> > > > > > > > > >> > > > >> > > >>>> >> >> to understand, and simpler in
>> the
>> > > code
>> > > > > as
>> > > > > > > > well.
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> What if we continued to use
>> > > > > > > > > >> > > > "unclean.leader.election.enable"
>> > > > > > > > > >> > > > >> > but
>> > > > > > > > > >> > > > >> > > >>>> >> extended
>> > > > > > > > > >> > > > >> > > >>>> >> >> it so that it took a string?
>> Then
>> > > the
>> > > > > > string
>> > > > > > > > > could
>> > > > > > > > > >> > have
>> > > > > > > > > >> > > > >> these
>> > > > > > > > > >> > > > >> > > >>>> values:
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> never
>> > > > > > > > > >> > > > >> > > >>>> >> >>     never automatically do an
>> > > unclean
>> > > > > > leader
>> > > > > > > > > >> election
>> > > > > > > > > >> > > > under
>> > > > > > > > > >> > > > >> > any
>> > > > > > > > > >> > > > >> > > >>>> >> conditions
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> false / default
>> > > > > > > > > >> > > > >> > > >>>> >> >>     only do an unclean leader
>> > > election
>> > > > > if
>> > > > > > > > there
>> > > > > > > > > >> may
>> > > > > > > > > >> > be
>> > > > > > > > > >> > > > >> > possible
>> > > > > > > > > >> > > > >> > > >>>> data
>> > > > > > > > > >> > > > >> > > >>>> >> loss
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> true / always
>> > > > > > > > > >> > > > >> > > >>>> >> >>     always do an unclean leader
>> > > > election
>> > > > > > if
>> > > > > > > we
>> > > > > > > > > >> can't
>> > > > > > > > > >> > > > >> > immediately
>> > > > > > > > > >> > > > >> > > >>>> elect a
>> > > > > > > > > >> > > > >> > > >>>> >> >> leader
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> It's a bit awkward that false
>> maps
>> > > to
>> > > > > > > default
>> > > > > > > > > >> rather
>> > > > > > > > > >> > > > than to
>> > > > > > > > > >> > > > >> > > >>>> never. But
>> > > > > > > > > >> > > > >> > > >>>> >> >> this awkwardness exists if we
>> use
>> > > two
>> > > > > > > > different
>> > > > > > > > > >> > > > >> configuration
>> > > > > > > > > >> > > > >> > > keys
>> > > > > > > > > >> > > > >> > > >>>> as
>> > > > > > > > > >> > > > >> > > >>>> >> well.
>> > > > > > > > > >> > > > >> > > >>>> >> >> The reason for the awkwardness
>> is
>> > > that
>> > > > > we
>> > > > > > > > simply
>> > > > > > > > > >> > don't
>> > > > > > > > > >> > > > want
>> > > > > > > > > >> > > > >> > most
>> > > > > > > > > >> > > > >> > > >>>> of the
>> > > > > > > > > >> > > > >> > > >>>> >> >> people currently setting
>> > > > > > > > > >> > > > >> unclean.leader.election.enable=false
>> > > > > > > > > >> > > > >> > to
>> > > > > > > > > >> > > > >> > > >>>> get the
>> > > > > > > > > >> > > > >> > > >>>> >> >> "never" behavior. We have to
>> bite
>> > > that
>> > > > > > > bullet.
>> > > > > > > > > >> Better
>> > > > > > > > > >> > > to
>> > > > > > > > > >> > > > be
>> > > > > > > > > >> > > > >> > > clear
>> > > > > > > > > >> > > > >> > > >>>> and
>> > > > > > > > > >> > > > >> > > >>>> >> >> explicit than hide it.
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> Another thing that's a bit
>> awkward
>> > > is
>> > > > > > having
>> > > > > > > > two
>> > > > > > > > > >> > > > different
>> > > > > > > > > >> > > > >> > ways
>> > > > > > > > > >> > > > >> > > to
>> > > > > > > > > >> > > > >> > > >>>> do
>> > > > > > > > > >> > > > >> > > >>>> >> >> unclean leader election
>> specified
>> > in
>> > > > the
>> > > > > > > KIP.
>> > > > > > > > > You
>> > > > > > > > > >> > > > descirbe
>> > > > > > > > > >> > > > >> two
>> > > > > > > > > >> > > > >> > > >>>> methods:
>> > > > > > > > > >> > > > >> > > >>>> >> the
>> > > > > > > > > >> > > > >> > > >>>> >> >> simple "choose the last leader"
>> > > > method,
>> > > > > > and
>> > > > > > > > the
>> > > > > > > > > >> > > "unclean
>> > > > > > > > > >> > > > >> > > recovery
>> > > > > > > > > >> > > > >> > > >>>> >> manager"
>> > > > > > > > > >> > > > >> > > >>>> >> >> method. I understand why you
>> did
>> > it
>> > > > this
>> > > > > > way
>> > > > > > > > --
>> > > > > > > > > >> > "choose
>> > > > > > > > > >> > > > the
>> > > > > > > > > >> > > > >> > last
>> > > > > > > > > >> > > > >> > > >>>> >> leader" is
>> > > > > > > > > >> > > > >> > > >>>> >> >> simple, and will help us
>> deliver
>> > an
>> > > > > > > > > implementation
>> > > > > > > > > >> > > > quickly,
>> > > > > > > > > >> > > > >> > > while
>> > > > > > > > > >> > > > >> > > >>>> the
>> > > > > > > > > >> > > > >> > > >>>> >> URM
>> > > > > > > > > >> > > > >> > > >>>> >> >> is preferable in the long
>> term. My
>> > > > > > > suggestion
>> > > > > > > > > >> here is
>> > > > > > > > > >> > > to
>> > > > > > > > > >> > > > >> > > separate
>> > > > > > > > > >> > > > >> > > >>>> the
>> > > > > > > > > >> > > > >> > > >>>> >> >> decision of HOW to do unclean
>> > leader
>> > > > > > > election
>> > > > > > > > > from
>> > > > > > > > > >> > the
>> > > > > > > > > >> > > > >> > decision
>> > > > > > > > > >> > > > >> > > of
>> > > > > > > > > >> > > > >> > > >>>> WHEN
>> > > > > > > > > >> > > > >> > > >>>> >> to
>> > > > > > > > > >> > > > >> > > >>>> >> >> do it.
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> So in other words, have
>> > > > > > > > > >> > > "unclean.leader.election.enable"
>> > > > > > > > > >> > > > >> > specify
>> > > > > > > > > >> > > > >> > > >>>> when we
>> > > > > > > > > >> > > > >> > > >>>> >> >> do unclean leader election, and
>> > > have a
>> > > > > new
>> > > > > > > > > >> > > configuration
>> > > > > > > > > >> > > > >> like
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> "unclean.recovery.manager.enable"
>> > to
>> > > > > > > determine
>> > > > > > > > > if
>> > > > > > > > > >> we
>> > > > > > > > > >> > > use
>> > > > > > > > > >> > > > the
>> > > > > > > > > >> > > > >> > > URM.
>> > > > > > > > > >> > > > >> > > >>>> >> >> Presumably the URM will take
>> some
>> > > time
>> > > > > to
>> > > > > > > get
>> > > > > > > > > >> fully
>> > > > > > > > > >> > > > stable,
>> > > > > > > > > >> > > > >> so
>> > > > > > > > > >> > > > >> > > >>>> this can
>> > > > > > > > > >> > > > >> > > >>>> >> >> default to false for a while,
>> and
>> > we
>> > > > can
>> > > > > > > flip
>> > > > > > > > > the
>> > > > > > > > > >> > > > default to
>> > > > > > > > > >> > > > >> > > true
>> > > > > > > > > >> > > > >> > > >>>> when
>> > > > > > > > > >> > > > >> > > >>>> >> we
>> > > > > > > > > >> > > > >> > > >>>> >> >> feel ready.
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> The URM is somewhat
>> > under-described
>> > > > > here.
>> > > > > > I
>> > > > > > > > > think
>> > > > > > > > > >> we
>> > > > > > > > > >> > > > need a
>> > > > > > > > > >> > > > >> > few
>> > > > > > > > > >> > > > >> > > >>>> >> >> configurations here for it. For
>> > > > example,
>> > > > > > we
>> > > > > > > > > need a
>> > > > > > > > > >> > > > >> > > configuration to
>> > > > > > > > > >> > > > >> > > >>>> >> specify
>> > > > > > > > > >> > > > >> > > >>>> >> >> how long it should wait for a
>> > broker
>> > > > to
>> > > > > > > > respond
>> > > > > > > > > to
>> > > > > > > > > >> > its
>> > > > > > > > > >> > > > RPCs
>> > > > > > > > > >> > > > >> > > before
>> > > > > > > > > >> > > > >> > > >>>> >> moving
>> > > > > > > > > >> > > > >> > > >>>> >> >> on. We also need to understand
>> how
>> > > the
>> > > > > URM
>> > > > > > > > > >> interacts
>> > > > > > > > > >> > > with
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > unclean.leader.election.enable=always. I
>> > > > > > > > assume
>> > > > > > > > > >> that
>> > > > > > > > > >> > > with
>> > > > > > > > > >> > > > >> > > "always"
>> > > > > > > > > >> > > > >> > > >>>> we
>> > > > > > > > > >> > > > >> > > >>>> >> will
>> > > > > > > > > >> > > > >> > > >>>> >> >> just unconditionally use the
>> URM
>> > > > rather
>> > > > > > than
>> > > > > > > > > >> choosing
>> > > > > > > > > >> > > > >> > randomly.
>> > > > > > > > > >> > > > >> > > >>>> But this
>> > > > > > > > > >> > > > >> > > >>>> >> >> should be spelled out in the
>> KIP.
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >> > DescribeTopicRequest
>> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >> >    1.
>> > > > > > > > > >> > > > >> > > >>>> >> >> >    Yes, the plan is to
>> replace
>> > the
>> > > > > > > > > >> MetadataRequest
>> > > > > > > > > >> > > with
>> > > > > > > > > >> > > > >> the
>> > > > > > > > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest for
>> the
>> > > > admin
>> > > > > > > > clients.
>> > > > > > > > > >> Will
>> > > > > > > > > >> > > > check
>> > > > > > > > > >> > > > >> > the
>> > > > > > > > > >> > > > >> > > >>>> details.
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> Sounds good. But as I said, you
>> > need
>> > > > to
>> > > > > > > > specify
>> > > > > > > > > >> how
>> > > > > > > > > >> > > > >> > AdminClient
>> > > > > > > > > >> > > > >> > > >>>> >> interacts
>> > > > > > > > > >> > > > >> > > >>>> >> >> with the new request. This will
>> > > > involve
>> > > > > > > adding
>> > > > > > > > > >> some
>> > > > > > > > > >> > > > fields
>> > > > > > > > > >> > > > >> to
>> > > > > > > > > >> > > > >> > > >>>> >> >> TopicDescription.java. And you
>> > need
>> > > to
>> > > > > > > specify
>> > > > > > > > > the
>> > > > > > > > > >> > > > changes
>> > > > > > > > > >> > > > >> to
>> > > > > > > > > >> > > > >> > > the
>> > > > > > > > > >> > > > >> > > >>>> >> >> kafka-topics.sh command line
>> tool.
>> > > > > > Otherwise
>> > > > > > > > we
>> > > > > > > > > >> > cannot
>> > > > > > > > > >> > > > use
>> > > > > > > > > >> > > > >> the
>> > > > > > > > > >> > > > >> > > >>>> tool to
>> > > > > > > > > >> > > > >> > > >>>> >> see
>> > > > > > > > > >> > > > >> > > >>>> >> >> the new information.
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> The new requests,
>> > > DescribeTopicRequest
>> > > > > and
>> > > > > > > > > >> > > > >> > > >>>> GetReplicaLogInfoRequest,
>> > > > > > > > > >> > > > >> > > >>>> >> need
>> > > > > > > > > >> > > > >> > > >>>> >> >> to have limits placed on them
>> so
>> > > that
>> > > > > > their
>> > > > > > > > size
>> > > > > > > > > >> > can't
>> > > > > > > > > >> > > be
>> > > > > > > > > >> > > > >> > > >>>> infinite. We
>> > > > > > > > > >> > > > >> > > >>>> >> >> don't want to propagate the
>> > current
>> > > > > > problems
>> > > > > > > > of
>> > > > > > > > > >> > > > >> > MetadataRequest,
>> > > > > > > > > >> > > > >> > > >>>> where
>> > > > > > > > > >> > > > >> > > >>>> >> >> clients can request massive
>> > > responses
>> > > > > that
>> > > > > > > can
>> > > > > > > > > >> mess
>> > > > > > > > > >> > up
>> > > > > > > > > >> > > > the
>> > > > > > > > > >> > > > >> JVM
>> > > > > > > > > >> > > > >> > > when
>> > > > > > > > > >> > > > >> > > >>>> >> handled.
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> Adding limits is simple for
>> > > > > > > > > >> GetReplicaLogInfoRequest
>> > > > > > > > > >> > --
>> > > > > > > > > >> > > > we
>> > > > > > > > > >> > > > >> can
>> > > > > > > > > >> > > > >> > > >>>> just say
>> > > > > > > > > >> > > > >> > > >>>> >> >> that only 2000 partitions at a
>> > time
>> > > > can
>> > > > > be
>> > > > > > > > > >> requested.
>> > > > > > > > > >> > > For
>> > > > > > > > > >> > > > >> > > >>>> >> >> DescribeTopicRequest we can
>> > probably
>> > > > > just
>> > > > > > > > limit
>> > > > > > > > > >> to 20
>> > > > > > > > > >> > > > topics
>> > > > > > > > > >> > > > >> > or
>> > > > > > > > > >> > > > >> > > >>>> >> something
>> > > > > > > > > >> > > > >> > > >>>> >> >> like that, to avoid the
>> complexity
>> > > of
>> > > > > > doing
>> > > > > > > > > >> > pagination
>> > > > > > > > > >> > > in
>> > > > > > > > > >> > > > >> this
>> > > > > > > > > >> > > > >> > > KIP.
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> >    2.
>> > > > > > > > > >> > > > >> > > >>>> >> >> >    I can let the broker load
>> the
>> > > ELR
>> > > > > > info
>> > > > > > > so
>> > > > > > > > > >> that
>> > > > > > > > > >> > > they
>> > > > > > > > > >> > > > can
>> > > > > > > > > >> > > > >> > > serve
>> > > > > > > > > >> > > > >> > > >>>> the
>> > > > > > > > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest as
>> well.
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> Yes, it's fine to add to
>> > > > MetadataCache.
>> > > > > In
>> > > > > > > > fact,
>> > > > > > > > > >> > you'll
>> > > > > > > > > >> > > > be
>> > > > > > > > > >> > > > >> > > loading
>> > > > > > > > > >> > > > >> > > >>>> it
>> > > > > > > > > >> > > > >> > > >>>> >> >> anyway once it's added to
>> > > > > PartitionImage.
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> >    3.
>> > > > > > > > > >> > > > >> > > >>>> >> >> >    Yeah, it does not make
>> sense
>> > to
>> > > > > have
>> > > > > > > the
>> > > > > > > > > >> topic
>> > > > > > > > > >> > id
>> > > > > > > > > >> > > if
>> > > > > > > > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest is
>> only
>> > > used
>> > > > > by
>> > > > > > > the
>> > > > > > > > > >> admin
>> > > > > > > > > >> > > > client.
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> OK. That makes things simpler.
>> We
>> > > can
>> > > > > > always
>> > > > > > > > > >> create a
>> > > > > > > > > >> > > new
>> > > > > > > > > >> > > > >> API
>> > > > > > > > > >> > > > >> > > later
>> > > > > > > > > >> > > > >> > > >>>> >> >> (hopefully not in this KIP!) to
>> > > query
>> > > > by
>> > > > > > > topic
>> > > > > > > > > ID.
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >> > Metrics
>> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >> > As for overall cluster health
>> > > > > metrics, I
>> > > > > > > > think
>> > > > > > > > > >> > > > >> under-min-ISR
>> > > > > > > > > >> > > > >> > > is
>> > > > > > > > > >> > > > >> > > >>>> still
>> > > > > > > > > >> > > > >> > > >>>> >> a
>> > > > > > > > > >> > > > >> > > >>>> >> >> > useful one. ELR is more like
>> a
>> > > > safety
>> > > > > > > belt.
>> > > > > > > > > When
>> > > > > > > > > >> > the
>> > > > > > > > > >> > > > ELR
>> > > > > > > > > >> > > > >> is
>> > > > > > > > > >> > > > >> > > >>>> used, the
>> > > > > > > > > >> > > > >> > > >>>> >> >> > cluster availability has
>> already
>> > > > been
>> > > > > > > > > impacted.
>> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >> > Maybe we can have a metric to
>> > > count
>> > > > > the
>> > > > > > > > > >> partitions
>> > > > > > > > > >> > > that
>> > > > > > > > > >> > > > >> > > sum(ISR,
>> > > > > > > > > >> > > > >> > > >>>> ELR)
>> > > > > > > > > >> > > > >> > > >>>> >> <
>> > > > > > > > > >> > > > >> > > >>>> >> >> min
>> > > > > > > > > >> > > > >> > > >>>> >> >> > ISR. What do you think?
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> How about:
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> A.  a metric for the totoal
>> number
>> > > of
>> > > > > > > > > >> under-min-isr
>> > > > > > > > > >> > > > >> > partitions?
>> > > > > > > > > >> > > > >> > > We
>> > > > > > > > > >> > > > >> > > >>>> don't
>> > > > > > > > > >> > > > >> > > >>>> >> >> have that in Apache Kafka at
>> the
>> > > > moment.
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> B. a metric for the number of
>> > > unclean
>> > > > > > leader
>> > > > > > > > > >> > elections
>> > > > > > > > > >> > > we
>> > > > > > > > > >> > > > >> did
>> > > > > > > > > >> > > > >> > > (for
>> > > > > > > > > >> > > > >> > > >>>> >> >> simplicity, it can reset to 0
>> on
>> > > > > > controller
>> > > > > > > > > >> restart:
>> > > > > > > > > >> > we
>> > > > > > > > > >> > > > >> expect
>> > > > > > > > > >> > > > >> > > >>>> people to
>> > > > > > > > > >> > > > >> > > >>>> >> >> monitor the change over time
>> > anyway)
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> best,
>> > > > > > > > > >> > > > >> > > >>>> >> >> Colin
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >> > Yeah, for the ongoing unclean
>> > > > > > recoveries,
>> > > > > > > > the
>> > > > > > > > > >> > > > controller
>> > > > > > > > > >> > > > >> can
>> > > > > > > > > >> > > > >> > > >>>> keep an
>> > > > > > > > > >> > > > >> > > >>>> >> >> > accurate count through
>> failover
>> > > > > because
>> > > > > > > > > >> partition
>> > > > > > > > > >> > > > >> > registration
>> > > > > > > > > >> > > > >> > > >>>> can
>> > > > > > > > > >> > > > >> > > >>>> >> >> indicate
>> > > > > > > > > >> > > > >> > > >>>> >> >> > whether a recovery is needed.
>> > > > However,
>> > > > > > for
>> > > > > > > > the
>> > > > > > > > > >> > > happened
>> > > > > > > > > >> > > > >> > ones,
>> > > > > > > > > >> > > > >> > > >>>> unless
>> > > > > > > > > >> > > > >> > > >>>> >> we
>> > > > > > > > > >> > > > >> > > >>>> >> >> > want to persist the number
>> > > > somewhere,
>> > > > > we
>> > > > > > > can
>> > > > > > > > > >> only
>> > > > > > > > > >> > > > figure
>> > > > > > > > > >> > > > >> it
>> > > > > > > > > >> > > > >> > > out
>> > > > > > > > > >> > > > >> > > >>>> from
>> > > > > > > > > >> > > > >> > > >>>> >> the
>> > > > > > > > > >> > > > >> > > >>>> >> >> > log.
>> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >> > On Tue, Sep 12, 2023 at
>> 3:16 PM
>> > > > Colin
>> > > > > > > > McCabe <
>> > > > > > > > > >> > > > >> > > cmccabe@apache.org
>> > > > > > > > > >> > > > >> > > >>>> >
>> > > > > > > > > >> > > > >> > > >>>> >> wrote:
>> > > > > > > > > >> > > > >> > > >>>> >> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> Also, we should have metrics
>> > that
>> > > > > show
>> > > > > > > what
>> > > > > > > > > is
>> > > > > > > > > >> > going
>> > > > > > > > > >> > > > on
>> > > > > > > > > >> > > > >> > with
>> > > > > > > > > >> > > > >> > > >>>> regard
>> > > > > > > > > >> > > > >> > > >>>> >> to
>> > > > > > > > > >> > > > >> > > >>>> >> >> the
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> eligible replica set. I'm
>> not
>> > > sure
>> > > > > > > exactly
>> > > > > > > > > >> what to
>> > > > > > > > > >> > > > >> suggest,
>> > > > > > > > > >> > > > >> > > but
>> > > > > > > > > >> > > > >> > > >>>> >> >> something
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> that could identify when
>> things
>> > > are
>> > > > > > going
>> > > > > > > > > >> wrong in
>> > > > > > > > > >> > > the
>> > > > > > > > > >> > > > >> > > clsuter.
>> > > > > > > > > >> > > > >> > > >>>> >> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> For example, maybe a metric
>> for
>> > > > > > > partitions
>> > > > > > > > > >> > > containing
>> > > > > > > > > >> > > > >> > > replicas
>> > > > > > > > > >> > > > >> > > >>>> that
>> > > > > > > > > >> > > > >> > > >>>> >> are
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> ineligible to be leader?
>> That
>> > > would
>> > > > > > show
>> > > > > > > a
>> > > > > > > > > >> spike
>> > > > > > > > > >> > > when
>> > > > > > > > > >> > > > a
>> > > > > > > > > >> > > > >> > > broker
>> > > > > > > > > >> > > > >> > > >>>> had an
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> unclean restart.
>> > > > > > > > > >> > > > >> > > >>>> >> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> Ideally, we'd also have a
>> > metric
>> > > > that
>> > > > > > > > > indicates
>> > > > > > > > > >> > when
>> > > > > > > > > >> > > > an
>> > > > > > > > > >> > > > >> > > unclear
>> > > > > > > > > >> > > > >> > > >>>> >> leader
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> election or a recovery
>> > happened.
>> > > > > It's a
>> > > > > > > bit
>> > > > > > > > > >> tricky
>> > > > > > > > > >> > > > >> because
>> > > > > > > > > >> > > > >> > > the
>> > > > > > > > > >> > > > >> > > >>>> simple
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> thing, of tracking it per
>> > > > controller,
>> > > > > > may
>> > > > > > > > be
>> > > > > > > > > a
>> > > > > > > > > >> bit
>> > > > > > > > > >> > > > >> > confusing
>> > > > > > > > > >> > > > >> > > >>>> during
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> failovers.
>> > > > > > > > > >> > > > >> > > >>>> >> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> best,
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> Colin
>> > > > > > > > > >> > > > >> > > >>>> >> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> >>
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> On Tue, Sep 12, 2023, at
>> 14:25,
>> > > > Colin
>> > > > > > > > McCabe
>> > > > > > > > > >> > wrote:
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> > Hi Calvin,
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> > Thanks for the KIP. I
>> think
>> > > this
>> > > > > is a
>> > > > > > > > great
>> > > > > > > > > >> > > > >> improvement.
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> >> Additional High Watermark
>> > > > advance
>> > > > > > > > > >> requirement
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> > Typo: change "advance" to
>> > > > > > "advancement"
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> >> A bit recap of some key
>> > > > concepts.
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> > Typo: change "bit" to
>> "quick"
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> >> Ack=1/all produce
>> request.
>> > It
>> > > > > > defines
>> > > > > > > > when
>> > > > > > > > > >> the
>> > > > > > > > > >> > > > Kafka
>> > > > > > > > > >> > > > >> > > server
>> > > > > > > > > >> > > > >> > > >>>> should
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> respond to the produce
>> request
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> > I think this section
>> would be
>> > > > > clearer
>> > > > > > > if
>> > > > > > > > we
>> > > > > > > > > >> > talked
>> > > > > > > > > >> > > > >> about
>> > > > > > > > > >> > > > >> > > the
>> > > > > > > > > >> > > > >> > > >>>> new
>> > > > > > > > > >> > > > >> > > >>>> >> high
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> > watermark advancement
>> > > requirement
>> > > > > > > first,
>> > > > > > > > > and
>> > > > > > > > > >> > THEN
>> > > > > > > > > >> > > > >> talked
>> > > > > > > > > >> > > > >> > > >>>> about its
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> > impact on acks=0, acks=1,
>> and
>> > > > > > > >  acks=all.
>> > > > > > > > > >> > > > acks=all
>> > > > > > > > > >> > > > >> is
>> > > > > > > > > >> > > > >> > of
>> > > > > > > > > >> > > > >> > > >>>> course
>> > > > > > > > > >> > > > >> > > >>>> >> the
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> > main case we care about
>> here,
>> > > so
>> > > > it
>> > > > > > > would
>> > > > > > > > > be
>> > > > > > > > > >> > good
>> > > > > > > > > >> > > to
>> > > > > > > > > >> > > > >> lead
>> > > > > > > > > >> > > > >> > > with
>> > > > > > > > > >> > > > >> > > >>>> >> that,
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> > rather than delving into
>> the
>> > > > > > > > technicalities
>> > > > > > > > > >> of
>> > > > > > > > > >> > > > acks=0/1
>> > > > > > > > > >> > > > >> > > first.
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> >> Unclean recovery
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> > So, here you are
>> introducing
>> > a
>> > > > new
>> > > > > > > > > >> > configuration,
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> > unclean.recovery.strategy.
>> > The
>> > > > > > > difficult
>> > > > > > > > > >> thing
>> > > > > > > > > >> > > here
>> > > > > > > > > >> > > > is
>> > > > > > > > > >> > > > >> > that
>> > > > > > > > > >> > > > >> > > >>>> there
>> > > > > > > > > >> > > > >> > > >>>> >> is a
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> > lot of overlap with
>> > > > > > > > > >> > > unclean.leader.election.enable.
>> > > > > > > > > >> > > > So
>> > > > > > > > > >> > > > >> we
>> > > > > > > > > >> > > > >> > > >>>> have 3
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> > different settings for
>> > > > > > > > > >> > unclean.recovery.strategy,
>> > > > > > > > > >> > > > plus
>> > > > > > > > > >> > > > >> 2
>> > > > > > > > > >> > > > >> > > >>>> different
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> > settings for
>> > > > > > > > > unclean.leader.election.enable,
>> > > > > > > > > >> > > giving
>> > > > > > > > > >> > > > a
>> > > > > > > > > >> > > > >> > cross
>> > > > > > > > > >> > > > >> > > >>>> >> product of
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> > 6 different options. The
>> > > > following
>> > > > > > > > "unclean
>> > > > > > > > > >> > > recovery
>> > > > > > > > > >> > > > >> > > manager"
>> > > > > > > > > >> > > > >> > > >>>> >> section
>> > > > > > > > > >> > > > >> > > >>>> >> >> >> > on
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>>
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

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

One more comment.

"The first partition to fetch details for. -1 means to fetch all
partitions." It seems that FirstPartitionId of 0 naturally means fetching
all partitions?

Thanks,

Jun

On Tue, Oct 10, 2023 at 12:40 PM Calvin Liu <ca...@confluent.io.invalid>
wrote:

> Hi Jun,
> Yeah, with the current Metadata request handling, we only return errors on
> the Topic level, like topic not found. It seems that querying a specific
> partition is not a valid use case. Will update.
> Thanks
>
> On Tue, Oct 10, 2023 at 11:55 AM Jun Rao <ju...@confluent.io.invalid> wrote:
>
> > Hi, Calvin,
> >
> > 60.  If the range query has errors for some of the partitions, do we
> expect
> > different responses when querying particular partitions?
> >
> > Thanks,
> >
> > Jun
> >
> > On Tue, Oct 10, 2023 at 10:50 AM Calvin Liu <ca...@confluent.io.invalid>
> > wrote:
> >
> > > Hi Jun
> > > 60. Yes, it is a good question. I was thinking the API could be
> flexible
> > to
> > > query the particular partitions if the range query has errors for some
> of
> > > the partitions. Not sure whether it is a valid assumption, what do you
> > > think?
> > >
> > > 61. Good point, I will update them to partition level with the same
> > limit.
> > >
> > > 62. Sure, will do.
> > >
> > > Thanks
> > >
> > > On Tue, Oct 10, 2023 at 10:12 AM Jun Rao <ju...@confluent.io.invalid>
> > wrote:
> > >
> > > > Hi, Calvin,
> > > >
> > > > A few more minor comments on your latest update.
> > > >
> > > > 60. DescribeTopicRequest: When will the Partitions field be used? It
> > > seems
> > > > that the FirstPartitionId field is enough for AdminClient usage.
> > > >
> > > > 61. Could we make the limit for DescribeTopicRequest,
> > > ElectLeadersRequest,
> > > > GetReplicaLogInfo consistent? Currently, ElectLeadersRequest's limit
> is
> > > at
> > > > topic level and GetReplicaLogInfo has a different partition level
> limit
> > > > from DescribeTopicRequest.
> > > >
> > > > 62. Should ElectLeadersRequest.DesiredLeaders be at the same level as
> > > > ElectLeadersRequest.TopicPartitions.Partitions? In the KIP, it looks
> > like
> > > > it's at the same level as ElectLeadersRequest.TopicPartitions.
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > > On Wed, Oct 4, 2023 at 3:55 PM Calvin Liu <caliu@confluent.io.invalid
> >
> > > > wrote:
> > > >
> > > > > Hi David,
> > > > > Thanks for the comments.
> > > > > ----
> > > > > I thought that a new snapshot with the downgraded MV is created in
> > this
> > > > > case. Isn’t it the case?
> > > > > Yes, you are right, a metadata delta will be generated after the MV
> > > > > downgrade. Then the user can start the software downgrade.
> > > > > -----
> > > > > Could you also elaborate a bit more on the reasoning behind adding
> > the
> > > > > limits to the admin RPCs? This is a new pattern in Kafka so it
> would
> > be
> > > > > good to clear on the motivation.
> > > > > Thanks to Colin for bringing it up. The current MetadataRequest
> does
> > > not
> > > > > have a limit on the number of topics to query in a single request.
> > > > Massive
> > > > > requests can mess up the JVM. We want to have some sort of throttle
> > on
> > > > the
> > > > > new APIs.
> > > > > -----
> > > > > Could you also explain how the client is supposed to handle the
> > > > > topics/partitions above the limit? I suppose that it will have to
> > retry
> > > > > those, correct?
> > > > > Corrent. For the official admin clients, it will split the large
> > > request
> > > > > into proper pieces and query one after another.
> > > > > -----
> > > > > My understanding is that the topics/partitions above the limit will
> > be
> > > > > failed with an invalid exception error. I wonder if this choice is
> > > > > judicious because the invalide request exception is usually fatal.
> It
> > > may
> > > > > be better to use an new and explicit error for this case.
> > > > >
> > > > > Thanks for bringing this up. How about "REQUEST_LIMIT_REACHED"?
> > > > > --------
> > > > > It seems that we still need to specify the changes to the admin api
> > to
> > > > > accommodate the new or updated apis. Do you plan to add them?
> > > > > Try to cover the following
> > > > > 1. The admin client will use the new DescribeTopicRequest to query
> > the
> > > > > topics
> > > > > 2. Mention the API limit and the new retriable error.
> > > > > 3. Output changes for the admin client when describing a topic (new
> > > > fields
> > > > > of ELR...)
> > > > > 4. Changes to data structures like TopicPartitionInfo to include
> the
> > > ELR.
> > > > > Anything else I missed?
> > > > >
> > > > > Thanks!
> > > > >
> > > > >
> > > > >
> > > > >
> > > > >
> > > > > On Wed, Oct 4, 2023 at 12:27 PM David Jacot <david.jacot@gmail.com
> >
> > > > wrote:
> > > > >
> > > > > > Hi Calvin,
> > > > > >
> > > > > > I thought that a new snapshot with the downgraded MV is created
> in
> > > this
> > > > > > case. Isn’t it the case?
> > > > > >
> > > > > > Could you also elaborate a bit more on the reasoning behind
> adding
> > > the
> > > > > > limits to the admin RPCs? This is a new pattern in Kafka so it
> > would
> > > be
> > > > > > good to clear on the motivation.
> > > > > >
> > > > > > Could you also explain how the client is supposed to handle the
> > > > > > topics/partitions above the limit? I suppose that it will have to
> > > retry
> > > > > > those, correct?
> > > > > >
> > > > > > My understanding is that the topics/partitions above the limit
> will
> > > be
> > > > > > failed with an invalid exception error. I wonder if this choice
> is
> > > > > > judicious because the invalide request exception is usually
> fatal.
> > It
> > > > may
> > > > > > be better to use an new and explicit error for this case.
> > > > > >
> > > > > > It seems that we still need to specify the changes to the admin
> api
> > > to
> > > > > > accommodate the new or updated apis. Do you plan to add them?
> > > > > >
> > > > > > Best,
> > > > > > David
> > > > > >
> > > > > > Le mer. 4 oct. 2023 à 20:39, Calvin Liu
> <caliu@confluent.io.invalid
> > >
> > > a
> > > > > > écrit :
> > > > > >
> > > > > > > Hi Jun,
> > > > > > > After the MV downgrade, the controller will write in the old
> > > version
> > > > of
> > > > > > the
> > > > > > > PartitionRecord/PartitionChangeRecord. If I understand
> correctly,
> > > it
> > > > is
> > > > > > > possible to downgrade the software version if the controller
> only
> > > has
> > > > > to
> > > > > > > handle old version records.
> > > > > > > However, the controller will not automatically rewrite the
> > > > > > PartitionRecord
> > > > > > > with the old version unless there is a partition update. Then,
> > the
> > > > user
> > > > > > may
> > > > > > > have to wait an unknown amount of time before the software
> > > downgrades
> > > > > > > unless they do a roll to force update every partition. If it
> > makes
> > > > > > sense, I
> > > > > > > can mention these steps to do a software downgrade.
> > > > > > > Thanks
> > > > > > >
> > > > > > > On Wed, Oct 4, 2023 at 11:20 AM Jun Rao
> <jun@confluent.io.invalid
> > >
> > > > > > wrote:
> > > > > > >
> > > > > > > > Hi, Calvin and Justine,
> > > > > > > >
> > > > > > > > Historically, when we change the record format in the log, we
> > > don't
> > > > > > > support
> > > > > > > > software version downgrading.
> > > > > > > >
> > > > > > > > For the record format change in the metadata log, have we
> > thought
> > > > > about
> > > > > > > > forcing the write of the latest metadata records with the old
> > > > version
> > > > > > > > during MV downgrading? This will in theory allow the old
> > version
> > > of
> > > > > the
> > > > > > > > software to obtain the latest metadata.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > >
> > > > > > > > Jun
> > > > > > > >
> > > > > > > > On Wed, Oct 4, 2023 at 9:53 AM Justine Olshan
> > > > > > > <jolshan@confluent.io.invalid
> > > > > > > > >
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Sorry -- not MV but software version.
> > > > > > > > >
> > > > > > > > > On Wed, Oct 4, 2023 at 9:51 AM Justine Olshan <
> > > > > jolshan@confluent.io>
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Catching up with this discussion.
> > > > > > > > > >
> > > > > > > > > > I was just curious -- have we had other instances where
> > > > > downgrading
> > > > > > > MV
> > > > > > > > is
> > > > > > > > > > not supported? I think Kafka typically tries to support
> > > > > downgrades,
> > > > > > > > and I
> > > > > > > > > > couldn't think of other examples.
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > Justine
> > > > > > > > > >
> > > > > > > > > > On Wed, Oct 4, 2023 at 9:40 AM Calvin Liu
> > > > > > <caliu@confluent.io.invalid
> > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > >> Hi Jun,
> > > > > > > > > >> 54. Marked the software downgrading is not supported. As
> > the
> > > > old
> > > > > > > > > >> controller
> > > > > > > > > >> will not understand the new PartitionRecord and
> > > > > > > PartitionChangeRecord.
> > > > > > > > > >> Thanks!
> > > > > > > > > >>
> > > > > > > > > >> On Wed, Oct 4, 2023 at 9:12 AM Jun Rao
> > > > <jun@confluent.io.invalid
> > > > > >
> > > > > > > > > wrote:
> > > > > > > > > >>
> > > > > > > > > >> > Hi, Calvin,
> > > > > > > > > >> >
> > > > > > > > > >> > Thanks for the reply. Just one more comment.
> > > > > > > > > >> >
> > > > > > > > > >> > 54. It seems that downgrading MV is supported. Is
> > > > downgrading
> > > > > > the
> > > > > > > > > >> software
> > > > > > > > > >> > version supported? It would be useful to document
> that.
> > > > > > > > > >> >
> > > > > > > > > >> > Thanks,
> > > > > > > > > >> >
> > > > > > > > > >> > Jun
> > > > > > > > > >> >
> > > > > > > > > >> > On Tue, Oct 3, 2023 at 4:55 PM Artem Livshits
> > > > > > > > > >> > <al...@confluent.io.invalid> wrote:
> > > > > > > > > >> >
> > > > > > > > > >> > > Hi Colin,
> > > > > > > > > >> > >
> > > > > > > > > >> > > I think in your example "do_unclean_recovery" would
> > need
> > > > to
> > > > > do
> > > > > > > > > >> different
> > > > > > > > > >> > > things depending on the strategy.
> > > > > > > > > >> > >
> > > > > > > > > >> > > do_unclean_recovery() {
> > > > > > > > > >> > >    if (unclean.recovery.manager.enabled) {
> > > > > > > > > >> > >     if (strategy == Aggressive)
> > > > > > > > > >> > >       use
> > UncleanRecoveryManager(waitLastKnownERL=false)
> > > > //
> > > > > > > just
> > > > > > > > > >> inspect
> > > > > > > > > >> > > logs from whoever is available
> > > > > > > > > >> > >     else
> > > > > > > > > >> > >       use
> > UncleanRecoveryManager(waitLastKnownERL=true)
> > > > //
> > > > > > > must
> > > > > > > > > wait
> > > > > > > > > >> > for
> > > > > > > > > >> > > at least last known ELR
> > > > > > > > > >> > >   } else {
> > > > > > > > > >> > >     if (strategy == Aggressive)
> > > > > > > > > >> > >       choose the last known leader if that is
> > available,
> > > > or
> > > > > a
> > > > > > > > random
> > > > > > > > > >> > leader
> > > > > > > > > >> > > if not)
> > > > > > > > > >> > >     else
> > > > > > > > > >> > >       wait for last known leader to get back
> > > > > > > > > >> > >   }
> > > > > > > > > >> > > }
> > > > > > > > > >> > >
> > > > > > > > > >> > > The idea is that the Aggressive strategy would kick
> in
> > > as
> > > > > soon
> > > > > > > as
> > > > > > > > we
> > > > > > > > > >> lost
> > > > > > > > > >> > > the leader and would pick a leader from whoever is
> > > > > available;
> > > > > > > but
> > > > > > > > > the
> > > > > > > > > >> > > Balanced will only kick in when ELR is empty and
> will
> > > wait
> > > > > for
> > > > > > > the
> > > > > > > > > >> > brokers
> > > > > > > > > >> > > that likely have most data to be available.
> > > > > > > > > >> > >
> > > > > > > > > >> > > On Tue, Oct 3, 2023 at 3:04 PM Colin McCabe <
> > > > > > cmccabe@apache.org
> > > > > > > >
> > > > > > > > > >> wrote:
> > > > > > > > > >> > >
> > > > > > > > > >> > > > On Tue, Oct 3, 2023, at 10:49, Jun Rao wrote:
> > > > > > > > > >> > > > > Hi, Calvin,
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > > Thanks for the update KIP. A few more comments.
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > > 41. Why would a user choose the option to
> select a
> > > > > random
> > > > > > > > > replica
> > > > > > > > > >> as
> > > > > > > > > >> > > the
> > > > > > > > > >> > > > > leader instead of using
> > > > > > unclean.recovery.strateg=Aggressive?
> > > > > > > > It
> > > > > > > > > >> seems
> > > > > > > > > >> > > > that
> > > > > > > > > >> > > > > the latter is strictly better? If that's not the
> > > case,
> > > > > > could
> > > > > > > > we
> > > > > > > > > >> fold
> > > > > > > > > >> > > this
> > > > > > > > > >> > > > > option under unclean.recovery.strategy instead
> of
> > > > > > > introducing
> > > > > > > > a
> > > > > > > > > >> > > separate
> > > > > > > > > >> > > > > config?
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > Hi Jun,
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > I thought the flow of control was:
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > If there is no leader for the partition {
> > > > > > > > > >> > > >   If (there are unfenced ELR members) {
> > > > > > > > > >> > > >     choose_an_unfenced_ELR_member
> > > > > > > > > >> > > >   } else if (there are fenced ELR members AND
> > > > > > > > > strategy=Aggressive) {
> > > > > > > > > >> > > >     do_unclean_recovery
> > > > > > > > > >> > > >   } else if (there are no ELR members AND strategy
> > !=
> > > > > None)
> > > > > > {
> > > > > > > > > >> > > >     do_unclean_recovery
> > > > > > > > > >> > > >   } else {
> > > > > > > > > >> > > >     do nothing about the missing leader
> > > > > > > > > >> > > >   }
> > > > > > > > > >> > > > }
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > do_unclean_recovery() {
> > > > > > > > > >> > > >    if (unclean.recovery.manager.enabled) {
> > > > > > > > > >> > > >     use UncleanRecoveryManager
> > > > > > > > > >> > > >   } else {
> > > > > > > > > >> > > >     choose the last known leader if that is
> > available,
> > > > or
> > > > > a
> > > > > > > > random
> > > > > > > > > >> > leader
> > > > > > > > > >> > > > if not)
> > > > > > > > > >> > > >   }
> > > > > > > > > >> > > > }
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > However, I think this could be clarified,
> especially
> > > the
> > > > > > > > behavior
> > > > > > > > > >> when
> > > > > > > > > >> > > > unclean.recovery.manager.enabled=false. Inuitively
> > the
> > > > > goal
> > > > > > > for
> > > > > > > > > >> > > > unclean.recovery.manager.enabled=false is to be
> "the
> > > > same
> > > > > as
> > > > > > > > now,
> > > > > > > > > >> > mostly"
> > > > > > > > > >> > > > but it's very underspecified in the KIP, I agree.
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > > 50. ElectLeadersRequest: "If more than 20 topics
> > are
> > > > > > > included,
> > > > > > > > > >> only
> > > > > > > > > >> > the
> > > > > > > > > >> > > > > first 20 will be served. Others will be returned
> > > with
> > > > > > > > > >> > DesiredLeaders."
> > > > > > > > > >> > > > Hmm,
> > > > > > > > > >> > > > > not sure that I understand this.
> > > ElectLeadersResponse
> > > > > > > doesn't
> > > > > > > > > >> have a
> > > > > > > > > >> > > > > DesiredLeaders field.
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > > 51. GetReplicaLogInfo: "If more than 2000
> > partitions
> > > > are
> > > > > > > > > included,
> > > > > > > > > >> > only
> > > > > > > > > >> > > > the
> > > > > > > > > >> > > > > first 2000 will be served" Do we return an error
> > for
> > > > the
> > > > > > > > > remaining
> > > > > > > > > >> > > > > partitions? Actually, should we include an
> > errorCode
> > > > > field
> > > > > > > at
> > > > > > > > > the
> > > > > > > > > >> > > > partition
> > > > > > > > > >> > > > > level in GetReplicaLogInfoResponse to cover
> > > > non-existing
> > > > > > > > > >> partitions
> > > > > > > > > >> > and
> > > > > > > > > >> > > > no
> > > > > > > > > >> > > > > authorization, etc?
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > > 52. The entry should matches => The entry should
> > > match
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > > 53. ElectLeadersRequest.DesiredLeaders: Should
> it
> > be
> > > > > > > nullable
> > > > > > > > > >> since a
> > > > > > > > > >> > > > user
> > > > > > > > > >> > > > > may not specify DesiredLeaders?
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > > 54. Downgrade: Is that indeed possible? I
> thought
> > > > > earlier
> > > > > > > you
> > > > > > > > > said
> > > > > > > > > >> > that
> > > > > > > > > >> > > > > once the new version of the records are in the
> > > > metadata
> > > > > > log,
> > > > > > > > one
> > > > > > > > > >> > can't
> > > > > > > > > >> > > > > downgrade since the old broker doesn't know how
> to
> > > > parse
> > > > > > the
> > > > > > > > new
> > > > > > > > > >> > > version
> > > > > > > > > >> > > > of
> > > > > > > > > >> > > > > the metadata records?
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > MetadataVersion downgrade is currently broken but
> we
> > > > have
> > > > > > > fixing
> > > > > > > > > it
> > > > > > > > > >> on
> > > > > > > > > >> > > our
> > > > > > > > > >> > > > plate for Kafka 3.7.
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > The way downgrade works is that "new features" are
> > > > > dropped,
> > > > > > > > > leaving
> > > > > > > > > >> > only
> > > > > > > > > >> > > > the old ones.
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > > 55. CleanShutdownFile: Should we add a version
> > field
> > > > for
> > > > > > > > future
> > > > > > > > > >> > > > extension?
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > > 56. Config changes are public facing. Could we
> > have
> > > a
> > > > > > > separate
> > > > > > > > > >> > section
> > > > > > > > > >> > > to
> > > > > > > > > >> > > > > document all the config changes?
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > +1. A separate section for this would be good.
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > best,
> > > > > > > > > >> > > > Colin
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > > Thanks,
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > > Jun
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > > On Mon, Sep 25, 2023 at 4:29 PM Calvin Liu
> > > > > > > > > >> > <caliu@confluent.io.invalid
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > > wrote:
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > >> Hi Jun
> > > > > > > > > >> > > > >> Thanks for the comments.
> > > > > > > > > >> > > > >>
> > > > > > > > > >> > > > >> 40. If we change to None, it is not guaranteed
> > for
> > > no
> > > > > > data
> > > > > > > > > loss.
> > > > > > > > > >> For
> > > > > > > > > >> > > > users
> > > > > > > > > >> > > > >> who are not able to validate the data with
> > external
> > > > > > > > resources,
> > > > > > > > > >> > manual
> > > > > > > > > >> > > > >> intervention does not give a better result but
> a
> > > loss
> > > > > of
> > > > > > > > > >> > availability.
> > > > > > > > > >> > > > So
> > > > > > > > > >> > > > >> practically speaking, the Balance mode would
> be a
> > > > > better
> > > > > > > > > default
> > > > > > > > > >> > > value.
> > > > > > > > > >> > > > >>
> > > > > > > > > >> > > > >> 41. No, it represents how we want to do the
> > unclean
> > > > > > leader
> > > > > > > > > >> election.
> > > > > > > > > >> > > If
> > > > > > > > > >> > > > it
> > > > > > > > > >> > > > >> is false, the unclean leader election will be
> the
> > > old
> > > > > > > random
> > > > > > > > > way.
> > > > > > > > > >> > > > >> Otherwise, the unclean recovery will be used.
> > > > > > > > > >> > > > >>
> > > > > > > > > >> > > > >> 42. Good catch. Updated.
> > > > > > > > > >> > > > >>
> > > > > > > > > >> > > > >> 43. Only the first 20 topics will be served.
> > Others
> > > > > will
> > > > > > be
> > > > > > > > > >> returned
> > > > > > > > > >> > > > with
> > > > > > > > > >> > > > >> InvalidRequestError
> > > > > > > > > >> > > > >>
> > > > > > > > > >> > > > >> 44. The order matters. The desired leader
> entries
> > > > match
> > > > > > > with
> > > > > > > > > the
> > > > > > > > > >> > topic
> > > > > > > > > >> > > > >> partition list by the index.
> > > > > > > > > >> > > > >>
> > > > > > > > > >> > > > >> 45. Thanks! Updated.
> > > > > > > > > >> > > > >>
> > > > > > > > > >> > > > >> 46. Good advice! Updated.
> > > > > > > > > >> > > > >>
> > > > > > > > > >> > > > >> 47.1, updated the comment. Basically it will
> > elect
> > > > the
> > > > > > > > replica
> > > > > > > > > in
> > > > > > > > > >> > the
> > > > > > > > > >> > > > >> desiredLeader field to be the leader
> > > > > > > > > >> > > > >>
> > > > > > > > > >> > > > >> 47.2 We can let the admin client do the
> > conversion.
> > > > > Using
> > > > > > > the
> > > > > > > > > >> > > > desiredLeader
> > > > > > > > > >> > > > >> field in the json format seems easier for
> users.
> > > > > > > > > >> > > > >>
> > > > > > > > > >> > > > >> 48. Once the MV version is downgraded, all the
> > ELR
> > > > > > related
> > > > > > > > > fields
> > > > > > > > > >> > will
> > > > > > > > > >> > > > be
> > > > > > > > > >> > > > >> removed on the next partition change. The
> > > controller
> > > > > will
> > > > > > > > also
> > > > > > > > > >> > ignore
> > > > > > > > > >> > > > the
> > > > > > > > > >> > > > >> ELR fields. Updated the KIP.
> > > > > > > > > >> > > > >>
> > > > > > > > > >> > > > >> 49. Yes, it would be deprecated/removed.
> > > > > > > > > >> > > > >>
> > > > > > > > > >> > > > >>
> > > > > > > > > >> > > > >> On Mon, Sep 25, 2023 at 3:49 PM Jun Rao
> > > > > > > > > <jun@confluent.io.invalid
> > > > > > > > > >> >
> > > > > > > > > >> > > > wrote:
> > > > > > > > > >> > > > >>
> > > > > > > > > >> > > > >> > Hi, Calvin,
> > > > > > > > > >> > > > >> >
> > > > > > > > > >> > > > >> > Thanks for the updated KIP. Made another
> pass.
> > A
> > > > few
> > > > > > more
> > > > > > > > > >> comments
> > > > > > > > > >> > > > below.
> > > > > > > > > >> > > > >> >
> > > > > > > > > >> > > > >> > 40. unclean.leader.election.enable.false ->
> > > > > > > > > >> > > > >> > unclean.recovery.strategy.Balanced: The
> > Balanced
> > > > mode
> > > > > > > could
> > > > > > > > > >> still
> > > > > > > > > >> > > > lead to
> > > > > > > > > >> > > > >> > data loss. So, I am wondering if
> > > > > > > > > >> > > unclean.leader.election.enable.false
> > > > > > > > > >> > > > >> > should map to None?
> > > > > > > > > >> > > > >> >
> > > > > > > > > >> > > > >> > 41. unclean.recovery.manager.enabled: I am
> not
> > > sure
> > > > > why
> > > > > > > we
> > > > > > > > > >> > introduce
> > > > > > > > > >> > > > this
> > > > > > > > > >> > > > >> > additional config. Is it the same as
> > > > > > > > > >> > unclean.recovery.strategy=None?
> > > > > > > > > >> > > > >> >
> > > > > > > > > >> > > > >> > 42.
> > > > DescribeTopicResponse.TopicAuthorizedOperations:
> > > > > > > Should
> > > > > > > > > >> this
> > > > > > > > > >> > be
> > > > > > > > > >> > > at
> > > > > > > > > >> > > > >> the
> > > > > > > > > >> > > > >> > topic level?
> > > > > > > > > >> > > > >> >
> > > > > > > > > >> > > > >> > 43. "Limit: 20 topics max per request": Could
> > we
> > > > > > describe
> > > > > > > > > what
> > > > > > > > > >> > > > happens if
> > > > > > > > > >> > > > >> > the request includes more than 20 topics?
> > > > > > > > > >> > > > >> >
> > > > > > > > > >> > > > >> > 44. ElectLeadersRequest.DesiredLeaders: Could
> > we
> > > > > > describe
> > > > > > > > > >> whether
> > > > > > > > > >> > > the
> > > > > > > > > >> > > > >> > ordering matters?
> > > > > > > > > >> > > > >> >
> > > > > > > > > >> > > > >> > 45. GetReplicaLogInfo.TopicPartitions:
> "about":
> > > > "The
> > > > > > > topic
> > > > > > > > > >> > > partitions
> > > > > > > > > >> > > > to
> > > > > > > > > >> > > > >> > elect leaders.": The description in "about"
> is
> > > > > > incorrect.
> > > > > > > > > >> > > > >> >
> > > > > > > > > >> > > > >> > 46. GetReplicaLogInfoResponse: Should we nest
> > > > > > partitions
> > > > > > > > > under
> > > > > > > > > >> > > > topicId to
> > > > > > > > > >> > > > >> > be consistent with other types of responses?
> > > > > > > > > >> > > > >> >
> > > > > > > > > >> > > > >> > 47. kafka-leader-election.sh:
> > > > > > > > > >> > > > >> > 47.1 Could we explain DESIGNATION?
> > > > > > > > > >> > > > >> > 47.2 desiredLeader: Should it be a list to
> > match
> > > > the
> > > > > > > field
> > > > > > > > in
> > > > > > > > > >> > > > >> > ElectLeadersRequest?
> > > > > > > > > >> > > > >> >
> > > > > > > > > >> > > > >> > 48. We could add a section on downgrade?
> > > > > > > > > >> > > > >> >
> > > > > > > > > >> > > > >> > 49. LastKnownLeader: This seems only needed
> in
> > > the
> > > > > > first
> > > > > > > > > phase
> > > > > > > > > >> of
> > > > > > > > > >> > > > >> > delivering ELR. Will it be removed when the
> > > > complete
> > > > > > KIP
> > > > > > > is
> > > > > > > > > >> > > delivered?
> > > > > > > > > >> > > > >> >
> > > > > > > > > >> > > > >> > Thanks,
> > > > > > > > > >> > > > >> >
> > > > > > > > > >> > > > >> > Jun
> > > > > > > > > >> > > > >> >
> > > > > > > > > >> > > > >> > On Tue, Sep 19, 2023 at 1:30 PM Colin McCabe
> <
> > > > > > > > > >> cmccabe@apache.org>
> > > > > > > > > >> > > > wrote:
> > > > > > > > > >> > > > >> >
> > > > > > > > > >> > > > >> > > Hi Calvin,
> > > > > > > > > >> > > > >> > >
> > > > > > > > > >> > > > >> > > Thanks for the explanations. I like the
> idea
> > of
> > > > > using
> > > > > > > > none,
> > > > > > > > > >> > > > balanced,
> > > > > > > > > >> > > > >> > > aggressive. We also had an offline
> discussion
> > > > about
> > > > > > why
> > > > > > > > it
> > > > > > > > > is
> > > > > > > > > >> > good
> > > > > > > > > >> > > > to
> > > > > > > > > >> > > > >> > use a
> > > > > > > > > >> > > > >> > > new config key (basically, so that we can
> > > > deprecate
> > > > > > the
> > > > > > > > old
> > > > > > > > > >> one
> > > > > > > > > >> > > > which
> > > > > > > > > >> > > > >> had
> > > > > > > > > >> > > > >> > > only false/true values in 4.0) With these
> > > > changes,
> > > > > I
> > > > > > am
> > > > > > > > +1.
> > > > > > > > > >> > > > >> > >
> > > > > > > > > >> > > > >> > > best,
> > > > > > > > > >> > > > >> > > Colin
> > > > > > > > > >> > > > >> > >
> > > > > > > > > >> > > > >> > > On Mon, Sep 18, 2023, at 15:54, Calvin Liu
> > > wrote:
> > > > > > > > > >> > > > >> > > > Hi Colin,
> > > > > > > > > >> > > > >> > > > Also, can we deprecate
> > > > > > unclean.leader.election.enable
> > > > > > > > in
> > > > > > > > > >> 4.0?
> > > > > > > > > >> > > > Before
> > > > > > > > > >> > > > >> > > that,
> > > > > > > > > >> > > > >> > > > we can have both the config
> > > > > > unclean.recovery.strategy
> > > > > > > > and
> > > > > > > > > >> > > > >> > > > unclean.leader.election.enable
> > > > > > > > > >> > > > >> > > > and using the unclean.recovery.Enabled to
> > > > > determine
> > > > > > > > which
> > > > > > > > > >> > config
> > > > > > > > > >> > > > to
> > > > > > > > > >> > > > >> use
> > > > > > > > > >> > > > >> > > > during the unclean leader election.
> > > > > > > > > >> > > > >> > > >
> > > > > > > > > >> > > > >> > > > On Mon, Sep 18, 2023 at 3:51 PM Calvin
> Liu
> > <
> > > > > > > > > >> > caliu@confluent.io>
> > > > > > > > > >> > > > >> wrote:
> > > > > > > > > >> > > > >> > > >
> > > > > > > > > >> > > > >> > > >> Hi Colin,
> > > > > > > > > >> > > > >> > > >> For the unclean.recovery.strategy config
> > > name,
> > > > > how
> > > > > > > > about
> > > > > > > > > >> we
> > > > > > > > > >> > use
> > > > > > > > > >> > > > the
> > > > > > > > > >> > > > >> > > >> following
> > > > > > > > > >> > > > >> > > >> None. It basically means no unclean
> > recovery
> > > > > will
> > > > > > be
> > > > > > > > > >> > performed.
> > > > > > > > > >> > > > >> > > >> Aggressive. It means availability goes
> > > first.
> > > > > > > Whenever
> > > > > > > > > the
> > > > > > > > > >> > > > partition
> > > > > > > > > >> > > > >> > > can't
> > > > > > > > > >> > > > >> > > >> elect a durable replica, the controller
> > will
> > > > try
> > > > > > the
> > > > > > > > > >> unclean
> > > > > > > > > >> > > > >> recovery.
> > > > > > > > > >> > > > >> > > >> Balanced. It is the balance point of the
> > > > > > > availability
> > > > > > > > > >> > > > >> > first(Aggressive)
> > > > > > > > > >> > > > >> > > >> and least availability(None). The
> > controller
> > > > > > > performs
> > > > > > > > > >> unclean
> > > > > > > > > >> > > > >> recovery
> > > > > > > > > >> > > > >> > > when
> > > > > > > > > >> > > > >> > > >> both ISR and ELR are empty.
> > > > > > > > > >> > > > >> > > >>
> > > > > > > > > >> > > > >> > > >>
> > > > > > > > > >> > > > >> > > >> On Fri, Sep 15, 2023 at 11:42 AM Calvin
> > Liu
> > > <
> > > > > > > > > >> > > caliu@confluent.io>
> > > > > > > > > >> > > > >> > wrote:
> > > > > > > > > >> > > > >> > > >>
> > > > > > > > > >> > > > >> > > >>> Hi Colin,
> > > > > > > > > >> > > > >> > > >>>
> > > > > > > > > >> > > > >> > > >>> > So, the proposal is that if someone
> > sets
> > > > > > > > > >> > > > >> > > "unclean.leader.election.enable
> > > > > > > > > >> > > > >> > > >>> = true"...
> > > > > > > > > >> > > > >> > > >>>
> > > > > > > > > >> > > > >> > > >>>
> > > > > > > > > >> > > > >> > > >>> The idea is to use one of the
> > > > > > > > > >> unclean.leader.election.enable
> > > > > > > > > >> > > and
> > > > > > > > > >> > > > >> > > >>> unclean.recovery.strategy based on the
> > > > > > > > > >> > > > unclean.recovery.Enabled. A
> > > > > > > > > >> > > > >> > > possible
> > > > > > > > > >> > > > >> > > >>> version can be
> > > > > > > > > >> > > > >> > > >>>
> > > > > > > > > >> > > > >> > > >>> If unclean.recovery.Enabled:
> > > > > > > > > >> > > > >> > > >>>
> > > > > > > > > >> > > > >> > > >>> {
> > > > > > > > > >> > > > >> > > >>>
> > > > > > > > > >> > > > >> > > >>> Check unclean.recovery.strategy. If
> set,
> > > use
> > > > > it.
> > > > > > > > > >> Otherwise,
> > > > > > > > > >> > > > check
> > > > > > > > > >> > > > >> > > >>> unclean.leader.election.enable and
> > > translate
> > > > it
> > > > > > to
> > > > > > > > > >> > > > >> > > >>> unclean.recovery.strategy.
> > > > > > > > > >> > > > >> > > >>>
> > > > > > > > > >> > > > >> > > >>> } else {
> > > > > > > > > >> > > > >> > > >>>
> > > > > > > > > >> > > > >> > > >>> Use unclean.leader.election.enable
> > > > > > > > > >> > > > >> > > >>>
> > > > > > > > > >> > > > >> > > >>> }
> > > > > > > > > >> > > > >> > > >>>
> > > > > > > > > >> > > > >> > > >>>
> > > > > > > > > >> > > > >> > > >>> —--------
> > > > > > > > > >> > > > >> > > >>>
> > > > > > > > > >> > > > >> > > >>> >The configuration key should be
> > > > > > > > > >> > > > >> "unclean.recovery.manager.enabled",
> > > > > > > > > >> > > > >> > > >>> right?
> > > > > > > > > >> > > > >> > > >>>
> > > > > > > > > >> > > > >> > > >>>
> > > > > > > > > >> > > > >> > > >>> I think we have two ways of choosing a
> > > leader
> > > > > > > > > uncleanly,
> > > > > > > > > >> > > unclean
> > > > > > > > > >> > > > >> > leader
> > > > > > > > > >> > > > >> > > >>> election and unclean recovery(log
> > > inspection)
> > > > > and
> > > > > > > we
> > > > > > > > > try
> > > > > > > > > >> to
> > > > > > > > > >> > > > switch
> > > > > > > > > >> > > > >> > > between
> > > > > > > > > >> > > > >> > > >>> them.
> > > > > > > > > >> > > > >> > > >>>
> > > > > > > > > >> > > > >> > > >>> Do you mean we want to develop two ways
> > of
> > > > > > > performing
> > > > > > > > > the
> > > > > > > > > >> > > > unclean
> > > > > > > > > >> > > > >> > > >>> recovery and one of them is using
> > “unclean
> > > > > > recovery
> > > > > > > > > >> > manager”?
> > > > > > > > > >> > > I
> > > > > > > > > >> > > > >> guess
> > > > > > > > > >> > > > >> > > we
> > > > > > > > > >> > > > >> > > >>> haven’t discussed the second way.
> > > > > > > > > >> > > > >> > > >>>
> > > > > > > > > >> > > > >> > > >>>
> > > > > > > > > >> > > > >> > > >>> —-------
> > > > > > > > > >> > > > >> > > >>>
> > > > > > > > > >> > > > >> > > >>> >How do these 4 levels of overrides
> > > interact
> > > > > with
> > > > > > > > your
> > > > > > > > > >> new
> > > > > > > > > >> > > > >> > > >>> configurations?
> > > > > > > > > >> > > > >> > > >>>
> > > > > > > > > >> > > > >> > > >>>
> > > > > > > > > >> > > > >> > > >>> I do notice in the Kraft controller
> code,
> > > the
> > > > > > > method
> > > > > > > > to
> > > > > > > > > >> > check
> > > > > > > > > >> > > > >> whether
> > > > > > > > > >> > > > >> > > >>> perform unclean leader election is hard
> > > coded
> > > > > to
> > > > > > > > false
> > > > > > > > > >> since
> > > > > > > > > >> > > > >> > > >>>
> > 2021(uncleanLeaderElectionEnabledForTopic).
> > > > > Isn’t
> > > > > > > it
> > > > > > > > a
> > > > > > > > > >> good
> > > > > > > > > >> > > > chance
> > > > > > > > > >> > > > >> to
> > > > > > > > > >> > > > >> > > >>> completely deprecate the
> > > > > > > > > unclean.leader.election.enable?
> > > > > > > > > >> We
> > > > > > > > > >> > > > don’t
> > > > > > > > > >> > > > >> > even
> > > > > > > > > >> > > > >> > > have
> > > > > > > > > >> > > > >> > > >>> to worry about the config conversion.
> > > > > > > > > >> > > > >> > > >>>
> > > > > > > > > >> > > > >> > > >>> On the other hand, whatever the
> override
> > > is,
> > > > as
> > > > > > > long
> > > > > > > > as
> > > > > > > > > >> the
> > > > > > > > > >> > > > >> > controller
> > > > > > > > > >> > > > >> > > >>> can have the final effective
> > > > > > > > > >> unclean.leader.election.enable,
> > > > > > > > > >> > > the
> > > > > > > > > >> > > > >> > topic
> > > > > > > > > >> > > > >> > > >>> level config unclean.recovery.strategy,
> > the
> > > > > > cluster
> > > > > > > > > level
> > > > > > > > > >> > > config
> > > > > > > > > >> > > > >> > > >>> unclean.recovery.Enabled, the
> controller
> > > can
> > > > > > > > calculate
> > > > > > > > > >> the
> > > > > > > > > >> > > > correct
> > > > > > > > > >> > > > >> > > methods
> > > > > > > > > >> > > > >> > > >>> to use right?
> > > > > > > > > >> > > > >> > > >>>
> > > > > > > > > >> > > > >> > > >>>
> > > > > > > > > >> > > > >> > > >>> On Fri, Sep 15, 2023 at 10:02 AM Colin
> > > > McCabe <
> > > > > > > > > >> > > > cmccabe@apache.org>
> > > > > > > > > >> > > > >> > > wrote:
> > > > > > > > > >> > > > >> > > >>>
> > > > > > > > > >> > > > >> > > >>>> On Thu, Sep 14, 2023, at 22:23, Calvin
> > Liu
> > > > > > wrote:
> > > > > > > > > >> > > > >> > > >>>> > Hi Colin
> > > > > > > > > >> > > > >> > > >>>> > 1. I think using the new config name
> > is
> > > > more
> > > > > > > > clear.
> > > > > > > > > >> > > > >> > > >>>> >        a. The unclean leader
> election
> > is
> > > > > > > actually
> > > > > > > > > >> removed
> > > > > > > > > >> > > if
> > > > > > > > > >> > > > >> > unclean
> > > > > > > > > >> > > > >> > > >>>> > recovery is in use.
> > > > > > > > > >> > > > >> > > >>>> >        b. Using multiple values in
> > > > > > > > > >> > > > >> unclean.leader.election.enable
> > > > > > > > > >> > > > >> > is
> > > > > > > > > >> > > > >> > > >>>> > confusing and it will be more
> > confusing
> > > > > after
> > > > > > > > people
> > > > > > > > > >> > forget
> > > > > > > > > >> > > > >> about
> > > > > > > > > >> > > > >> > > this
> > > > > > > > > >> > > > >> > > >>>> > discussion.
> > > > > > > > > >> > > > >> > > >>>>
> > > > > > > > > >> > > > >> > > >>>> Hi Calvin,
> > > > > > > > > >> > > > >> > > >>>>
> > > > > > > > > >> > > > >> > > >>>> So, the proposal is that if someone
> sets
> > > > > > > > > >> > > > >> > > "unclean.leader.election.enable
> > > > > > > > > >> > > > >> > > >>>> = true" but then sets one of your new
> > > > > > > > configurations,
> > > > > > > > > >> the
> > > > > > > > > >> > > > value of
> > > > > > > > > >> > > > >> > > >>>> unclean.leader.election.enable is
> > ignored?
> > > > > That
> > > > > > > > seems
> > > > > > > > > >> less
> > > > > > > > > >> > > > clear
> > > > > > > > > >> > > > >> to
> > > > > > > > > >> > > > >> > > me, not
> > > > > > > > > >> > > > >> > > >>>> more. Just in general, having multiple
> > > > > > > configuration
> > > > > > > > > >> keys
> > > > > > > > > >> > to
> > > > > > > > > >> > > > >> control
> > > > > > > > > >> > > > >> > > the
> > > > > > > > > >> > > > >> > > >>>> same thing confuses users. Basically,
> > they
> > > > are
> > > > > > > > sitting
> > > > > > > > > >> at a
> > > > > > > > > >> > > > giant
> > > > > > > > > >> > > > >> > > control
> > > > > > > > > >> > > > >> > > >>>> panel, and some of the levers do
> > nothing.
> > > > > > > > > >> > > > >> > > >>>>
> > > > > > > > > >> > > > >> > > >>>> > 2. Sorry I forgot to mention in the
> > > > response
> > > > > > > that
> > > > > > > > I
> > > > > > > > > >> did
> > > > > > > > > >> > add
> > > > > > > > > >> > > > the
> > > > > > > > > >> > > > >> > > >>>> > unclean.recovery.Enabled flag.
> > > > > > > > > >> > > > >> > > >>>>
> > > > > > > > > >> > > > >> > > >>>> The configuration key should be
> > > > > > > > > >> > > > >> "unclean.recovery.manager.enabled",
> > > > > > > > > >> > > > >> > > >>>> right? Becuase we can do "unclean
> > > recovery"
> > > > > > > without
> > > > > > > > > the
> > > > > > > > > >> > > > manager.
> > > > > > > > > >> > > > >> > > Disabling
> > > > > > > > > >> > > > >> > > >>>> the manager just means we use a
> > different
> > > > > > > mechanism
> > > > > > > > > for
> > > > > > > > > >> > > > recovery.
> > > > > > > > > >> > > > >> > > >>>>
> > > > > > > > > >> > > > >> > > >>>> >        c. Maybe I underestimated the
> > > > > challenge
> > > > > > > of
> > > > > > > > > >> > replacing
> > > > > > > > > >> > > > the
> > > > > > > > > >> > > > >> > > >>>> config. Any
> > > > > > > > > >> > > > >> > > >>>> > implementation problems ahead?
> > > > > > > > > >> > > > >> > > >>>>
> > > > > > > > > >> > > > >> > > >>>> There are four levels of overrides for
> > > > > > > > > >> > > > >> > unclean.leader.election.enable.
> > > > > > > > > >> > > > >> > > >>>>
> > > > > > > > > >> > > > >> > > >>>> 1. static configuration for node.
> > > > > > > > > >> > > > >> > > >>>>     This goes in the configuration
> file,
> > > > > > typically
> > > > > > > > > named
> > > > > > > > > >> > > > >> > > >>>> server.properties
> > > > > > > > > >> > > > >> > > >>>>
> > > > > > > > > >> > > > >> > > >>>> 2. dynamic configuration for node
> > default
> > > > > > > > > >> > > > >> > > >>>>   ConfigResource(type=BROKER, name="")
> > > > > > > > > >> > > > >> > > >>>>
> > > > > > > > > >> > > > >> > > >>>> 3. dynamic configuration for node
> > > > > > > > > >> > > > >> > > >>>>   ConfigResource(type=BROKER,
> > > > name=<controller
> > > > > > > id>)
> > > > > > > > > >> > > > >> > > >>>>
> > > > > > > > > >> > > > >> > > >>>> 4. dynamic configuration for topic
> > > > > > > > > >> > > > >> > > >>>>   ConfigResource(type=TOPIC,
> > > > > name=<topic-name>)
> > > > > > > > > >> > > > >> > > >>>>
> > > > > > > > > >> > > > >> > > >>>> How do these 4 levels of overrides
> > > interact
> > > > > with
> > > > > > > > your
> > > > > > > > > >> new
> > > > > > > > > >> > > > >> > > >>>> configurations? If the new
> > configurations
> > > > > > dominate
> > > > > > > > > over
> > > > > > > > > >> the
> > > > > > > > > >> > > old
> > > > > > > > > >> > > > >> > ones,
> > > > > > > > > >> > > > >> > > it
> > > > > > > > > >> > > > >> > > >>>> seems like this will get a lot more
> > > > confusing
> > > > > to
> > > > > > > > > >> implement
> > > > > > > > > >> > > (and
> > > > > > > > > >> > > > >> also
> > > > > > > > > >> > > > >> > > to
> > > > > > > > > >> > > > >> > > >>>> use.)
> > > > > > > > > >> > > > >> > > >>>>
> > > > > > > > > >> > > > >> > > >>>> Again, I'd recommend just adding some
> > new
> > > > > values
> > > > > > > to
> > > > > > > > > >> > > > >> > > >>>> unclean.leader.election.enable. It's
> > > simple
> > > > > and
> > > > > > > will
> > > > > > > > > >> > prevent
> > > > > > > > > >> > > > user
> > > > > > > > > >> > > > >> > > confusion
> > > > > > > > > >> > > > >> > > >>>> (as well as developer confusion.)
> > > > > > > > > >> > > > >> > > >>>>
> > > > > > > > > >> > > > >> > > >>>> best,
> > > > > > > > > >> > > > >> > > >>>> Colin
> > > > > > > > > >> > > > >> > > >>>>
> > > > > > > > > >> > > > >> > > >>>>
> > > > > > > > > >> > > > >> > > >>>> > 3. About the admin client, I
> > mentioned 3
> > > > > > changes
> > > > > > > > in
> > > > > > > > > >> the
> > > > > > > > > >> > > > client.
> > > > > > > > > >> > > > >> > > >>>> Anything
> > > > > > > > > >> > > > >> > > >>>> > else I missed in the KIP?
> > > > > > > > > >> > > > >> > > >>>> >       a. The client will switch to
> > using
> > > > the
> > > > > > new
> > > > > > > > RPC
> > > > > > > > > >> > > instead
> > > > > > > > > >> > > > of
> > > > > > > > > >> > > > >> > > >>>> > MetadataRequest for the topics.
> > > > > > > > > >> > > > >> > > >>>> >       b. The TopicPartitionInfo used
> > in
> > > > > > > > > >> TopicDescription
> > > > > > > > > >> > > > needs
> > > > > > > > > >> > > > >> to
> > > > > > > > > >> > > > >> > > add
> > > > > > > > > >> > > > >> > > >>>> new
> > > > > > > > > >> > > > >> > > >>>> > fields related to the ELR.
> > > > > > > > > >> > > > >> > > >>>> >       c. The outputs will add the
> ELR
> > > > > related
> > > > > > > > > fields.
> > > > > > > > > >> > > > >> > > >>>> >
> > > > > > > > > >> > > > >> > > >>>> > On Thu, Sep 14, 2023 at 9:19 PM
> Colin
> > > > > McCabe <
> > > > > > > > > >> > > > >> cmccabe@apache.org>
> > > > > > > > > >> > > > >> > > >>>> wrote:
> > > > > > > > > >> > > > >> > > >>>> >
> > > > > > > > > >> > > > >> > > >>>> >> Hi Calvin,
> > > > > > > > > >> > > > >> > > >>>> >>
> > > > > > > > > >> > > > >> > > >>>> >> Thanks for the changes.
> > > > > > > > > >> > > > >> > > >>>> >>
> > > > > > > > > >> > > > >> > > >>>> >> 1. Earlier I commented that
> creating
> > > > > > > > > >> > > > >> "unclean.recovery.strategy "
> > > > > > > > > >> > > > >> > > is
> > > > > > > > > >> > > > >> > > >>>> not
> > > > > > > > > >> > > > >> > > >>>> >> necessary, and we can just reuse
> the
> > > > > existing
> > > > > > > > > >> > > > >> > > >>>> >> "unclean.leader.election.enable"
> > > > > > configuration
> > > > > > > > key.
> > > > > > > > > >> > Let's
> > > > > > > > > >> > > > >> discuss
> > > > > > > > > >> > > > >> > > >>>> that.
> > > > > > > > > >> > > > >> > > >>>> >>
> > > > > > > > > >> > > > >> > > >>>> >> 2.I also don't understand why you
> > > didn't
> > > > > add
> > > > > > a
> > > > > > > > > >> > > > configuration to
> > > > > > > > > >> > > > >> > > >>>> enable or
> > > > > > > > > >> > > > >> > > >>>> >> disable the Unclean Recovery
> Manager.
> > > > This
> > > > > > > seems
> > > > > > > > > >> like a
> > > > > > > > > >> > > very
> > > > > > > > > >> > > > >> > simple
> > > > > > > > > >> > > > >> > > >>>> way to
> > > > > > > > > >> > > > >> > > >>>> >> handle the staging issue which we
> > > > > discussed.
> > > > > > > The
> > > > > > > > > URM
> > > > > > > > > >> can
> > > > > > > > > >> > > > just
> > > > > > > > > >> > > > >> be
> > > > > > > > > >> > > > >> > > >>>> turned off
> > > > > > > > > >> > > > >> > > >>>> >> until it is production ready. Let's
> > > > discuss
> > > > > > > this.
> > > > > > > > > >> > > > >> > > >>>> >>
> > > > > > > > > >> > > > >> > > >>>> >> 3. You still need to describe the
> > > changes
> > > > > to
> > > > > > > > > >> AdminClient
> > > > > > > > > >> > > > that
> > > > > > > > > >> > > > >> are
> > > > > > > > > >> > > > >> > > >>>> needed
> > > > > > > > > >> > > > >> > > >>>> >> to use DescribeTopicRequest.
> > > > > > > > > >> > > > >> > > >>>> >>
> > > > > > > > > >> > > > >> > > >>>> >> Keep at it. It's looking better. :)
> > > > > > > > > >> > > > >> > > >>>> >>
> > > > > > > > > >> > > > >> > > >>>> >> best,
> > > > > > > > > >> > > > >> > > >>>> >> Colin
> > > > > > > > > >> > > > >> > > >>>> >>
> > > > > > > > > >> > > > >> > > >>>> >>
> > > > > > > > > >> > > > >> > > >>>> >> On Thu, Sep 14, 2023, at 11:03,
> > Calvin
> > > > Liu
> > > > > > > wrote:
> > > > > > > > > >> > > > >> > > >>>> >> > Hi Colin
> > > > > > > > > >> > > > >> > > >>>> >> > Thanks for the comments!
> > > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > > >> > > > >> > > >>>> >> > I did the following changes
> > > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >    1.
> > > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >    Simplified the API spec
> section
> > to
> > > > > only
> > > > > > > > > include
> > > > > > > > > >> the
> > > > > > > > > >> > > > diff.
> > > > > > > > > >> > > > >> > > >>>> >> >    2.
> > > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >    Reordered the HWM requirement
> > > > section.
> > > > > > > > > >> > > > >> > > >>>> >> >    3.
> > > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >    Removed the URM implementation
> > > > details
> > > > > > to
> > > > > > > > keep
> > > > > > > > > >> the
> > > > > > > > > >> > > > >> necessary
> > > > > > > > > >> > > > >> > > >>>> >> >    characteristics to perform the
> > > > unclean
> > > > > > > > > recovery.
> > > > > > > > > >> > > > >> > > >>>> >> >    1.
> > > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >       When to perform the unclean
> > > > > recovery
> > > > > > > > > >> > > > >> > > >>>> >> >       2.
> > > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >       Under different config, how
> > the
> > > > > > unclean
> > > > > > > > > >> recovery
> > > > > > > > > >> > > > finds
> > > > > > > > > >> > > > >> > the
> > > > > > > > > >> > > > >> > > >>>> leader.
> > > > > > > > > >> > > > >> > > >>>> >> >       3.
> > > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >       How the config
> > > > > > > > > unclean.leader.election.enable
> > > > > > > > > >> > and
> > > > > > > > > >> > > > >> > > >>>> >> >       unclean.recovery.strategy
> are
> > > > > > converted
> > > > > > > > > when
> > > > > > > > > >> > users
> > > > > > > > > >> > > > >> > > >>>> enable/disable
> > > > > > > > > >> > > > >> > > >>>> >> the
> > > > > > > > > >> > > > >> > > >>>> >> >       unclean recovery.
> > > > > > > > > >> > > > >> > > >>>> >> >       4.
> > > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >    More details about how we
> change
> > > > admin
> > > > > > > > client.
> > > > > > > > > >> > > > >> > > >>>> >> >    5.
> > > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >    API limits on the
> > > > > > GetReplicaLogInfoRequest
> > > > > > > > and
> > > > > > > > > >> > > > >> > > >>>> DescribeTopicRequest.
> > > > > > > > > >> > > > >> > > >>>> >> >    6.
> > > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >    Two metrics added
> > > > > > > > > >> > > > >> > > >>>> >> >    1.
> > > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > > >> > > >
> > Kafka.controller.global_under_min_isr_partition_count
> > > > > > > > > >> > > > >> > > >>>> >> >       2.
> > > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > > >>  kafka.controller.unclean_recovery_finished_count
> > > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > > >> > > > >> > > >>>> >> > On Wed, Sep 13, 2023 at 10:46 AM
> > > Colin
> > > > > > > McCabe <
> > > > > > > > > >> > > > >> > > cmccabe@apache.org>
> > > > > > > > > >> > > > >> > > >>>> >> wrote:
> > > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >> On Tue, Sep 12, 2023, at 17:21,
> > > Calvin
> > > > > Liu
> > > > > > > > > wrote:
> > > > > > > > > >> > > > >> > > >>>> >> >> > Hi Colin
> > > > > > > > > >> > > > >> > > >>>> >> >> > Thanks for the comments!
> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> Hi Calvin,
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> Thanks again for the KIP.
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> One meta-comment: it's usually
> > > better
> > > > to
> > > > > > > just
> > > > > > > > > do a
> > > > > > > > > >> > diff
> > > > > > > > > >> > > > on a
> > > > > > > > > >> > > > >> > > >>>> message
> > > > > > > > > >> > > > >> > > >>>> >> spec
> > > > > > > > > >> > > > >> > > >>>> >> >> file or java file if you're
> > > including
> > > > > > > changes
> > > > > > > > to
> > > > > > > > > >> it
> > > > > > > > > >> > in
> > > > > > > > > >> > > > the
> > > > > > > > > >> > > > >> > KIP.
> > > > > > > > > >> > > > >> > > >>>> This is
> > > > > > > > > >> > > > >> > > >>>> >> >> easier to read than looking for
> > "new
> > > > > > fields
> > > > > > > > > begin"
> > > > > > > > > >> > etc.
> > > > > > > > > >> > > > in
> > > > > > > > > >> > > > >> the
> > > > > > > > > >> > > > >> > > >>>> text, and
> > > > > > > > > >> > > > >> > > >>>> >> >> gracefully handles the case
> where
> > > > > existing
> > > > > > > > > fields
> > > > > > > > > >> > were
> > > > > > > > > >> > > > >> > changed.
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> > Rewrite the Additional High
> > > > Watermark
> > > > > > > > > >> advancement
> > > > > > > > > >> > > > >> > requirement
> > > > > > > > > >> > > > >> > > >>>> >> >> > There was feedback on this
> > section
> > > > > that
> > > > > > > some
> > > > > > > > > >> > readers
> > > > > > > > > >> > > > may
> > > > > > > > > >> > > > >> not
> > > > > > > > > >> > > > >> > > be
> > > > > > > > > >> > > > >> > > >>>> >> familiar
> > > > > > > > > >> > > > >> > > >>>> >> >> > with HWM and Ack=0,1,all
> > requests.
> > > > > This
> > > > > > > can
> > > > > > > > > help
> > > > > > > > > >> > them
> > > > > > > > > >> > > > >> > > understand
> > > > > > > > > >> > > > >> > > >>>> the
> > > > > > > > > >> > > > >> > > >>>> >> >> > proposal. I will rewrite this
> > part
> > > > for
> > > > > > > more
> > > > > > > > > >> > > > readability.
> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> To be clear, I wasn't suggesting
> > > > > dropping
> > > > > > > > either
> > > > > > > > > >> > > > section. I
> > > > > > > > > >> > > > >> > > agree
> > > > > > > > > >> > > > >> > > >>>> that
> > > > > > > > > >> > > > >> > > >>>> >> >> they add useful background. I
> was
> > > just
> > > > > > > > > suggesting
> > > > > > > > > >> > that
> > > > > > > > > >> > > we
> > > > > > > > > >> > > > >> > should
> > > > > > > > > >> > > > >> > > >>>> discuss
> > > > > > > > > >> > > > >> > > >>>> >> >> the "acks" setting AFTER
> > discussing
> > > > the
> > > > > > new
> > > > > > > > high
> > > > > > > > > >> > > > watermark
> > > > > > > > > >> > > > >> > > >>>> advancement
> > > > > > > > > >> > > > >> > > >>>> >> >> conditions. We also should
> discuss
> > > > > acks=0.
> > > > > > > > While
> > > > > > > > > >> it
> > > > > > > > > >> > > isn't
> > > > > > > > > >> > > > >> > > >>>> conceptually
> > > > > > > > > >> > > > >> > > >>>> >> much
> > > > > > > > > >> > > > >> > > >>>> >> >> different than acks=1 here, its
> > > > omission
> > > > > > > from
> > > > > > > > > this
> > > > > > > > > >> > > > section
> > > > > > > > > >> > > > >> is
> > > > > > > > > >> > > > >> > > >>>> confusing.
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> > Unclean recovery
> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >> > The plan is to replace the
> > > > > > > > > >> > > > unclean.leader.election.enable
> > > > > > > > > >> > > > >> > with
> > > > > > > > > >> > > > >> > > >>>> >> >> > unclean.recovery.strategy. If
> > the
> > > > > > Unclean
> > > > > > > > > >> Recovery
> > > > > > > > > >> > is
> > > > > > > > > >> > > > >> > enabled
> > > > > > > > > >> > > > >> > > >>>> then it
> > > > > > > > > >> > > > >> > > >>>> >> >> deals
> > > > > > > > > >> > > > >> > > >>>> >> >> > with the three options in the
> > > > > > > > > >> > > > unclean.recovery.strategy.
> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >> > Let’s refine the Unclean
> > Recovery.
> > > > We
> > > > > > have
> > > > > > > > > >> already
> > > > > > > > > >> > > > taken a
> > > > > > > > > >> > > > >> > > lot of
> > > > > > > > > >> > > > >> > > >>>> >> >> > suggestions and I hope to
> > enhance
> > > > the
> > > > > > > > > >> durability of
> > > > > > > > > >> > > > Kafka
> > > > > > > > > >> > > > >> to
> > > > > > > > > >> > > > >> > > the
> > > > > > > > > >> > > > >> > > >>>> next
> > > > > > > > > >> > > > >> > > >>>> >> >> level
> > > > > > > > > >> > > > >> > > >>>> >> >> > with this KIP.
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> I am OK with doing the unclean
> > > leader
> > > > > > > recovery
> > > > > > > > > >> > > > improvements
> > > > > > > > > >> > > > >> in
> > > > > > > > > >> > > > >> > > >>>> this KIP.
> > > > > > > > > >> > > > >> > > >>>> >> >> However, I think we need to
> really
> > > > work
> > > > > on
> > > > > > > the
> > > > > > > > > >> > > > configuration
> > > > > > > > > >> > > > >> > > >>>> settings.
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> Configuration overrides are
> often
> > > > quite
> > > > > > > messy.
> > > > > > > > > For
> > > > > > > > > >> > > > example,
> > > > > > > > > >> > > > >> > the
> > > > > > > > > >> > > > >> > > >>>> cases
> > > > > > > > > >> > > > >> > > >>>> >> >> where we have log.roll.hours and
> > > > > > > > > >> log.roll.segment.ms
> > > > > > > > > >> > ,
> > > > > > > > > >> > > > the
> > > > > > > > > >> > > > >> > user
> > > > > > > > > >> > > > >> > > >>>> has to
> > > > > > > > > >> > > > >> > > >>>> >> >> remember which one takes
> > precedence,
> > > > and
> > > > > > it
> > > > > > > is
> > > > > > > > > not
> > > > > > > > > >> > > > obvious.
> > > > > > > > > >> > > > >> > So,
> > > > > > > > > >> > > > >> > > >>>> rather
> > > > > > > > > >> > > > >> > > >>>> >> than
> > > > > > > > > >> > > > >> > > >>>> >> >> creating a new configuration,
> why
> > > not
> > > > > add
> > > > > > > > > >> additional
> > > > > > > > > >> > > > values
> > > > > > > > > >> > > > >> to
> > > > > > > > > >> > > > >> > > >>>> >> >>
> "unclean.leader.election.enable"?
> > I
> > > > > think
> > > > > > > this
> > > > > > > > > >> will
> > > > > > > > > >> > be
> > > > > > > > > >> > > > >> simpler
> > > > > > > > > >> > > > >> > > for
> > > > > > > > > >> > > > >> > > >>>> >> people
> > > > > > > > > >> > > > >> > > >>>> >> >> to understand, and simpler in
> the
> > > code
> > > > > as
> > > > > > > > well.
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> What if we continued to use
> > > > > > > > > >> > > > "unclean.leader.election.enable"
> > > > > > > > > >> > > > >> > but
> > > > > > > > > >> > > > >> > > >>>> >> extended
> > > > > > > > > >> > > > >> > > >>>> >> >> it so that it took a string?
> Then
> > > the
> > > > > > string
> > > > > > > > > could
> > > > > > > > > >> > have
> > > > > > > > > >> > > > >> these
> > > > > > > > > >> > > > >> > > >>>> values:
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> never
> > > > > > > > > >> > > > >> > > >>>> >> >>     never automatically do an
> > > unclean
> > > > > > leader
> > > > > > > > > >> election
> > > > > > > > > >> > > > under
> > > > > > > > > >> > > > >> > any
> > > > > > > > > >> > > > >> > > >>>> >> conditions
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> false / default
> > > > > > > > > >> > > > >> > > >>>> >> >>     only do an unclean leader
> > > election
> > > > > if
> > > > > > > > there
> > > > > > > > > >> may
> > > > > > > > > >> > be
> > > > > > > > > >> > > > >> > possible
> > > > > > > > > >> > > > >> > > >>>> data
> > > > > > > > > >> > > > >> > > >>>> >> loss
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> true / always
> > > > > > > > > >> > > > >> > > >>>> >> >>     always do an unclean leader
> > > > election
> > > > > > if
> > > > > > > we
> > > > > > > > > >> can't
> > > > > > > > > >> > > > >> > immediately
> > > > > > > > > >> > > > >> > > >>>> elect a
> > > > > > > > > >> > > > >> > > >>>> >> >> leader
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> It's a bit awkward that false
> maps
> > > to
> > > > > > > default
> > > > > > > > > >> rather
> > > > > > > > > >> > > > than to
> > > > > > > > > >> > > > >> > > >>>> never. But
> > > > > > > > > >> > > > >> > > >>>> >> >> this awkwardness exists if we
> use
> > > two
> > > > > > > > different
> > > > > > > > > >> > > > >> configuration
> > > > > > > > > >> > > > >> > > keys
> > > > > > > > > >> > > > >> > > >>>> as
> > > > > > > > > >> > > > >> > > >>>> >> well.
> > > > > > > > > >> > > > >> > > >>>> >> >> The reason for the awkwardness
> is
> > > that
> > > > > we
> > > > > > > > simply
> > > > > > > > > >> > don't
> > > > > > > > > >> > > > want
> > > > > > > > > >> > > > >> > most
> > > > > > > > > >> > > > >> > > >>>> of the
> > > > > > > > > >> > > > >> > > >>>> >> >> people currently setting
> > > > > > > > > >> > > > >> unclean.leader.election.enable=false
> > > > > > > > > >> > > > >> > to
> > > > > > > > > >> > > > >> > > >>>> get the
> > > > > > > > > >> > > > >> > > >>>> >> >> "never" behavior. We have to
> bite
> > > that
> > > > > > > bullet.
> > > > > > > > > >> Better
> > > > > > > > > >> > > to
> > > > > > > > > >> > > > be
> > > > > > > > > >> > > > >> > > clear
> > > > > > > > > >> > > > >> > > >>>> and
> > > > > > > > > >> > > > >> > > >>>> >> >> explicit than hide it.
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> Another thing that's a bit
> awkward
> > > is
> > > > > > having
> > > > > > > > two
> > > > > > > > > >> > > > different
> > > > > > > > > >> > > > >> > ways
> > > > > > > > > >> > > > >> > > to
> > > > > > > > > >> > > > >> > > >>>> do
> > > > > > > > > >> > > > >> > > >>>> >> >> unclean leader election
> specified
> > in
> > > > the
> > > > > > > KIP.
> > > > > > > > > You
> > > > > > > > > >> > > > descirbe
> > > > > > > > > >> > > > >> two
> > > > > > > > > >> > > > >> > > >>>> methods:
> > > > > > > > > >> > > > >> > > >>>> >> the
> > > > > > > > > >> > > > >> > > >>>> >> >> simple "choose the last leader"
> > > > method,
> > > > > > and
> > > > > > > > the
> > > > > > > > > >> > > "unclean
> > > > > > > > > >> > > > >> > > recovery
> > > > > > > > > >> > > > >> > > >>>> >> manager"
> > > > > > > > > >> > > > >> > > >>>> >> >> method. I understand why you did
> > it
> > > > this
> > > > > > way
> > > > > > > > --
> > > > > > > > > >> > "choose
> > > > > > > > > >> > > > the
> > > > > > > > > >> > > > >> > last
> > > > > > > > > >> > > > >> > > >>>> >> leader" is
> > > > > > > > > >> > > > >> > > >>>> >> >> simple, and will help us deliver
> > an
> > > > > > > > > implementation
> > > > > > > > > >> > > > quickly,
> > > > > > > > > >> > > > >> > > while
> > > > > > > > > >> > > > >> > > >>>> the
> > > > > > > > > >> > > > >> > > >>>> >> URM
> > > > > > > > > >> > > > >> > > >>>> >> >> is preferable in the long term.
> My
> > > > > > > suggestion
> > > > > > > > > >> here is
> > > > > > > > > >> > > to
> > > > > > > > > >> > > > >> > > separate
> > > > > > > > > >> > > > >> > > >>>> the
> > > > > > > > > >> > > > >> > > >>>> >> >> decision of HOW to do unclean
> > leader
> > > > > > > election
> > > > > > > > > from
> > > > > > > > > >> > the
> > > > > > > > > >> > > > >> > decision
> > > > > > > > > >> > > > >> > > of
> > > > > > > > > >> > > > >> > > >>>> WHEN
> > > > > > > > > >> > > > >> > > >>>> >> to
> > > > > > > > > >> > > > >> > > >>>> >> >> do it.
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> So in other words, have
> > > > > > > > > >> > > "unclean.leader.election.enable"
> > > > > > > > > >> > > > >> > specify
> > > > > > > > > >> > > > >> > > >>>> when we
> > > > > > > > > >> > > > >> > > >>>> >> >> do unclean leader election, and
> > > have a
> > > > > new
> > > > > > > > > >> > > configuration
> > > > > > > > > >> > > > >> like
> > > > > > > > > >> > > > >> > > >>>> >> >>
> "unclean.recovery.manager.enable"
> > to
> > > > > > > determine
> > > > > > > > > if
> > > > > > > > > >> we
> > > > > > > > > >> > > use
> > > > > > > > > >> > > > the
> > > > > > > > > >> > > > >> > > URM.
> > > > > > > > > >> > > > >> > > >>>> >> >> Presumably the URM will take
> some
> > > time
> > > > > to
> > > > > > > get
> > > > > > > > > >> fully
> > > > > > > > > >> > > > stable,
> > > > > > > > > >> > > > >> so
> > > > > > > > > >> > > > >> > > >>>> this can
> > > > > > > > > >> > > > >> > > >>>> >> >> default to false for a while,
> and
> > we
> > > > can
> > > > > > > flip
> > > > > > > > > the
> > > > > > > > > >> > > > default to
> > > > > > > > > >> > > > >> > > true
> > > > > > > > > >> > > > >> > > >>>> when
> > > > > > > > > >> > > > >> > > >>>> >> we
> > > > > > > > > >> > > > >> > > >>>> >> >> feel ready.
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> The URM is somewhat
> > under-described
> > > > > here.
> > > > > > I
> > > > > > > > > think
> > > > > > > > > >> we
> > > > > > > > > >> > > > need a
> > > > > > > > > >> > > > >> > few
> > > > > > > > > >> > > > >> > > >>>> >> >> configurations here for it. For
> > > > example,
> > > > > > we
> > > > > > > > > need a
> > > > > > > > > >> > > > >> > > configuration to
> > > > > > > > > >> > > > >> > > >>>> >> specify
> > > > > > > > > >> > > > >> > > >>>> >> >> how long it should wait for a
> > broker
> > > > to
> > > > > > > > respond
> > > > > > > > > to
> > > > > > > > > >> > its
> > > > > > > > > >> > > > RPCs
> > > > > > > > > >> > > > >> > > before
> > > > > > > > > >> > > > >> > > >>>> >> moving
> > > > > > > > > >> > > > >> > > >>>> >> >> on. We also need to understand
> how
> > > the
> > > > > URM
> > > > > > > > > >> interacts
> > > > > > > > > >> > > with
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > unclean.leader.election.enable=always. I
> > > > > > > > assume
> > > > > > > > > >> that
> > > > > > > > > >> > > with
> > > > > > > > > >> > > > >> > > "always"
> > > > > > > > > >> > > > >> > > >>>> we
> > > > > > > > > >> > > > >> > > >>>> >> will
> > > > > > > > > >> > > > >> > > >>>> >> >> just unconditionally use the URM
> > > > rather
> > > > > > than
> > > > > > > > > >> choosing
> > > > > > > > > >> > > > >> > randomly.
> > > > > > > > > >> > > > >> > > >>>> But this
> > > > > > > > > >> > > > >> > > >>>> >> >> should be spelled out in the
> KIP.
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >> > DescribeTopicRequest
> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >> >    1.
> > > > > > > > > >> > > > >> > > >>>> >> >> >    Yes, the plan is to replace
> > the
> > > > > > > > > >> MetadataRequest
> > > > > > > > > >> > > with
> > > > > > > > > >> > > > >> the
> > > > > > > > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest for
> the
> > > > admin
> > > > > > > > clients.
> > > > > > > > > >> Will
> > > > > > > > > >> > > > check
> > > > > > > > > >> > > > >> > the
> > > > > > > > > >> > > > >> > > >>>> details.
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> Sounds good. But as I said, you
> > need
> > > > to
> > > > > > > > specify
> > > > > > > > > >> how
> > > > > > > > > >> > > > >> > AdminClient
> > > > > > > > > >> > > > >> > > >>>> >> interacts
> > > > > > > > > >> > > > >> > > >>>> >> >> with the new request. This will
> > > > involve
> > > > > > > adding
> > > > > > > > > >> some
> > > > > > > > > >> > > > fields
> > > > > > > > > >> > > > >> to
> > > > > > > > > >> > > > >> > > >>>> >> >> TopicDescription.java. And you
> > need
> > > to
> > > > > > > specify
> > > > > > > > > the
> > > > > > > > > >> > > > changes
> > > > > > > > > >> > > > >> to
> > > > > > > > > >> > > > >> > > the
> > > > > > > > > >> > > > >> > > >>>> >> >> kafka-topics.sh command line
> tool.
> > > > > > Otherwise
> > > > > > > > we
> > > > > > > > > >> > cannot
> > > > > > > > > >> > > > use
> > > > > > > > > >> > > > >> the
> > > > > > > > > >> > > > >> > > >>>> tool to
> > > > > > > > > >> > > > >> > > >>>> >> see
> > > > > > > > > >> > > > >> > > >>>> >> >> the new information.
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> The new requests,
> > > DescribeTopicRequest
> > > > > and
> > > > > > > > > >> > > > >> > > >>>> GetReplicaLogInfoRequest,
> > > > > > > > > >> > > > >> > > >>>> >> need
> > > > > > > > > >> > > > >> > > >>>> >> >> to have limits placed on them so
> > > that
> > > > > > their
> > > > > > > > size
> > > > > > > > > >> > can't
> > > > > > > > > >> > > be
> > > > > > > > > >> > > > >> > > >>>> infinite. We
> > > > > > > > > >> > > > >> > > >>>> >> >> don't want to propagate the
> > current
> > > > > > problems
> > > > > > > > of
> > > > > > > > > >> > > > >> > MetadataRequest,
> > > > > > > > > >> > > > >> > > >>>> where
> > > > > > > > > >> > > > >> > > >>>> >> >> clients can request massive
> > > responses
> > > > > that
> > > > > > > can
> > > > > > > > > >> mess
> > > > > > > > > >> > up
> > > > > > > > > >> > > > the
> > > > > > > > > >> > > > >> JVM
> > > > > > > > > >> > > > >> > > when
> > > > > > > > > >> > > > >> > > >>>> >> handled.
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> Adding limits is simple for
> > > > > > > > > >> GetReplicaLogInfoRequest
> > > > > > > > > >> > --
> > > > > > > > > >> > > > we
> > > > > > > > > >> > > > >> can
> > > > > > > > > >> > > > >> > > >>>> just say
> > > > > > > > > >> > > > >> > > >>>> >> >> that only 2000 partitions at a
> > time
> > > > can
> > > > > be
> > > > > > > > > >> requested.
> > > > > > > > > >> > > For
> > > > > > > > > >> > > > >> > > >>>> >> >> DescribeTopicRequest we can
> > probably
> > > > > just
> > > > > > > > limit
> > > > > > > > > >> to 20
> > > > > > > > > >> > > > topics
> > > > > > > > > >> > > > >> > or
> > > > > > > > > >> > > > >> > > >>>> >> something
> > > > > > > > > >> > > > >> > > >>>> >> >> like that, to avoid the
> complexity
> > > of
> > > > > > doing
> > > > > > > > > >> > pagination
> > > > > > > > > >> > > in
> > > > > > > > > >> > > > >> this
> > > > > > > > > >> > > > >> > > KIP.
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> >    2.
> > > > > > > > > >> > > > >> > > >>>> >> >> >    I can let the broker load
> the
> > > ELR
> > > > > > info
> > > > > > > so
> > > > > > > > > >> that
> > > > > > > > > >> > > they
> > > > > > > > > >> > > > can
> > > > > > > > > >> > > > >> > > serve
> > > > > > > > > >> > > > >> > > >>>> the
> > > > > > > > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest as
> well.
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> Yes, it's fine to add to
> > > > MetadataCache.
> > > > > In
> > > > > > > > fact,
> > > > > > > > > >> > you'll
> > > > > > > > > >> > > > be
> > > > > > > > > >> > > > >> > > loading
> > > > > > > > > >> > > > >> > > >>>> it
> > > > > > > > > >> > > > >> > > >>>> >> >> anyway once it's added to
> > > > > PartitionImage.
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> >    3.
> > > > > > > > > >> > > > >> > > >>>> >> >> >    Yeah, it does not make
> sense
> > to
> > > > > have
> > > > > > > the
> > > > > > > > > >> topic
> > > > > > > > > >> > id
> > > > > > > > > >> > > if
> > > > > > > > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest is
> only
> > > used
> > > > > by
> > > > > > > the
> > > > > > > > > >> admin
> > > > > > > > > >> > > > client.
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> OK. That makes things simpler.
> We
> > > can
> > > > > > always
> > > > > > > > > >> create a
> > > > > > > > > >> > > new
> > > > > > > > > >> > > > >> API
> > > > > > > > > >> > > > >> > > later
> > > > > > > > > >> > > > >> > > >>>> >> >> (hopefully not in this KIP!) to
> > > query
> > > > by
> > > > > > > topic
> > > > > > > > > ID.
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >> > Metrics
> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >> > As for overall cluster health
> > > > > metrics, I
> > > > > > > > think
> > > > > > > > > >> > > > >> under-min-ISR
> > > > > > > > > >> > > > >> > > is
> > > > > > > > > >> > > > >> > > >>>> still
> > > > > > > > > >> > > > >> > > >>>> >> a
> > > > > > > > > >> > > > >> > > >>>> >> >> > useful one. ELR is more like a
> > > > safety
> > > > > > > belt.
> > > > > > > > > When
> > > > > > > > > >> > the
> > > > > > > > > >> > > > ELR
> > > > > > > > > >> > > > >> is
> > > > > > > > > >> > > > >> > > >>>> used, the
> > > > > > > > > >> > > > >> > > >>>> >> >> > cluster availability has
> already
> > > > been
> > > > > > > > > impacted.
> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >> > Maybe we can have a metric to
> > > count
> > > > > the
> > > > > > > > > >> partitions
> > > > > > > > > >> > > that
> > > > > > > > > >> > > > >> > > sum(ISR,
> > > > > > > > > >> > > > >> > > >>>> ELR)
> > > > > > > > > >> > > > >> > > >>>> >> <
> > > > > > > > > >> > > > >> > > >>>> >> >> min
> > > > > > > > > >> > > > >> > > >>>> >> >> > ISR. What do you think?
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> How about:
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> A.  a metric for the totoal
> number
> > > of
> > > > > > > > > >> under-min-isr
> > > > > > > > > >> > > > >> > partitions?
> > > > > > > > > >> > > > >> > > We
> > > > > > > > > >> > > > >> > > >>>> don't
> > > > > > > > > >> > > > >> > > >>>> >> >> have that in Apache Kafka at the
> > > > moment.
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> B. a metric for the number of
> > > unclean
> > > > > > leader
> > > > > > > > > >> > elections
> > > > > > > > > >> > > we
> > > > > > > > > >> > > > >> did
> > > > > > > > > >> > > > >> > > (for
> > > > > > > > > >> > > > >> > > >>>> >> >> simplicity, it can reset to 0 on
> > > > > > controller
> > > > > > > > > >> restart:
> > > > > > > > > >> > we
> > > > > > > > > >> > > > >> expect
> > > > > > > > > >> > > > >> > > >>>> people to
> > > > > > > > > >> > > > >> > > >>>> >> >> monitor the change over time
> > anyway)
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> best,
> > > > > > > > > >> > > > >> > > >>>> >> >> Colin
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >> > Yeah, for the ongoing unclean
> > > > > > recoveries,
> > > > > > > > the
> > > > > > > > > >> > > > controller
> > > > > > > > > >> > > > >> can
> > > > > > > > > >> > > > >> > > >>>> keep an
> > > > > > > > > >> > > > >> > > >>>> >> >> > accurate count through
> failover
> > > > > because
> > > > > > > > > >> partition
> > > > > > > > > >> > > > >> > registration
> > > > > > > > > >> > > > >> > > >>>> can
> > > > > > > > > >> > > > >> > > >>>> >> >> indicate
> > > > > > > > > >> > > > >> > > >>>> >> >> > whether a recovery is needed.
> > > > However,
> > > > > > for
> > > > > > > > the
> > > > > > > > > >> > > happened
> > > > > > > > > >> > > > >> > ones,
> > > > > > > > > >> > > > >> > > >>>> unless
> > > > > > > > > >> > > > >> > > >>>> >> we
> > > > > > > > > >> > > > >> > > >>>> >> >> > want to persist the number
> > > > somewhere,
> > > > > we
> > > > > > > can
> > > > > > > > > >> only
> > > > > > > > > >> > > > figure
> > > > > > > > > >> > > > >> it
> > > > > > > > > >> > > > >> > > out
> > > > > > > > > >> > > > >> > > >>>> from
> > > > > > > > > >> > > > >> > > >>>> >> the
> > > > > > > > > >> > > > >> > > >>>> >> >> > log.
> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >> > On Tue, Sep 12, 2023 at
> 3:16 PM
> > > > Colin
> > > > > > > > McCabe <
> > > > > > > > > >> > > > >> > > cmccabe@apache.org
> > > > > > > > > >> > > > >> > > >>>> >
> > > > > > > > > >> > > > >> > > >>>> >> wrote:
> > > > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >> >> Also, we should have metrics
> > that
> > > > > show
> > > > > > > what
> > > > > > > > > is
> > > > > > > > > >> > going
> > > > > > > > > >> > > > on
> > > > > > > > > >> > > > >> > with
> > > > > > > > > >> > > > >> > > >>>> regard
> > > > > > > > > >> > > > >> > > >>>> >> to
> > > > > > > > > >> > > > >> > > >>>> >> >> the
> > > > > > > > > >> > > > >> > > >>>> >> >> >> eligible replica set. I'm not
> > > sure
> > > > > > > exactly
> > > > > > > > > >> what to
> > > > > > > > > >> > > > >> suggest,
> > > > > > > > > >> > > > >> > > but
> > > > > > > > > >> > > > >> > > >>>> >> >> something
> > > > > > > > > >> > > > >> > > >>>> >> >> >> that could identify when
> things
> > > are
> > > > > > going
> > > > > > > > > >> wrong in
> > > > > > > > > >> > > the
> > > > > > > > > >> > > > >> > > clsuter.
> > > > > > > > > >> > > > >> > > >>>> >> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> >> For example, maybe a metric
> for
> > > > > > > partitions
> > > > > > > > > >> > > containing
> > > > > > > > > >> > > > >> > > replicas
> > > > > > > > > >> > > > >> > > >>>> that
> > > > > > > > > >> > > > >> > > >>>> >> are
> > > > > > > > > >> > > > >> > > >>>> >> >> >> ineligible to be leader? That
> > > would
> > > > > > show
> > > > > > > a
> > > > > > > > > >> spike
> > > > > > > > > >> > > when
> > > > > > > > > >> > > > a
> > > > > > > > > >> > > > >> > > broker
> > > > > > > > > >> > > > >> > > >>>> had an
> > > > > > > > > >> > > > >> > > >>>> >> >> >> unclean restart.
> > > > > > > > > >> > > > >> > > >>>> >> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> >> Ideally, we'd also have a
> > metric
> > > > that
> > > > > > > > > indicates
> > > > > > > > > >> > when
> > > > > > > > > >> > > > an
> > > > > > > > > >> > > > >> > > unclear
> > > > > > > > > >> > > > >> > > >>>> >> leader
> > > > > > > > > >> > > > >> > > >>>> >> >> >> election or a recovery
> > happened.
> > > > > It's a
> > > > > > > bit
> > > > > > > > > >> tricky
> > > > > > > > > >> > > > >> because
> > > > > > > > > >> > > > >> > > the
> > > > > > > > > >> > > > >> > > >>>> simple
> > > > > > > > > >> > > > >> > > >>>> >> >> >> thing, of tracking it per
> > > > controller,
> > > > > > may
> > > > > > > > be
> > > > > > > > > a
> > > > > > > > > >> bit
> > > > > > > > > >> > > > >> > confusing
> > > > > > > > > >> > > > >> > > >>>> during
> > > > > > > > > >> > > > >> > > >>>> >> >> >> failovers.
> > > > > > > > > >> > > > >> > > >>>> >> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> >> best,
> > > > > > > > > >> > > > >> > > >>>> >> >> >> Colin
> > > > > > > > > >> > > > >> > > >>>> >> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> >>
> > > > > > > > > >> > > > >> > > >>>> >> >> >> On Tue, Sep 12, 2023, at
> 14:25,
> > > > Colin
> > > > > > > > McCabe
> > > > > > > > > >> > wrote:
> > > > > > > > > >> > > > >> > > >>>> >> >> >> > Hi Calvin,
> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >> >> > Thanks for the KIP. I think
> > > this
> > > > > is a
> > > > > > > > great
> > > > > > > > > >> > > > >> improvement.
> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >> >> >> Additional High Watermark
> > > > advance
> > > > > > > > > >> requirement
> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >> >> > Typo: change "advance" to
> > > > > > "advancement"
> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >> >> >> A bit recap of some key
> > > > concepts.
> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >> >> > Typo: change "bit" to
> "quick"
> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >> >> >> Ack=1/all produce request.
> > It
> > > > > > defines
> > > > > > > > when
> > > > > > > > > >> the
> > > > > > > > > >> > > > Kafka
> > > > > > > > > >> > > > >> > > server
> > > > > > > > > >> > > > >> > > >>>> should
> > > > > > > > > >> > > > >> > > >>>> >> >> >> respond to the produce
> request
> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >> >> > I think this section would
> be
> > > > > clearer
> > > > > > > if
> > > > > > > > we
> > > > > > > > > >> > talked
> > > > > > > > > >> > > > >> about
> > > > > > > > > >> > > > >> > > the
> > > > > > > > > >> > > > >> > > >>>> new
> > > > > > > > > >> > > > >> > > >>>> >> high
> > > > > > > > > >> > > > >> > > >>>> >> >> >> > watermark advancement
> > > requirement
> > > > > > > first,
> > > > > > > > > and
> > > > > > > > > >> > THEN
> > > > > > > > > >> > > > >> talked
> > > > > > > > > >> > > > >> > > >>>> about its
> > > > > > > > > >> > > > >> > > >>>> >> >> >> > impact on acks=0, acks=1,
> and
> > > > > > > >  acks=all.
> > > > > > > > > >> > > > acks=all
> > > > > > > > > >> > > > >> is
> > > > > > > > > >> > > > >> > of
> > > > > > > > > >> > > > >> > > >>>> course
> > > > > > > > > >> > > > >> > > >>>> >> the
> > > > > > > > > >> > > > >> > > >>>> >> >> >> > main case we care about
> here,
> > > so
> > > > it
> > > > > > > would
> > > > > > > > > be
> > > > > > > > > >> > good
> > > > > > > > > >> > > to
> > > > > > > > > >> > > > >> lead
> > > > > > > > > >> > > > >> > > with
> > > > > > > > > >> > > > >> > > >>>> >> that,
> > > > > > > > > >> > > > >> > > >>>> >> >> >> > rather than delving into
> the
> > > > > > > > technicalities
> > > > > > > > > >> of
> > > > > > > > > >> > > > acks=0/1
> > > > > > > > > >> > > > >> > > first.
> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >> >> >> Unclean recovery
> > > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > > > > >> > > > >> > > >>>> >> >> >> > So, here you are
> introducing
> > a
> > > > new
> > > > > > > > > >> > configuration,
> > > > > > > > > >> > > > >> > > >>>> >> >> >> > unclean.recovery.strategy.
> > The
> > > > > > > difficult
> > > > > > > > > >> thing
> > > > > > > > > >> > > here
> > > > > > > > > >> > > > is
> > > > > > > > > >> > > > >> > that
> > > > > > > > > >> > > > >> > > >>>> there
> > > > > > > > > >> > > > >> > > >>>> >> is a
> > > > > > > > > >> > > > >> > > >>>> >> >> >> > lot of overlap with
> > > > > > > > > >> > > unclean.leader.election.enable.
> > > > > > > > > >> > > > So
> > > > > > > > > >> > > > >> we
> > > > > > > > > >> > > > >> > > >>>> have 3
> > > > > > > > > >> > > > >> > > >>>> >> >> >> > different settings for
> > > > > > > > > >> > unclean.recovery.strategy,
> > > > > > > > > >> > > > plus
> > > > > > > > > >> > > > >> 2
> > > > > > > > > >> > > > >> > > >>>> different
> > > > > > > > > >> > > > >> > > >>>> >> >> >> > settings for
> > > > > > > > > unclean.leader.election.enable,
> > > > > > > > > >> > > giving
> > > > > > > > > >> > > > a
> > > > > > > > > >> > > > >> > cross
> > > > > > > > > >> > > > >> > > >>>> >> product of
> > > > > > > > > >> > > > >> > > >>>> >> >> >> > 6 different options. The
> > > > following
> > > > > > > > "unclean
> > > > > > > > > >> > > recovery
> > > > > > > > > >> > > > >> > > manager"
> > > > > > > > > >> > > > >> > > >>>> >> section
> > > > > > > > > >> > > > >> > > >>>> >> >> >> > on
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Calvin Liu <ca...@confluent.io.INVALID>.
Hi Jun,
Yeah, with the current Metadata request handling, we only return errors on
the Topic level, like topic not found. It seems that querying a specific
partition is not a valid use case. Will update.
Thanks

On Tue, Oct 10, 2023 at 11:55 AM Jun Rao <ju...@confluent.io.invalid> wrote:

> Hi, Calvin,
>
> 60.  If the range query has errors for some of the partitions, do we expect
> different responses when querying particular partitions?
>
> Thanks,
>
> Jun
>
> On Tue, Oct 10, 2023 at 10:50 AM Calvin Liu <ca...@confluent.io.invalid>
> wrote:
>
> > Hi Jun
> > 60. Yes, it is a good question. I was thinking the API could be flexible
> to
> > query the particular partitions if the range query has errors for some of
> > the partitions. Not sure whether it is a valid assumption, what do you
> > think?
> >
> > 61. Good point, I will update them to partition level with the same
> limit.
> >
> > 62. Sure, will do.
> >
> > Thanks
> >
> > On Tue, Oct 10, 2023 at 10:12 AM Jun Rao <ju...@confluent.io.invalid>
> wrote:
> >
> > > Hi, Calvin,
> > >
> > > A few more minor comments on your latest update.
> > >
> > > 60. DescribeTopicRequest: When will the Partitions field be used? It
> > seems
> > > that the FirstPartitionId field is enough for AdminClient usage.
> > >
> > > 61. Could we make the limit for DescribeTopicRequest,
> > ElectLeadersRequest,
> > > GetReplicaLogInfo consistent? Currently, ElectLeadersRequest's limit is
> > at
> > > topic level and GetReplicaLogInfo has a different partition level limit
> > > from DescribeTopicRequest.
> > >
> > > 62. Should ElectLeadersRequest.DesiredLeaders be at the same level as
> > > ElectLeadersRequest.TopicPartitions.Partitions? In the KIP, it looks
> like
> > > it's at the same level as ElectLeadersRequest.TopicPartitions.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Wed, Oct 4, 2023 at 3:55 PM Calvin Liu <ca...@confluent.io.invalid>
> > > wrote:
> > >
> > > > Hi David,
> > > > Thanks for the comments.
> > > > ----
> > > > I thought that a new snapshot with the downgraded MV is created in
> this
> > > > case. Isn’t it the case?
> > > > Yes, you are right, a metadata delta will be generated after the MV
> > > > downgrade. Then the user can start the software downgrade.
> > > > -----
> > > > Could you also elaborate a bit more on the reasoning behind adding
> the
> > > > limits to the admin RPCs? This is a new pattern in Kafka so it would
> be
> > > > good to clear on the motivation.
> > > > Thanks to Colin for bringing it up. The current MetadataRequest does
> > not
> > > > have a limit on the number of topics to query in a single request.
> > > Massive
> > > > requests can mess up the JVM. We want to have some sort of throttle
> on
> > > the
> > > > new APIs.
> > > > -----
> > > > Could you also explain how the client is supposed to handle the
> > > > topics/partitions above the limit? I suppose that it will have to
> retry
> > > > those, correct?
> > > > Corrent. For the official admin clients, it will split the large
> > request
> > > > into proper pieces and query one after another.
> > > > -----
> > > > My understanding is that the topics/partitions above the limit will
> be
> > > > failed with an invalid exception error. I wonder if this choice is
> > > > judicious because the invalide request exception is usually fatal. It
> > may
> > > > be better to use an new and explicit error for this case.
> > > >
> > > > Thanks for bringing this up. How about "REQUEST_LIMIT_REACHED"?
> > > > --------
> > > > It seems that we still need to specify the changes to the admin api
> to
> > > > accommodate the new or updated apis. Do you plan to add them?
> > > > Try to cover the following
> > > > 1. The admin client will use the new DescribeTopicRequest to query
> the
> > > > topics
> > > > 2. Mention the API limit and the new retriable error.
> > > > 3. Output changes for the admin client when describing a topic (new
> > > fields
> > > > of ELR...)
> > > > 4. Changes to data structures like TopicPartitionInfo to include the
> > ELR.
> > > > Anything else I missed?
> > > >
> > > > Thanks!
> > > >
> > > >
> > > >
> > > >
> > > >
> > > > On Wed, Oct 4, 2023 at 12:27 PM David Jacot <da...@gmail.com>
> > > wrote:
> > > >
> > > > > Hi Calvin,
> > > > >
> > > > > I thought that a new snapshot with the downgraded MV is created in
> > this
> > > > > case. Isn’t it the case?
> > > > >
> > > > > Could you also elaborate a bit more on the reasoning behind adding
> > the
> > > > > limits to the admin RPCs? This is a new pattern in Kafka so it
> would
> > be
> > > > > good to clear on the motivation.
> > > > >
> > > > > Could you also explain how the client is supposed to handle the
> > > > > topics/partitions above the limit? I suppose that it will have to
> > retry
> > > > > those, correct?
> > > > >
> > > > > My understanding is that the topics/partitions above the limit will
> > be
> > > > > failed with an invalid exception error. I wonder if this choice is
> > > > > judicious because the invalide request exception is usually fatal.
> It
> > > may
> > > > > be better to use an new and explicit error for this case.
> > > > >
> > > > > It seems that we still need to specify the changes to the admin api
> > to
> > > > > accommodate the new or updated apis. Do you plan to add them?
> > > > >
> > > > > Best,
> > > > > David
> > > > >
> > > > > Le mer. 4 oct. 2023 à 20:39, Calvin Liu <caliu@confluent.io.invalid
> >
> > a
> > > > > écrit :
> > > > >
> > > > > > Hi Jun,
> > > > > > After the MV downgrade, the controller will write in the old
> > version
> > > of
> > > > > the
> > > > > > PartitionRecord/PartitionChangeRecord. If I understand correctly,
> > it
> > > is
> > > > > > possible to downgrade the software version if the controller only
> > has
> > > > to
> > > > > > handle old version records.
> > > > > > However, the controller will not automatically rewrite the
> > > > > PartitionRecord
> > > > > > with the old version unless there is a partition update. Then,
> the
> > > user
> > > > > may
> > > > > > have to wait an unknown amount of time before the software
> > downgrades
> > > > > > unless they do a roll to force update every partition. If it
> makes
> > > > > sense, I
> > > > > > can mention these steps to do a software downgrade.
> > > > > > Thanks
> > > > > >
> > > > > > On Wed, Oct 4, 2023 at 11:20 AM Jun Rao <jun@confluent.io.invalid
> >
> > > > > wrote:
> > > > > >
> > > > > > > Hi, Calvin and Justine,
> > > > > > >
> > > > > > > Historically, when we change the record format in the log, we
> > don't
> > > > > > support
> > > > > > > software version downgrading.
> > > > > > >
> > > > > > > For the record format change in the metadata log, have we
> thought
> > > > about
> > > > > > > forcing the write of the latest metadata records with the old
> > > version
> > > > > > > during MV downgrading? This will in theory allow the old
> version
> > of
> > > > the
> > > > > > > software to obtain the latest metadata.
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > Jun
> > > > > > >
> > > > > > > On Wed, Oct 4, 2023 at 9:53 AM Justine Olshan
> > > > > > <jolshan@confluent.io.invalid
> > > > > > > >
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Sorry -- not MV but software version.
> > > > > > > >
> > > > > > > > On Wed, Oct 4, 2023 at 9:51 AM Justine Olshan <
> > > > jolshan@confluent.io>
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Catching up with this discussion.
> > > > > > > > >
> > > > > > > > > I was just curious -- have we had other instances where
> > > > downgrading
> > > > > > MV
> > > > > > > is
> > > > > > > > > not supported? I think Kafka typically tries to support
> > > > downgrades,
> > > > > > > and I
> > > > > > > > > couldn't think of other examples.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Justine
> > > > > > > > >
> > > > > > > > > On Wed, Oct 4, 2023 at 9:40 AM Calvin Liu
> > > > > <caliu@confluent.io.invalid
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > >> Hi Jun,
> > > > > > > > >> 54. Marked the software downgrading is not supported. As
> the
> > > old
> > > > > > > > >> controller
> > > > > > > > >> will not understand the new PartitionRecord and
> > > > > > PartitionChangeRecord.
> > > > > > > > >> Thanks!
> > > > > > > > >>
> > > > > > > > >> On Wed, Oct 4, 2023 at 9:12 AM Jun Rao
> > > <jun@confluent.io.invalid
> > > > >
> > > > > > > > wrote:
> > > > > > > > >>
> > > > > > > > >> > Hi, Calvin,
> > > > > > > > >> >
> > > > > > > > >> > Thanks for the reply. Just one more comment.
> > > > > > > > >> >
> > > > > > > > >> > 54. It seems that downgrading MV is supported. Is
> > > downgrading
> > > > > the
> > > > > > > > >> software
> > > > > > > > >> > version supported? It would be useful to document that.
> > > > > > > > >> >
> > > > > > > > >> > Thanks,
> > > > > > > > >> >
> > > > > > > > >> > Jun
> > > > > > > > >> >
> > > > > > > > >> > On Tue, Oct 3, 2023 at 4:55 PM Artem Livshits
> > > > > > > > >> > <al...@confluent.io.invalid> wrote:
> > > > > > > > >> >
> > > > > > > > >> > > Hi Colin,
> > > > > > > > >> > >
> > > > > > > > >> > > I think in your example "do_unclean_recovery" would
> need
> > > to
> > > > do
> > > > > > > > >> different
> > > > > > > > >> > > things depending on the strategy.
> > > > > > > > >> > >
> > > > > > > > >> > > do_unclean_recovery() {
> > > > > > > > >> > >    if (unclean.recovery.manager.enabled) {
> > > > > > > > >> > >     if (strategy == Aggressive)
> > > > > > > > >> > >       use
> UncleanRecoveryManager(waitLastKnownERL=false)
> > > //
> > > > > > just
> > > > > > > > >> inspect
> > > > > > > > >> > > logs from whoever is available
> > > > > > > > >> > >     else
> > > > > > > > >> > >       use
> UncleanRecoveryManager(waitLastKnownERL=true)
> > > //
> > > > > > must
> > > > > > > > wait
> > > > > > > > >> > for
> > > > > > > > >> > > at least last known ELR
> > > > > > > > >> > >   } else {
> > > > > > > > >> > >     if (strategy == Aggressive)
> > > > > > > > >> > >       choose the last known leader if that is
> available,
> > > or
> > > > a
> > > > > > > random
> > > > > > > > >> > leader
> > > > > > > > >> > > if not)
> > > > > > > > >> > >     else
> > > > > > > > >> > >       wait for last known leader to get back
> > > > > > > > >> > >   }
> > > > > > > > >> > > }
> > > > > > > > >> > >
> > > > > > > > >> > > The idea is that the Aggressive strategy would kick in
> > as
> > > > soon
> > > > > > as
> > > > > > > we
> > > > > > > > >> lost
> > > > > > > > >> > > the leader and would pick a leader from whoever is
> > > > available;
> > > > > > but
> > > > > > > > the
> > > > > > > > >> > > Balanced will only kick in when ELR is empty and will
> > wait
> > > > for
> > > > > > the
> > > > > > > > >> > brokers
> > > > > > > > >> > > that likely have most data to be available.
> > > > > > > > >> > >
> > > > > > > > >> > > On Tue, Oct 3, 2023 at 3:04 PM Colin McCabe <
> > > > > cmccabe@apache.org
> > > > > > >
> > > > > > > > >> wrote:
> > > > > > > > >> > >
> > > > > > > > >> > > > On Tue, Oct 3, 2023, at 10:49, Jun Rao wrote:
> > > > > > > > >> > > > > Hi, Calvin,
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > Thanks for the update KIP. A few more comments.
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > 41. Why would a user choose the option to select a
> > > > random
> > > > > > > > replica
> > > > > > > > >> as
> > > > > > > > >> > > the
> > > > > > > > >> > > > > leader instead of using
> > > > > unclean.recovery.strateg=Aggressive?
> > > > > > > It
> > > > > > > > >> seems
> > > > > > > > >> > > > that
> > > > > > > > >> > > > > the latter is strictly better? If that's not the
> > case,
> > > > > could
> > > > > > > we
> > > > > > > > >> fold
> > > > > > > > >> > > this
> > > > > > > > >> > > > > option under unclean.recovery.strategy instead of
> > > > > > introducing
> > > > > > > a
> > > > > > > > >> > > separate
> > > > > > > > >> > > > > config?
> > > > > > > > >> > > >
> > > > > > > > >> > > > Hi Jun,
> > > > > > > > >> > > >
> > > > > > > > >> > > > I thought the flow of control was:
> > > > > > > > >> > > >
> > > > > > > > >> > > > If there is no leader for the partition {
> > > > > > > > >> > > >   If (there are unfenced ELR members) {
> > > > > > > > >> > > >     choose_an_unfenced_ELR_member
> > > > > > > > >> > > >   } else if (there are fenced ELR members AND
> > > > > > > > strategy=Aggressive) {
> > > > > > > > >> > > >     do_unclean_recovery
> > > > > > > > >> > > >   } else if (there are no ELR members AND strategy
> !=
> > > > None)
> > > > > {
> > > > > > > > >> > > >     do_unclean_recovery
> > > > > > > > >> > > >   } else {
> > > > > > > > >> > > >     do nothing about the missing leader
> > > > > > > > >> > > >   }
> > > > > > > > >> > > > }
> > > > > > > > >> > > >
> > > > > > > > >> > > > do_unclean_recovery() {
> > > > > > > > >> > > >    if (unclean.recovery.manager.enabled) {
> > > > > > > > >> > > >     use UncleanRecoveryManager
> > > > > > > > >> > > >   } else {
> > > > > > > > >> > > >     choose the last known leader if that is
> available,
> > > or
> > > > a
> > > > > > > random
> > > > > > > > >> > leader
> > > > > > > > >> > > > if not)
> > > > > > > > >> > > >   }
> > > > > > > > >> > > > }
> > > > > > > > >> > > >
> > > > > > > > >> > > > However, I think this could be clarified, especially
> > the
> > > > > > > behavior
> > > > > > > > >> when
> > > > > > > > >> > > > unclean.recovery.manager.enabled=false. Inuitively
> the
> > > > goal
> > > > > > for
> > > > > > > > >> > > > unclean.recovery.manager.enabled=false is to be "the
> > > same
> > > > as
> > > > > > > now,
> > > > > > > > >> > mostly"
> > > > > > > > >> > > > but it's very underspecified in the KIP, I agree.
> > > > > > > > >> > > >
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > 50. ElectLeadersRequest: "If more than 20 topics
> are
> > > > > > included,
> > > > > > > > >> only
> > > > > > > > >> > the
> > > > > > > > >> > > > > first 20 will be served. Others will be returned
> > with
> > > > > > > > >> > DesiredLeaders."
> > > > > > > > >> > > > Hmm,
> > > > > > > > >> > > > > not sure that I understand this.
> > ElectLeadersResponse
> > > > > > doesn't
> > > > > > > > >> have a
> > > > > > > > >> > > > > DesiredLeaders field.
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > 51. GetReplicaLogInfo: "If more than 2000
> partitions
> > > are
> > > > > > > > included,
> > > > > > > > >> > only
> > > > > > > > >> > > > the
> > > > > > > > >> > > > > first 2000 will be served" Do we return an error
> for
> > > the
> > > > > > > > remaining
> > > > > > > > >> > > > > partitions? Actually, should we include an
> errorCode
> > > > field
> > > > > > at
> > > > > > > > the
> > > > > > > > >> > > > partition
> > > > > > > > >> > > > > level in GetReplicaLogInfoResponse to cover
> > > non-existing
> > > > > > > > >> partitions
> > > > > > > > >> > and
> > > > > > > > >> > > > no
> > > > > > > > >> > > > > authorization, etc?
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > 52. The entry should matches => The entry should
> > match
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > 53. ElectLeadersRequest.DesiredLeaders: Should it
> be
> > > > > > nullable
> > > > > > > > >> since a
> > > > > > > > >> > > > user
> > > > > > > > >> > > > > may not specify DesiredLeaders?
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > 54. Downgrade: Is that indeed possible? I thought
> > > > earlier
> > > > > > you
> > > > > > > > said
> > > > > > > > >> > that
> > > > > > > > >> > > > > once the new version of the records are in the
> > > metadata
> > > > > log,
> > > > > > > one
> > > > > > > > >> > can't
> > > > > > > > >> > > > > downgrade since the old broker doesn't know how to
> > > parse
> > > > > the
> > > > > > > new
> > > > > > > > >> > > version
> > > > > > > > >> > > > of
> > > > > > > > >> > > > > the metadata records?
> > > > > > > > >> > > > >
> > > > > > > > >> > > >
> > > > > > > > >> > > > MetadataVersion downgrade is currently broken but we
> > > have
> > > > > > fixing
> > > > > > > > it
> > > > > > > > >> on
> > > > > > > > >> > > our
> > > > > > > > >> > > > plate for Kafka 3.7.
> > > > > > > > >> > > >
> > > > > > > > >> > > > The way downgrade works is that "new features" are
> > > > dropped,
> > > > > > > > leaving
> > > > > > > > >> > only
> > > > > > > > >> > > > the old ones.
> > > > > > > > >> > > >
> > > > > > > > >> > > > > 55. CleanShutdownFile: Should we add a version
> field
> > > for
> > > > > > > future
> > > > > > > > >> > > > extension?
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > 56. Config changes are public facing. Could we
> have
> > a
> > > > > > separate
> > > > > > > > >> > section
> > > > > > > > >> > > to
> > > > > > > > >> > > > > document all the config changes?
> > > > > > > > >> > > >
> > > > > > > > >> > > > +1. A separate section for this would be good.
> > > > > > > > >> > > >
> > > > > > > > >> > > > best,
> > > > > > > > >> > > > Colin
> > > > > > > > >> > > >
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > Thanks,
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > Jun
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > On Mon, Sep 25, 2023 at 4:29 PM Calvin Liu
> > > > > > > > >> > <caliu@confluent.io.invalid
> > > > > > > > >> > > >
> > > > > > > > >> > > > > wrote:
> > > > > > > > >> > > > >
> > > > > > > > >> > > > >> Hi Jun
> > > > > > > > >> > > > >> Thanks for the comments.
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >> 40. If we change to None, it is not guaranteed
> for
> > no
> > > > > data
> > > > > > > > loss.
> > > > > > > > >> For
> > > > > > > > >> > > > users
> > > > > > > > >> > > > >> who are not able to validate the data with
> external
> > > > > > > resources,
> > > > > > > > >> > manual
> > > > > > > > >> > > > >> intervention does not give a better result but a
> > loss
> > > > of
> > > > > > > > >> > availability.
> > > > > > > > >> > > > So
> > > > > > > > >> > > > >> practically speaking, the Balance mode would be a
> > > > better
> > > > > > > > default
> > > > > > > > >> > > value.
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >> 41. No, it represents how we want to do the
> unclean
> > > > > leader
> > > > > > > > >> election.
> > > > > > > > >> > > If
> > > > > > > > >> > > > it
> > > > > > > > >> > > > >> is false, the unclean leader election will be the
> > old
> > > > > > random
> > > > > > > > way.
> > > > > > > > >> > > > >> Otherwise, the unclean recovery will be used.
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >> 42. Good catch. Updated.
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >> 43. Only the first 20 topics will be served.
> Others
> > > > will
> > > > > be
> > > > > > > > >> returned
> > > > > > > > >> > > > with
> > > > > > > > >> > > > >> InvalidRequestError
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >> 44. The order matters. The desired leader entries
> > > match
> > > > > > with
> > > > > > > > the
> > > > > > > > >> > topic
> > > > > > > > >> > > > >> partition list by the index.
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >> 45. Thanks! Updated.
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >> 46. Good advice! Updated.
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >> 47.1, updated the comment. Basically it will
> elect
> > > the
> > > > > > > replica
> > > > > > > > in
> > > > > > > > >> > the
> > > > > > > > >> > > > >> desiredLeader field to be the leader
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >> 47.2 We can let the admin client do the
> conversion.
> > > > Using
> > > > > > the
> > > > > > > > >> > > > desiredLeader
> > > > > > > > >> > > > >> field in the json format seems easier for users.
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >> 48. Once the MV version is downgraded, all the
> ELR
> > > > > related
> > > > > > > > fields
> > > > > > > > >> > will
> > > > > > > > >> > > > be
> > > > > > > > >> > > > >> removed on the next partition change. The
> > controller
> > > > will
> > > > > > > also
> > > > > > > > >> > ignore
> > > > > > > > >> > > > the
> > > > > > > > >> > > > >> ELR fields. Updated the KIP.
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >> 49. Yes, it would be deprecated/removed.
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >> On Mon, Sep 25, 2023 at 3:49 PM Jun Rao
> > > > > > > > <jun@confluent.io.invalid
> > > > > > > > >> >
> > > > > > > > >> > > > wrote:
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >> > Hi, Calvin,
> > > > > > > > >> > > > >> >
> > > > > > > > >> > > > >> > Thanks for the updated KIP. Made another pass.
> A
> > > few
> > > > > more
> > > > > > > > >> comments
> > > > > > > > >> > > > below.
> > > > > > > > >> > > > >> >
> > > > > > > > >> > > > >> > 40. unclean.leader.election.enable.false ->
> > > > > > > > >> > > > >> > unclean.recovery.strategy.Balanced: The
> Balanced
> > > mode
> > > > > > could
> > > > > > > > >> still
> > > > > > > > >> > > > lead to
> > > > > > > > >> > > > >> > data loss. So, I am wondering if
> > > > > > > > >> > > unclean.leader.election.enable.false
> > > > > > > > >> > > > >> > should map to None?
> > > > > > > > >> > > > >> >
> > > > > > > > >> > > > >> > 41. unclean.recovery.manager.enabled: I am not
> > sure
> > > > why
> > > > > > we
> > > > > > > > >> > introduce
> > > > > > > > >> > > > this
> > > > > > > > >> > > > >> > additional config. Is it the same as
> > > > > > > > >> > unclean.recovery.strategy=None?
> > > > > > > > >> > > > >> >
> > > > > > > > >> > > > >> > 42.
> > > DescribeTopicResponse.TopicAuthorizedOperations:
> > > > > > Should
> > > > > > > > >> this
> > > > > > > > >> > be
> > > > > > > > >> > > at
> > > > > > > > >> > > > >> the
> > > > > > > > >> > > > >> > topic level?
> > > > > > > > >> > > > >> >
> > > > > > > > >> > > > >> > 43. "Limit: 20 topics max per request": Could
> we
> > > > > describe
> > > > > > > > what
> > > > > > > > >> > > > happens if
> > > > > > > > >> > > > >> > the request includes more than 20 topics?
> > > > > > > > >> > > > >> >
> > > > > > > > >> > > > >> > 44. ElectLeadersRequest.DesiredLeaders: Could
> we
> > > > > describe
> > > > > > > > >> whether
> > > > > > > > >> > > the
> > > > > > > > >> > > > >> > ordering matters?
> > > > > > > > >> > > > >> >
> > > > > > > > >> > > > >> > 45. GetReplicaLogInfo.TopicPartitions: "about":
> > > "The
> > > > > > topic
> > > > > > > > >> > > partitions
> > > > > > > > >> > > > to
> > > > > > > > >> > > > >> > elect leaders.": The description in "about" is
> > > > > incorrect.
> > > > > > > > >> > > > >> >
> > > > > > > > >> > > > >> > 46. GetReplicaLogInfoResponse: Should we nest
> > > > > partitions
> > > > > > > > under
> > > > > > > > >> > > > topicId to
> > > > > > > > >> > > > >> > be consistent with other types of responses?
> > > > > > > > >> > > > >> >
> > > > > > > > >> > > > >> > 47. kafka-leader-election.sh:
> > > > > > > > >> > > > >> > 47.1 Could we explain DESIGNATION?
> > > > > > > > >> > > > >> > 47.2 desiredLeader: Should it be a list to
> match
> > > the
> > > > > > field
> > > > > > > in
> > > > > > > > >> > > > >> > ElectLeadersRequest?
> > > > > > > > >> > > > >> >
> > > > > > > > >> > > > >> > 48. We could add a section on downgrade?
> > > > > > > > >> > > > >> >
> > > > > > > > >> > > > >> > 49. LastKnownLeader: This seems only needed in
> > the
> > > > > first
> > > > > > > > phase
> > > > > > > > >> of
> > > > > > > > >> > > > >> > delivering ELR. Will it be removed when the
> > > complete
> > > > > KIP
> > > > > > is
> > > > > > > > >> > > delivered?
> > > > > > > > >> > > > >> >
> > > > > > > > >> > > > >> > Thanks,
> > > > > > > > >> > > > >> >
> > > > > > > > >> > > > >> > Jun
> > > > > > > > >> > > > >> >
> > > > > > > > >> > > > >> > On Tue, Sep 19, 2023 at 1:30 PM Colin McCabe <
> > > > > > > > >> cmccabe@apache.org>
> > > > > > > > >> > > > wrote:
> > > > > > > > >> > > > >> >
> > > > > > > > >> > > > >> > > Hi Calvin,
> > > > > > > > >> > > > >> > >
> > > > > > > > >> > > > >> > > Thanks for the explanations. I like the idea
> of
> > > > using
> > > > > > > none,
> > > > > > > > >> > > > balanced,
> > > > > > > > >> > > > >> > > aggressive. We also had an offline discussion
> > > about
> > > > > why
> > > > > > > it
> > > > > > > > is
> > > > > > > > >> > good
> > > > > > > > >> > > > to
> > > > > > > > >> > > > >> > use a
> > > > > > > > >> > > > >> > > new config key (basically, so that we can
> > > deprecate
> > > > > the
> > > > > > > old
> > > > > > > > >> one
> > > > > > > > >> > > > which
> > > > > > > > >> > > > >> had
> > > > > > > > >> > > > >> > > only false/true values in 4.0) With these
> > > changes,
> > > > I
> > > > > am
> > > > > > > +1.
> > > > > > > > >> > > > >> > >
> > > > > > > > >> > > > >> > > best,
> > > > > > > > >> > > > >> > > Colin
> > > > > > > > >> > > > >> > >
> > > > > > > > >> > > > >> > > On Mon, Sep 18, 2023, at 15:54, Calvin Liu
> > wrote:
> > > > > > > > >> > > > >> > > > Hi Colin,
> > > > > > > > >> > > > >> > > > Also, can we deprecate
> > > > > unclean.leader.election.enable
> > > > > > > in
> > > > > > > > >> 4.0?
> > > > > > > > >> > > > Before
> > > > > > > > >> > > > >> > > that,
> > > > > > > > >> > > > >> > > > we can have both the config
> > > > > unclean.recovery.strategy
> > > > > > > and
> > > > > > > > >> > > > >> > > > unclean.leader.election.enable
> > > > > > > > >> > > > >> > > > and using the unclean.recovery.Enabled to
> > > > determine
> > > > > > > which
> > > > > > > > >> > config
> > > > > > > > >> > > > to
> > > > > > > > >> > > > >> use
> > > > > > > > >> > > > >> > > > during the unclean leader election.
> > > > > > > > >> > > > >> > > >
> > > > > > > > >> > > > >> > > > On Mon, Sep 18, 2023 at 3:51 PM Calvin Liu
> <
> > > > > > > > >> > caliu@confluent.io>
> > > > > > > > >> > > > >> wrote:
> > > > > > > > >> > > > >> > > >
> > > > > > > > >> > > > >> > > >> Hi Colin,
> > > > > > > > >> > > > >> > > >> For the unclean.recovery.strategy config
> > name,
> > > > how
> > > > > > > about
> > > > > > > > >> we
> > > > > > > > >> > use
> > > > > > > > >> > > > the
> > > > > > > > >> > > > >> > > >> following
> > > > > > > > >> > > > >> > > >> None. It basically means no unclean
> recovery
> > > > will
> > > > > be
> > > > > > > > >> > performed.
> > > > > > > > >> > > > >> > > >> Aggressive. It means availability goes
> > first.
> > > > > > Whenever
> > > > > > > > the
> > > > > > > > >> > > > partition
> > > > > > > > >> > > > >> > > can't
> > > > > > > > >> > > > >> > > >> elect a durable replica, the controller
> will
> > > try
> > > > > the
> > > > > > > > >> unclean
> > > > > > > > >> > > > >> recovery.
> > > > > > > > >> > > > >> > > >> Balanced. It is the balance point of the
> > > > > > availability
> > > > > > > > >> > > > >> > first(Aggressive)
> > > > > > > > >> > > > >> > > >> and least availability(None). The
> controller
> > > > > > performs
> > > > > > > > >> unclean
> > > > > > > > >> > > > >> recovery
> > > > > > > > >> > > > >> > > when
> > > > > > > > >> > > > >> > > >> both ISR and ELR are empty.
> > > > > > > > >> > > > >> > > >>
> > > > > > > > >> > > > >> > > >>
> > > > > > > > >> > > > >> > > >> On Fri, Sep 15, 2023 at 11:42 AM Calvin
> Liu
> > <
> > > > > > > > >> > > caliu@confluent.io>
> > > > > > > > >> > > > >> > wrote:
> > > > > > > > >> > > > >> > > >>
> > > > > > > > >> > > > >> > > >>> Hi Colin,
> > > > > > > > >> > > > >> > > >>>
> > > > > > > > >> > > > >> > > >>> > So, the proposal is that if someone
> sets
> > > > > > > > >> > > > >> > > "unclean.leader.election.enable
> > > > > > > > >> > > > >> > > >>> = true"...
> > > > > > > > >> > > > >> > > >>>
> > > > > > > > >> > > > >> > > >>>
> > > > > > > > >> > > > >> > > >>> The idea is to use one of the
> > > > > > > > >> unclean.leader.election.enable
> > > > > > > > >> > > and
> > > > > > > > >> > > > >> > > >>> unclean.recovery.strategy based on the
> > > > > > > > >> > > > unclean.recovery.Enabled. A
> > > > > > > > >> > > > >> > > possible
> > > > > > > > >> > > > >> > > >>> version can be
> > > > > > > > >> > > > >> > > >>>
> > > > > > > > >> > > > >> > > >>> If unclean.recovery.Enabled:
> > > > > > > > >> > > > >> > > >>>
> > > > > > > > >> > > > >> > > >>> {
> > > > > > > > >> > > > >> > > >>>
> > > > > > > > >> > > > >> > > >>> Check unclean.recovery.strategy. If set,
> > use
> > > > it.
> > > > > > > > >> Otherwise,
> > > > > > > > >> > > > check
> > > > > > > > >> > > > >> > > >>> unclean.leader.election.enable and
> > translate
> > > it
> > > > > to
> > > > > > > > >> > > > >> > > >>> unclean.recovery.strategy.
> > > > > > > > >> > > > >> > > >>>
> > > > > > > > >> > > > >> > > >>> } else {
> > > > > > > > >> > > > >> > > >>>
> > > > > > > > >> > > > >> > > >>> Use unclean.leader.election.enable
> > > > > > > > >> > > > >> > > >>>
> > > > > > > > >> > > > >> > > >>> }
> > > > > > > > >> > > > >> > > >>>
> > > > > > > > >> > > > >> > > >>>
> > > > > > > > >> > > > >> > > >>> —--------
> > > > > > > > >> > > > >> > > >>>
> > > > > > > > >> > > > >> > > >>> >The configuration key should be
> > > > > > > > >> > > > >> "unclean.recovery.manager.enabled",
> > > > > > > > >> > > > >> > > >>> right?
> > > > > > > > >> > > > >> > > >>>
> > > > > > > > >> > > > >> > > >>>
> > > > > > > > >> > > > >> > > >>> I think we have two ways of choosing a
> > leader
> > > > > > > > uncleanly,
> > > > > > > > >> > > unclean
> > > > > > > > >> > > > >> > leader
> > > > > > > > >> > > > >> > > >>> election and unclean recovery(log
> > inspection)
> > > > and
> > > > > > we
> > > > > > > > try
> > > > > > > > >> to
> > > > > > > > >> > > > switch
> > > > > > > > >> > > > >> > > between
> > > > > > > > >> > > > >> > > >>> them.
> > > > > > > > >> > > > >> > > >>>
> > > > > > > > >> > > > >> > > >>> Do you mean we want to develop two ways
> of
> > > > > > performing
> > > > > > > > the
> > > > > > > > >> > > > unclean
> > > > > > > > >> > > > >> > > >>> recovery and one of them is using
> “unclean
> > > > > recovery
> > > > > > > > >> > manager”?
> > > > > > > > >> > > I
> > > > > > > > >> > > > >> guess
> > > > > > > > >> > > > >> > > we
> > > > > > > > >> > > > >> > > >>> haven’t discussed the second way.
> > > > > > > > >> > > > >> > > >>>
> > > > > > > > >> > > > >> > > >>>
> > > > > > > > >> > > > >> > > >>> —-------
> > > > > > > > >> > > > >> > > >>>
> > > > > > > > >> > > > >> > > >>> >How do these 4 levels of overrides
> > interact
> > > > with
> > > > > > > your
> > > > > > > > >> new
> > > > > > > > >> > > > >> > > >>> configurations?
> > > > > > > > >> > > > >> > > >>>
> > > > > > > > >> > > > >> > > >>>
> > > > > > > > >> > > > >> > > >>> I do notice in the Kraft controller code,
> > the
> > > > > > method
> > > > > > > to
> > > > > > > > >> > check
> > > > > > > > >> > > > >> whether
> > > > > > > > >> > > > >> > > >>> perform unclean leader election is hard
> > coded
> > > > to
> > > > > > > false
> > > > > > > > >> since
> > > > > > > > >> > > > >> > > >>>
> 2021(uncleanLeaderElectionEnabledForTopic).
> > > > Isn’t
> > > > > > it
> > > > > > > a
> > > > > > > > >> good
> > > > > > > > >> > > > chance
> > > > > > > > >> > > > >> to
> > > > > > > > >> > > > >> > > >>> completely deprecate the
> > > > > > > > unclean.leader.election.enable?
> > > > > > > > >> We
> > > > > > > > >> > > > don’t
> > > > > > > > >> > > > >> > even
> > > > > > > > >> > > > >> > > have
> > > > > > > > >> > > > >> > > >>> to worry about the config conversion.
> > > > > > > > >> > > > >> > > >>>
> > > > > > > > >> > > > >> > > >>> On the other hand, whatever the override
> > is,
> > > as
> > > > > > long
> > > > > > > as
> > > > > > > > >> the
> > > > > > > > >> > > > >> > controller
> > > > > > > > >> > > > >> > > >>> can have the final effective
> > > > > > > > >> unclean.leader.election.enable,
> > > > > > > > >> > > the
> > > > > > > > >> > > > >> > topic
> > > > > > > > >> > > > >> > > >>> level config unclean.recovery.strategy,
> the
> > > > > cluster
> > > > > > > > level
> > > > > > > > >> > > config
> > > > > > > > >> > > > >> > > >>> unclean.recovery.Enabled, the controller
> > can
> > > > > > > calculate
> > > > > > > > >> the
> > > > > > > > >> > > > correct
> > > > > > > > >> > > > >> > > methods
> > > > > > > > >> > > > >> > > >>> to use right?
> > > > > > > > >> > > > >> > > >>>
> > > > > > > > >> > > > >> > > >>>
> > > > > > > > >> > > > >> > > >>> On Fri, Sep 15, 2023 at 10:02 AM Colin
> > > McCabe <
> > > > > > > > >> > > > cmccabe@apache.org>
> > > > > > > > >> > > > >> > > wrote:
> > > > > > > > >> > > > >> > > >>>
> > > > > > > > >> > > > >> > > >>>> On Thu, Sep 14, 2023, at 22:23, Calvin
> Liu
> > > > > wrote:
> > > > > > > > >> > > > >> > > >>>> > Hi Colin
> > > > > > > > >> > > > >> > > >>>> > 1. I think using the new config name
> is
> > > more
> > > > > > > clear.
> > > > > > > > >> > > > >> > > >>>> >        a. The unclean leader election
> is
> > > > > > actually
> > > > > > > > >> removed
> > > > > > > > >> > > if
> > > > > > > > >> > > > >> > unclean
> > > > > > > > >> > > > >> > > >>>> > recovery is in use.
> > > > > > > > >> > > > >> > > >>>> >        b. Using multiple values in
> > > > > > > > >> > > > >> unclean.leader.election.enable
> > > > > > > > >> > > > >> > is
> > > > > > > > >> > > > >> > > >>>> > confusing and it will be more
> confusing
> > > > after
> > > > > > > people
> > > > > > > > >> > forget
> > > > > > > > >> > > > >> about
> > > > > > > > >> > > > >> > > this
> > > > > > > > >> > > > >> > > >>>> > discussion.
> > > > > > > > >> > > > >> > > >>>>
> > > > > > > > >> > > > >> > > >>>> Hi Calvin,
> > > > > > > > >> > > > >> > > >>>>
> > > > > > > > >> > > > >> > > >>>> So, the proposal is that if someone sets
> > > > > > > > >> > > > >> > > "unclean.leader.election.enable
> > > > > > > > >> > > > >> > > >>>> = true" but then sets one of your new
> > > > > > > configurations,
> > > > > > > > >> the
> > > > > > > > >> > > > value of
> > > > > > > > >> > > > >> > > >>>> unclean.leader.election.enable is
> ignored?
> > > > That
> > > > > > > seems
> > > > > > > > >> less
> > > > > > > > >> > > > clear
> > > > > > > > >> > > > >> to
> > > > > > > > >> > > > >> > > me, not
> > > > > > > > >> > > > >> > > >>>> more. Just in general, having multiple
> > > > > > configuration
> > > > > > > > >> keys
> > > > > > > > >> > to
> > > > > > > > >> > > > >> control
> > > > > > > > >> > > > >> > > the
> > > > > > > > >> > > > >> > > >>>> same thing confuses users. Basically,
> they
> > > are
> > > > > > > sitting
> > > > > > > > >> at a
> > > > > > > > >> > > > giant
> > > > > > > > >> > > > >> > > control
> > > > > > > > >> > > > >> > > >>>> panel, and some of the levers do
> nothing.
> > > > > > > > >> > > > >> > > >>>>
> > > > > > > > >> > > > >> > > >>>> > 2. Sorry I forgot to mention in the
> > > response
> > > > > > that
> > > > > > > I
> > > > > > > > >> did
> > > > > > > > >> > add
> > > > > > > > >> > > > the
> > > > > > > > >> > > > >> > > >>>> > unclean.recovery.Enabled flag.
> > > > > > > > >> > > > >> > > >>>>
> > > > > > > > >> > > > >> > > >>>> The configuration key should be
> > > > > > > > >> > > > >> "unclean.recovery.manager.enabled",
> > > > > > > > >> > > > >> > > >>>> right? Becuase we can do "unclean
> > recovery"
> > > > > > without
> > > > > > > > the
> > > > > > > > >> > > > manager.
> > > > > > > > >> > > > >> > > Disabling
> > > > > > > > >> > > > >> > > >>>> the manager just means we use a
> different
> > > > > > mechanism
> > > > > > > > for
> > > > > > > > >> > > > recovery.
> > > > > > > > >> > > > >> > > >>>>
> > > > > > > > >> > > > >> > > >>>> >        c. Maybe I underestimated the
> > > > challenge
> > > > > > of
> > > > > > > > >> > replacing
> > > > > > > > >> > > > the
> > > > > > > > >> > > > >> > > >>>> config. Any
> > > > > > > > >> > > > >> > > >>>> > implementation problems ahead?
> > > > > > > > >> > > > >> > > >>>>
> > > > > > > > >> > > > >> > > >>>> There are four levels of overrides for
> > > > > > > > >> > > > >> > unclean.leader.election.enable.
> > > > > > > > >> > > > >> > > >>>>
> > > > > > > > >> > > > >> > > >>>> 1. static configuration for node.
> > > > > > > > >> > > > >> > > >>>>     This goes in the configuration file,
> > > > > typically
> > > > > > > > named
> > > > > > > > >> > > > >> > > >>>> server.properties
> > > > > > > > >> > > > >> > > >>>>
> > > > > > > > >> > > > >> > > >>>> 2. dynamic configuration for node
> default
> > > > > > > > >> > > > >> > > >>>>   ConfigResource(type=BROKER, name="")
> > > > > > > > >> > > > >> > > >>>>
> > > > > > > > >> > > > >> > > >>>> 3. dynamic configuration for node
> > > > > > > > >> > > > >> > > >>>>   ConfigResource(type=BROKER,
> > > name=<controller
> > > > > > id>)
> > > > > > > > >> > > > >> > > >>>>
> > > > > > > > >> > > > >> > > >>>> 4. dynamic configuration for topic
> > > > > > > > >> > > > >> > > >>>>   ConfigResource(type=TOPIC,
> > > > name=<topic-name>)
> > > > > > > > >> > > > >> > > >>>>
> > > > > > > > >> > > > >> > > >>>> How do these 4 levels of overrides
> > interact
> > > > with
> > > > > > > your
> > > > > > > > >> new
> > > > > > > > >> > > > >> > > >>>> configurations? If the new
> configurations
> > > > > dominate
> > > > > > > > over
> > > > > > > > >> the
> > > > > > > > >> > > old
> > > > > > > > >> > > > >> > ones,
> > > > > > > > >> > > > >> > > it
> > > > > > > > >> > > > >> > > >>>> seems like this will get a lot more
> > > confusing
> > > > to
> > > > > > > > >> implement
> > > > > > > > >> > > (and
> > > > > > > > >> > > > >> also
> > > > > > > > >> > > > >> > > to
> > > > > > > > >> > > > >> > > >>>> use.)
> > > > > > > > >> > > > >> > > >>>>
> > > > > > > > >> > > > >> > > >>>> Again, I'd recommend just adding some
> new
> > > > values
> > > > > > to
> > > > > > > > >> > > > >> > > >>>> unclean.leader.election.enable. It's
> > simple
> > > > and
> > > > > > will
> > > > > > > > >> > prevent
> > > > > > > > >> > > > user
> > > > > > > > >> > > > >> > > confusion
> > > > > > > > >> > > > >> > > >>>> (as well as developer confusion.)
> > > > > > > > >> > > > >> > > >>>>
> > > > > > > > >> > > > >> > > >>>> best,
> > > > > > > > >> > > > >> > > >>>> Colin
> > > > > > > > >> > > > >> > > >>>>
> > > > > > > > >> > > > >> > > >>>>
> > > > > > > > >> > > > >> > > >>>> > 3. About the admin client, I
> mentioned 3
> > > > > changes
> > > > > > > in
> > > > > > > > >> the
> > > > > > > > >> > > > client.
> > > > > > > > >> > > > >> > > >>>> Anything
> > > > > > > > >> > > > >> > > >>>> > else I missed in the KIP?
> > > > > > > > >> > > > >> > > >>>> >       a. The client will switch to
> using
> > > the
> > > > > new
> > > > > > > RPC
> > > > > > > > >> > > instead
> > > > > > > > >> > > > of
> > > > > > > > >> > > > >> > > >>>> > MetadataRequest for the topics.
> > > > > > > > >> > > > >> > > >>>> >       b. The TopicPartitionInfo used
> in
> > > > > > > > >> TopicDescription
> > > > > > > > >> > > > needs
> > > > > > > > >> > > > >> to
> > > > > > > > >> > > > >> > > add
> > > > > > > > >> > > > >> > > >>>> new
> > > > > > > > >> > > > >> > > >>>> > fields related to the ELR.
> > > > > > > > >> > > > >> > > >>>> >       c. The outputs will add the ELR
> > > > related
> > > > > > > > fields.
> > > > > > > > >> > > > >> > > >>>> >
> > > > > > > > >> > > > >> > > >>>> > On Thu, Sep 14, 2023 at 9:19 PM Colin
> > > > McCabe <
> > > > > > > > >> > > > >> cmccabe@apache.org>
> > > > > > > > >> > > > >> > > >>>> wrote:
> > > > > > > > >> > > > >> > > >>>> >
> > > > > > > > >> > > > >> > > >>>> >> Hi Calvin,
> > > > > > > > >> > > > >> > > >>>> >>
> > > > > > > > >> > > > >> > > >>>> >> Thanks for the changes.
> > > > > > > > >> > > > >> > > >>>> >>
> > > > > > > > >> > > > >> > > >>>> >> 1. Earlier I commented that creating
> > > > > > > > >> > > > >> "unclean.recovery.strategy "
> > > > > > > > >> > > > >> > > is
> > > > > > > > >> > > > >> > > >>>> not
> > > > > > > > >> > > > >> > > >>>> >> necessary, and we can just reuse the
> > > > existing
> > > > > > > > >> > > > >> > > >>>> >> "unclean.leader.election.enable"
> > > > > configuration
> > > > > > > key.
> > > > > > > > >> > Let's
> > > > > > > > >> > > > >> discuss
> > > > > > > > >> > > > >> > > >>>> that.
> > > > > > > > >> > > > >> > > >>>> >>
> > > > > > > > >> > > > >> > > >>>> >> 2.I also don't understand why you
> > didn't
> > > > add
> > > > > a
> > > > > > > > >> > > > configuration to
> > > > > > > > >> > > > >> > > >>>> enable or
> > > > > > > > >> > > > >> > > >>>> >> disable the Unclean Recovery Manager.
> > > This
> > > > > > seems
> > > > > > > > >> like a
> > > > > > > > >> > > very
> > > > > > > > >> > > > >> > simple
> > > > > > > > >> > > > >> > > >>>> way to
> > > > > > > > >> > > > >> > > >>>> >> handle the staging issue which we
> > > > discussed.
> > > > > > The
> > > > > > > > URM
> > > > > > > > >> can
> > > > > > > > >> > > > just
> > > > > > > > >> > > > >> be
> > > > > > > > >> > > > >> > > >>>> turned off
> > > > > > > > >> > > > >> > > >>>> >> until it is production ready. Let's
> > > discuss
> > > > > > this.
> > > > > > > > >> > > > >> > > >>>> >>
> > > > > > > > >> > > > >> > > >>>> >> 3. You still need to describe the
> > changes
> > > > to
> > > > > > > > >> AdminClient
> > > > > > > > >> > > > that
> > > > > > > > >> > > > >> are
> > > > > > > > >> > > > >> > > >>>> needed
> > > > > > > > >> > > > >> > > >>>> >> to use DescribeTopicRequest.
> > > > > > > > >> > > > >> > > >>>> >>
> > > > > > > > >> > > > >> > > >>>> >> Keep at it. It's looking better. :)
> > > > > > > > >> > > > >> > > >>>> >>
> > > > > > > > >> > > > >> > > >>>> >> best,
> > > > > > > > >> > > > >> > > >>>> >> Colin
> > > > > > > > >> > > > >> > > >>>> >>
> > > > > > > > >> > > > >> > > >>>> >>
> > > > > > > > >> > > > >> > > >>>> >> On Thu, Sep 14, 2023, at 11:03,
> Calvin
> > > Liu
> > > > > > wrote:
> > > > > > > > >> > > > >> > > >>>> >> > Hi Colin
> > > > > > > > >> > > > >> > > >>>> >> > Thanks for the comments!
> > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > >> > > > >> > > >>>> >> > I did the following changes
> > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > >> > > > >> > > >>>> >> >    1.
> > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > >> > > > >> > > >>>> >> >    Simplified the API spec section
> to
> > > > only
> > > > > > > > include
> > > > > > > > >> the
> > > > > > > > >> > > > diff.
> > > > > > > > >> > > > >> > > >>>> >> >    2.
> > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > >> > > > >> > > >>>> >> >    Reordered the HWM requirement
> > > section.
> > > > > > > > >> > > > >> > > >>>> >> >    3.
> > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > >> > > > >> > > >>>> >> >    Removed the URM implementation
> > > details
> > > > > to
> > > > > > > keep
> > > > > > > > >> the
> > > > > > > > >> > > > >> necessary
> > > > > > > > >> > > > >> > > >>>> >> >    characteristics to perform the
> > > unclean
> > > > > > > > recovery.
> > > > > > > > >> > > > >> > > >>>> >> >    1.
> > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > >> > > > >> > > >>>> >> >       When to perform the unclean
> > > > recovery
> > > > > > > > >> > > > >> > > >>>> >> >       2.
> > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > >> > > > >> > > >>>> >> >       Under different config, how
> the
> > > > > unclean
> > > > > > > > >> recovery
> > > > > > > > >> > > > finds
> > > > > > > > >> > > > >> > the
> > > > > > > > >> > > > >> > > >>>> leader.
> > > > > > > > >> > > > >> > > >>>> >> >       3.
> > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > >> > > > >> > > >>>> >> >       How the config
> > > > > > > > unclean.leader.election.enable
> > > > > > > > >> > and
> > > > > > > > >> > > > >> > > >>>> >> >       unclean.recovery.strategy are
> > > > > converted
> > > > > > > > when
> > > > > > > > >> > users
> > > > > > > > >> > > > >> > > >>>> enable/disable
> > > > > > > > >> > > > >> > > >>>> >> the
> > > > > > > > >> > > > >> > > >>>> >> >       unclean recovery.
> > > > > > > > >> > > > >> > > >>>> >> >       4.
> > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > >> > > > >> > > >>>> >> >    More details about how we change
> > > admin
> > > > > > > client.
> > > > > > > > >> > > > >> > > >>>> >> >    5.
> > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > >> > > > >> > > >>>> >> >    API limits on the
> > > > > GetReplicaLogInfoRequest
> > > > > > > and
> > > > > > > > >> > > > >> > > >>>> DescribeTopicRequest.
> > > > > > > > >> > > > >> > > >>>> >> >    6.
> > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > >> > > > >> > > >>>> >> >    Two metrics added
> > > > > > > > >> > > > >> > > >>>> >> >    1.
> > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > >> > > >
> Kafka.controller.global_under_min_isr_partition_count
> > > > > > > > >> > > > >> > > >>>> >> >       2.
> > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > >>  kafka.controller.unclean_recovery_finished_count
> > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > >> > > > >> > > >>>> >> > On Wed, Sep 13, 2023 at 10:46 AM
> > Colin
> > > > > > McCabe <
> > > > > > > > >> > > > >> > > cmccabe@apache.org>
> > > > > > > > >> > > > >> > > >>>> >> wrote:
> > > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > > >> > > > >> > > >>>> >> >> On Tue, Sep 12, 2023, at 17:21,
> > Calvin
> > > > Liu
> > > > > > > > wrote:
> > > > > > > > >> > > > >> > > >>>> >> >> > Hi Colin
> > > > > > > > >> > > > >> > > >>>> >> >> > Thanks for the comments!
> > > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> Hi Calvin,
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> Thanks again for the KIP.
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> One meta-comment: it's usually
> > better
> > > to
> > > > > > just
> > > > > > > > do a
> > > > > > > > >> > diff
> > > > > > > > >> > > > on a
> > > > > > > > >> > > > >> > > >>>> message
> > > > > > > > >> > > > >> > > >>>> >> spec
> > > > > > > > >> > > > >> > > >>>> >> >> file or java file if you're
> > including
> > > > > > changes
> > > > > > > to
> > > > > > > > >> it
> > > > > > > > >> > in
> > > > > > > > >> > > > the
> > > > > > > > >> > > > >> > KIP.
> > > > > > > > >> > > > >> > > >>>> This is
> > > > > > > > >> > > > >> > > >>>> >> >> easier to read than looking for
> "new
> > > > > fields
> > > > > > > > begin"
> > > > > > > > >> > etc.
> > > > > > > > >> > > > in
> > > > > > > > >> > > > >> the
> > > > > > > > >> > > > >> > > >>>> text, and
> > > > > > > > >> > > > >> > > >>>> >> >> gracefully handles the case where
> > > > existing
> > > > > > > > fields
> > > > > > > > >> > were
> > > > > > > > >> > > > >> > changed.
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> > Rewrite the Additional High
> > > Watermark
> > > > > > > > >> advancement
> > > > > > > > >> > > > >> > requirement
> > > > > > > > >> > > > >> > > >>>> >> >> > There was feedback on this
> section
> > > > that
> > > > > > some
> > > > > > > > >> > readers
> > > > > > > > >> > > > may
> > > > > > > > >> > > > >> not
> > > > > > > > >> > > > >> > > be
> > > > > > > > >> > > > >> > > >>>> >> familiar
> > > > > > > > >> > > > >> > > >>>> >> >> > with HWM and Ack=0,1,all
> requests.
> > > > This
> > > > > > can
> > > > > > > > help
> > > > > > > > >> > them
> > > > > > > > >> > > > >> > > understand
> > > > > > > > >> > > > >> > > >>>> the
> > > > > > > > >> > > > >> > > >>>> >> >> > proposal. I will rewrite this
> part
> > > for
> > > > > > more
> > > > > > > > >> > > > readability.
> > > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> To be clear, I wasn't suggesting
> > > > dropping
> > > > > > > either
> > > > > > > > >> > > > section. I
> > > > > > > > >> > > > >> > > agree
> > > > > > > > >> > > > >> > > >>>> that
> > > > > > > > >> > > > >> > > >>>> >> >> they add useful background. I was
> > just
> > > > > > > > suggesting
> > > > > > > > >> > that
> > > > > > > > >> > > we
> > > > > > > > >> > > > >> > should
> > > > > > > > >> > > > >> > > >>>> discuss
> > > > > > > > >> > > > >> > > >>>> >> >> the "acks" setting AFTER
> discussing
> > > the
> > > > > new
> > > > > > > high
> > > > > > > > >> > > > watermark
> > > > > > > > >> > > > >> > > >>>> advancement
> > > > > > > > >> > > > >> > > >>>> >> >> conditions. We also should discuss
> > > > acks=0.
> > > > > > > While
> > > > > > > > >> it
> > > > > > > > >> > > isn't
> > > > > > > > >> > > > >> > > >>>> conceptually
> > > > > > > > >> > > > >> > > >>>> >> much
> > > > > > > > >> > > > >> > > >>>> >> >> different than acks=1 here, its
> > > omission
> > > > > > from
> > > > > > > > this
> > > > > > > > >> > > > section
> > > > > > > > >> > > > >> is
> > > > > > > > >> > > > >> > > >>>> confusing.
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> > Unclean recovery
> > > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > > >> > > > >> > > >>>> >> >> > The plan is to replace the
> > > > > > > > >> > > > unclean.leader.election.enable
> > > > > > > > >> > > > >> > with
> > > > > > > > >> > > > >> > > >>>> >> >> > unclean.recovery.strategy. If
> the
> > > > > Unclean
> > > > > > > > >> Recovery
> > > > > > > > >> > is
> > > > > > > > >> > > > >> > enabled
> > > > > > > > >> > > > >> > > >>>> then it
> > > > > > > > >> > > > >> > > >>>> >> >> deals
> > > > > > > > >> > > > >> > > >>>> >> >> > with the three options in the
> > > > > > > > >> > > > unclean.recovery.strategy.
> > > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > > >> > > > >> > > >>>> >> >> > Let’s refine the Unclean
> Recovery.
> > > We
> > > > > have
> > > > > > > > >> already
> > > > > > > > >> > > > taken a
> > > > > > > > >> > > > >> > > lot of
> > > > > > > > >> > > > >> > > >>>> >> >> > suggestions and I hope to
> enhance
> > > the
> > > > > > > > >> durability of
> > > > > > > > >> > > > Kafka
> > > > > > > > >> > > > >> to
> > > > > > > > >> > > > >> > > the
> > > > > > > > >> > > > >> > > >>>> next
> > > > > > > > >> > > > >> > > >>>> >> >> level
> > > > > > > > >> > > > >> > > >>>> >> >> > with this KIP.
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> I am OK with doing the unclean
> > leader
> > > > > > recovery
> > > > > > > > >> > > > improvements
> > > > > > > > >> > > > >> in
> > > > > > > > >> > > > >> > > >>>> this KIP.
> > > > > > > > >> > > > >> > > >>>> >> >> However, I think we need to really
> > > work
> > > > on
> > > > > > the
> > > > > > > > >> > > > configuration
> > > > > > > > >> > > > >> > > >>>> settings.
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> Configuration overrides are often
> > > quite
> > > > > > messy.
> > > > > > > > For
> > > > > > > > >> > > > example,
> > > > > > > > >> > > > >> > the
> > > > > > > > >> > > > >> > > >>>> cases
> > > > > > > > >> > > > >> > > >>>> >> >> where we have log.roll.hours and
> > > > > > > > >> log.roll.segment.ms
> > > > > > > > >> > ,
> > > > > > > > >> > > > the
> > > > > > > > >> > > > >> > user
> > > > > > > > >> > > > >> > > >>>> has to
> > > > > > > > >> > > > >> > > >>>> >> >> remember which one takes
> precedence,
> > > and
> > > > > it
> > > > > > is
> > > > > > > > not
> > > > > > > > >> > > > obvious.
> > > > > > > > >> > > > >> > So,
> > > > > > > > >> > > > >> > > >>>> rather
> > > > > > > > >> > > > >> > > >>>> >> than
> > > > > > > > >> > > > >> > > >>>> >> >> creating a new configuration, why
> > not
> > > > add
> > > > > > > > >> additional
> > > > > > > > >> > > > values
> > > > > > > > >> > > > >> to
> > > > > > > > >> > > > >> > > >>>> >> >> "unclean.leader.election.enable"?
> I
> > > > think
> > > > > > this
> > > > > > > > >> will
> > > > > > > > >> > be
> > > > > > > > >> > > > >> simpler
> > > > > > > > >> > > > >> > > for
> > > > > > > > >> > > > >> > > >>>> >> people
> > > > > > > > >> > > > >> > > >>>> >> >> to understand, and simpler in the
> > code
> > > > as
> > > > > > > well.
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> What if we continued to use
> > > > > > > > >> > > > "unclean.leader.election.enable"
> > > > > > > > >> > > > >> > but
> > > > > > > > >> > > > >> > > >>>> >> extended
> > > > > > > > >> > > > >> > > >>>> >> >> it so that it took a string? Then
> > the
> > > > > string
> > > > > > > > could
> > > > > > > > >> > have
> > > > > > > > >> > > > >> these
> > > > > > > > >> > > > >> > > >>>> values:
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> never
> > > > > > > > >> > > > >> > > >>>> >> >>     never automatically do an
> > unclean
> > > > > leader
> > > > > > > > >> election
> > > > > > > > >> > > > under
> > > > > > > > >> > > > >> > any
> > > > > > > > >> > > > >> > > >>>> >> conditions
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> false / default
> > > > > > > > >> > > > >> > > >>>> >> >>     only do an unclean leader
> > election
> > > > if
> > > > > > > there
> > > > > > > > >> may
> > > > > > > > >> > be
> > > > > > > > >> > > > >> > possible
> > > > > > > > >> > > > >> > > >>>> data
> > > > > > > > >> > > > >> > > >>>> >> loss
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> true / always
> > > > > > > > >> > > > >> > > >>>> >> >>     always do an unclean leader
> > > election
> > > > > if
> > > > > > we
> > > > > > > > >> can't
> > > > > > > > >> > > > >> > immediately
> > > > > > > > >> > > > >> > > >>>> elect a
> > > > > > > > >> > > > >> > > >>>> >> >> leader
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> It's a bit awkward that false maps
> > to
> > > > > > default
> > > > > > > > >> rather
> > > > > > > > >> > > > than to
> > > > > > > > >> > > > >> > > >>>> never. But
> > > > > > > > >> > > > >> > > >>>> >> >> this awkwardness exists if we use
> > two
> > > > > > > different
> > > > > > > > >> > > > >> configuration
> > > > > > > > >> > > > >> > > keys
> > > > > > > > >> > > > >> > > >>>> as
> > > > > > > > >> > > > >> > > >>>> >> well.
> > > > > > > > >> > > > >> > > >>>> >> >> The reason for the awkwardness is
> > that
> > > > we
> > > > > > > simply
> > > > > > > > >> > don't
> > > > > > > > >> > > > want
> > > > > > > > >> > > > >> > most
> > > > > > > > >> > > > >> > > >>>> of the
> > > > > > > > >> > > > >> > > >>>> >> >> people currently setting
> > > > > > > > >> > > > >> unclean.leader.election.enable=false
> > > > > > > > >> > > > >> > to
> > > > > > > > >> > > > >> > > >>>> get the
> > > > > > > > >> > > > >> > > >>>> >> >> "never" behavior. We have to bite
> > that
> > > > > > bullet.
> > > > > > > > >> Better
> > > > > > > > >> > > to
> > > > > > > > >> > > > be
> > > > > > > > >> > > > >> > > clear
> > > > > > > > >> > > > >> > > >>>> and
> > > > > > > > >> > > > >> > > >>>> >> >> explicit than hide it.
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> Another thing that's a bit awkward
> > is
> > > > > having
> > > > > > > two
> > > > > > > > >> > > > different
> > > > > > > > >> > > > >> > ways
> > > > > > > > >> > > > >> > > to
> > > > > > > > >> > > > >> > > >>>> do
> > > > > > > > >> > > > >> > > >>>> >> >> unclean leader election specified
> in
> > > the
> > > > > > KIP.
> > > > > > > > You
> > > > > > > > >> > > > descirbe
> > > > > > > > >> > > > >> two
> > > > > > > > >> > > > >> > > >>>> methods:
> > > > > > > > >> > > > >> > > >>>> >> the
> > > > > > > > >> > > > >> > > >>>> >> >> simple "choose the last leader"
> > > method,
> > > > > and
> > > > > > > the
> > > > > > > > >> > > "unclean
> > > > > > > > >> > > > >> > > recovery
> > > > > > > > >> > > > >> > > >>>> >> manager"
> > > > > > > > >> > > > >> > > >>>> >> >> method. I understand why you did
> it
> > > this
> > > > > way
> > > > > > > --
> > > > > > > > >> > "choose
> > > > > > > > >> > > > the
> > > > > > > > >> > > > >> > last
> > > > > > > > >> > > > >> > > >>>> >> leader" is
> > > > > > > > >> > > > >> > > >>>> >> >> simple, and will help us deliver
> an
> > > > > > > > implementation
> > > > > > > > >> > > > quickly,
> > > > > > > > >> > > > >> > > while
> > > > > > > > >> > > > >> > > >>>> the
> > > > > > > > >> > > > >> > > >>>> >> URM
> > > > > > > > >> > > > >> > > >>>> >> >> is preferable in the long term. My
> > > > > > suggestion
> > > > > > > > >> here is
> > > > > > > > >> > > to
> > > > > > > > >> > > > >> > > separate
> > > > > > > > >> > > > >> > > >>>> the
> > > > > > > > >> > > > >> > > >>>> >> >> decision of HOW to do unclean
> leader
> > > > > > election
> > > > > > > > from
> > > > > > > > >> > the
> > > > > > > > >> > > > >> > decision
> > > > > > > > >> > > > >> > > of
> > > > > > > > >> > > > >> > > >>>> WHEN
> > > > > > > > >> > > > >> > > >>>> >> to
> > > > > > > > >> > > > >> > > >>>> >> >> do it.
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> So in other words, have
> > > > > > > > >> > > "unclean.leader.election.enable"
> > > > > > > > >> > > > >> > specify
> > > > > > > > >> > > > >> > > >>>> when we
> > > > > > > > >> > > > >> > > >>>> >> >> do unclean leader election, and
> > have a
> > > > new
> > > > > > > > >> > > configuration
> > > > > > > > >> > > > >> like
> > > > > > > > >> > > > >> > > >>>> >> >> "unclean.recovery.manager.enable"
> to
> > > > > > determine
> > > > > > > > if
> > > > > > > > >> we
> > > > > > > > >> > > use
> > > > > > > > >> > > > the
> > > > > > > > >> > > > >> > > URM.
> > > > > > > > >> > > > >> > > >>>> >> >> Presumably the URM will take some
> > time
> > > > to
> > > > > > get
> > > > > > > > >> fully
> > > > > > > > >> > > > stable,
> > > > > > > > >> > > > >> so
> > > > > > > > >> > > > >> > > >>>> this can
> > > > > > > > >> > > > >> > > >>>> >> >> default to false for a while, and
> we
> > > can
> > > > > > flip
> > > > > > > > the
> > > > > > > > >> > > > default to
> > > > > > > > >> > > > >> > > true
> > > > > > > > >> > > > >> > > >>>> when
> > > > > > > > >> > > > >> > > >>>> >> we
> > > > > > > > >> > > > >> > > >>>> >> >> feel ready.
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> The URM is somewhat
> under-described
> > > > here.
> > > > > I
> > > > > > > > think
> > > > > > > > >> we
> > > > > > > > >> > > > need a
> > > > > > > > >> > > > >> > few
> > > > > > > > >> > > > >> > > >>>> >> >> configurations here for it. For
> > > example,
> > > > > we
> > > > > > > > need a
> > > > > > > > >> > > > >> > > configuration to
> > > > > > > > >> > > > >> > > >>>> >> specify
> > > > > > > > >> > > > >> > > >>>> >> >> how long it should wait for a
> broker
> > > to
> > > > > > > respond
> > > > > > > > to
> > > > > > > > >> > its
> > > > > > > > >> > > > RPCs
> > > > > > > > >> > > > >> > > before
> > > > > > > > >> > > > >> > > >>>> >> moving
> > > > > > > > >> > > > >> > > >>>> >> >> on. We also need to understand how
> > the
> > > > URM
> > > > > > > > >> interacts
> > > > > > > > >> > > with
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > unclean.leader.election.enable=always. I
> > > > > > > assume
> > > > > > > > >> that
> > > > > > > > >> > > with
> > > > > > > > >> > > > >> > > "always"
> > > > > > > > >> > > > >> > > >>>> we
> > > > > > > > >> > > > >> > > >>>> >> will
> > > > > > > > >> > > > >> > > >>>> >> >> just unconditionally use the URM
> > > rather
> > > > > than
> > > > > > > > >> choosing
> > > > > > > > >> > > > >> > randomly.
> > > > > > > > >> > > > >> > > >>>> But this
> > > > > > > > >> > > > >> > > >>>> >> >> should be spelled out in the KIP.
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > > >> > > > >> > > >>>> >> >> > DescribeTopicRequest
> > > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > > >> > > > >> > > >>>> >> >> >    1.
> > > > > > > > >> > > > >> > > >>>> >> >> >    Yes, the plan is to replace
> the
> > > > > > > > >> MetadataRequest
> > > > > > > > >> > > with
> > > > > > > > >> > > > >> the
> > > > > > > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest for the
> > > admin
> > > > > > > clients.
> > > > > > > > >> Will
> > > > > > > > >> > > > check
> > > > > > > > >> > > > >> > the
> > > > > > > > >> > > > >> > > >>>> details.
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> Sounds good. But as I said, you
> need
> > > to
> > > > > > > specify
> > > > > > > > >> how
> > > > > > > > >> > > > >> > AdminClient
> > > > > > > > >> > > > >> > > >>>> >> interacts
> > > > > > > > >> > > > >> > > >>>> >> >> with the new request. This will
> > > involve
> > > > > > adding
> > > > > > > > >> some
> > > > > > > > >> > > > fields
> > > > > > > > >> > > > >> to
> > > > > > > > >> > > > >> > > >>>> >> >> TopicDescription.java. And you
> need
> > to
> > > > > > specify
> > > > > > > > the
> > > > > > > > >> > > > changes
> > > > > > > > >> > > > >> to
> > > > > > > > >> > > > >> > > the
> > > > > > > > >> > > > >> > > >>>> >> >> kafka-topics.sh command line tool.
> > > > > Otherwise
> > > > > > > we
> > > > > > > > >> > cannot
> > > > > > > > >> > > > use
> > > > > > > > >> > > > >> the
> > > > > > > > >> > > > >> > > >>>> tool to
> > > > > > > > >> > > > >> > > >>>> >> see
> > > > > > > > >> > > > >> > > >>>> >> >> the new information.
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> The new requests,
> > DescribeTopicRequest
> > > > and
> > > > > > > > >> > > > >> > > >>>> GetReplicaLogInfoRequest,
> > > > > > > > >> > > > >> > > >>>> >> need
> > > > > > > > >> > > > >> > > >>>> >> >> to have limits placed on them so
> > that
> > > > > their
> > > > > > > size
> > > > > > > > >> > can't
> > > > > > > > >> > > be
> > > > > > > > >> > > > >> > > >>>> infinite. We
> > > > > > > > >> > > > >> > > >>>> >> >> don't want to propagate the
> current
> > > > > problems
> > > > > > > of
> > > > > > > > >> > > > >> > MetadataRequest,
> > > > > > > > >> > > > >> > > >>>> where
> > > > > > > > >> > > > >> > > >>>> >> >> clients can request massive
> > responses
> > > > that
> > > > > > can
> > > > > > > > >> mess
> > > > > > > > >> > up
> > > > > > > > >> > > > the
> > > > > > > > >> > > > >> JVM
> > > > > > > > >> > > > >> > > when
> > > > > > > > >> > > > >> > > >>>> >> handled.
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> Adding limits is simple for
> > > > > > > > >> GetReplicaLogInfoRequest
> > > > > > > > >> > --
> > > > > > > > >> > > > we
> > > > > > > > >> > > > >> can
> > > > > > > > >> > > > >> > > >>>> just say
> > > > > > > > >> > > > >> > > >>>> >> >> that only 2000 partitions at a
> time
> > > can
> > > > be
> > > > > > > > >> requested.
> > > > > > > > >> > > For
> > > > > > > > >> > > > >> > > >>>> >> >> DescribeTopicRequest we can
> probably
> > > > just
> > > > > > > limit
> > > > > > > > >> to 20
> > > > > > > > >> > > > topics
> > > > > > > > >> > > > >> > or
> > > > > > > > >> > > > >> > > >>>> >> something
> > > > > > > > >> > > > >> > > >>>> >> >> like that, to avoid the complexity
> > of
> > > > > doing
> > > > > > > > >> > pagination
> > > > > > > > >> > > in
> > > > > > > > >> > > > >> this
> > > > > > > > >> > > > >> > > KIP.
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> >    2.
> > > > > > > > >> > > > >> > > >>>> >> >> >    I can let the broker load the
> > ELR
> > > > > info
> > > > > > so
> > > > > > > > >> that
> > > > > > > > >> > > they
> > > > > > > > >> > > > can
> > > > > > > > >> > > > >> > > serve
> > > > > > > > >> > > > >> > > >>>> the
> > > > > > > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest as well.
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> Yes, it's fine to add to
> > > MetadataCache.
> > > > In
> > > > > > > fact,
> > > > > > > > >> > you'll
> > > > > > > > >> > > > be
> > > > > > > > >> > > > >> > > loading
> > > > > > > > >> > > > >> > > >>>> it
> > > > > > > > >> > > > >> > > >>>> >> >> anyway once it's added to
> > > > PartitionImage.
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> >    3.
> > > > > > > > >> > > > >> > > >>>> >> >> >    Yeah, it does not make sense
> to
> > > > have
> > > > > > the
> > > > > > > > >> topic
> > > > > > > > >> > id
> > > > > > > > >> > > if
> > > > > > > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest is only
> > used
> > > > by
> > > > > > the
> > > > > > > > >> admin
> > > > > > > > >> > > > client.
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> OK. That makes things simpler. We
> > can
> > > > > always
> > > > > > > > >> create a
> > > > > > > > >> > > new
> > > > > > > > >> > > > >> API
> > > > > > > > >> > > > >> > > later
> > > > > > > > >> > > > >> > > >>>> >> >> (hopefully not in this KIP!) to
> > query
> > > by
> > > > > > topic
> > > > > > > > ID.
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > > >> > > > >> > > >>>> >> >> > Metrics
> > > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > > >> > > > >> > > >>>> >> >> > As for overall cluster health
> > > > metrics, I
> > > > > > > think
> > > > > > > > >> > > > >> under-min-ISR
> > > > > > > > >> > > > >> > > is
> > > > > > > > >> > > > >> > > >>>> still
> > > > > > > > >> > > > >> > > >>>> >> a
> > > > > > > > >> > > > >> > > >>>> >> >> > useful one. ELR is more like a
> > > safety
> > > > > > belt.
> > > > > > > > When
> > > > > > > > >> > the
> > > > > > > > >> > > > ELR
> > > > > > > > >> > > > >> is
> > > > > > > > >> > > > >> > > >>>> used, the
> > > > > > > > >> > > > >> > > >>>> >> >> > cluster availability has already
> > > been
> > > > > > > > impacted.
> > > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > > >> > > > >> > > >>>> >> >> > Maybe we can have a metric to
> > count
> > > > the
> > > > > > > > >> partitions
> > > > > > > > >> > > that
> > > > > > > > >> > > > >> > > sum(ISR,
> > > > > > > > >> > > > >> > > >>>> ELR)
> > > > > > > > >> > > > >> > > >>>> >> <
> > > > > > > > >> > > > >> > > >>>> >> >> min
> > > > > > > > >> > > > >> > > >>>> >> >> > ISR. What do you think?
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> How about:
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> A.  a metric for the totoal number
> > of
> > > > > > > > >> under-min-isr
> > > > > > > > >> > > > >> > partitions?
> > > > > > > > >> > > > >> > > We
> > > > > > > > >> > > > >> > > >>>> don't
> > > > > > > > >> > > > >> > > >>>> >> >> have that in Apache Kafka at the
> > > moment.
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> B. a metric for the number of
> > unclean
> > > > > leader
> > > > > > > > >> > elections
> > > > > > > > >> > > we
> > > > > > > > >> > > > >> did
> > > > > > > > >> > > > >> > > (for
> > > > > > > > >> > > > >> > > >>>> >> >> simplicity, it can reset to 0 on
> > > > > controller
> > > > > > > > >> restart:
> > > > > > > > >> > we
> > > > > > > > >> > > > >> expect
> > > > > > > > >> > > > >> > > >>>> people to
> > > > > > > > >> > > > >> > > >>>> >> >> monitor the change over time
> anyway)
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> best,
> > > > > > > > >> > > > >> > > >>>> >> >> Colin
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > > >> > > > >> > > >>>> >> >> > Yeah, for the ongoing unclean
> > > > > recoveries,
> > > > > > > the
> > > > > > > > >> > > > controller
> > > > > > > > >> > > > >> can
> > > > > > > > >> > > > >> > > >>>> keep an
> > > > > > > > >> > > > >> > > >>>> >> >> > accurate count through failover
> > > > because
> > > > > > > > >> partition
> > > > > > > > >> > > > >> > registration
> > > > > > > > >> > > > >> > > >>>> can
> > > > > > > > >> > > > >> > > >>>> >> >> indicate
> > > > > > > > >> > > > >> > > >>>> >> >> > whether a recovery is needed.
> > > However,
> > > > > for
> > > > > > > the
> > > > > > > > >> > > happened
> > > > > > > > >> > > > >> > ones,
> > > > > > > > >> > > > >> > > >>>> unless
> > > > > > > > >> > > > >> > > >>>> >> we
> > > > > > > > >> > > > >> > > >>>> >> >> > want to persist the number
> > > somewhere,
> > > > we
> > > > > > can
> > > > > > > > >> only
> > > > > > > > >> > > > figure
> > > > > > > > >> > > > >> it
> > > > > > > > >> > > > >> > > out
> > > > > > > > >> > > > >> > > >>>> from
> > > > > > > > >> > > > >> > > >>>> >> the
> > > > > > > > >> > > > >> > > >>>> >> >> > log.
> > > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > > >> > > > >> > > >>>> >> >> > On Tue, Sep 12, 2023 at 3:16 PM
> > > Colin
> > > > > > > McCabe <
> > > > > > > > >> > > > >> > > cmccabe@apache.org
> > > > > > > > >> > > > >> > > >>>> >
> > > > > > > > >> > > > >> > > >>>> >> wrote:
> > > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > > >> > > > >> > > >>>> >> >> >> Also, we should have metrics
> that
> > > > show
> > > > > > what
> > > > > > > > is
> > > > > > > > >> > going
> > > > > > > > >> > > > on
> > > > > > > > >> > > > >> > with
> > > > > > > > >> > > > >> > > >>>> regard
> > > > > > > > >> > > > >> > > >>>> >> to
> > > > > > > > >> > > > >> > > >>>> >> >> the
> > > > > > > > >> > > > >> > > >>>> >> >> >> eligible replica set. I'm not
> > sure
> > > > > > exactly
> > > > > > > > >> what to
> > > > > > > > >> > > > >> suggest,
> > > > > > > > >> > > > >> > > but
> > > > > > > > >> > > > >> > > >>>> >> >> something
> > > > > > > > >> > > > >> > > >>>> >> >> >> that could identify when things
> > are
> > > > > going
> > > > > > > > >> wrong in
> > > > > > > > >> > > the
> > > > > > > > >> > > > >> > > clsuter.
> > > > > > > > >> > > > >> > > >>>> >> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> >> For example, maybe a metric for
> > > > > > partitions
> > > > > > > > >> > > containing
> > > > > > > > >> > > > >> > > replicas
> > > > > > > > >> > > > >> > > >>>> that
> > > > > > > > >> > > > >> > > >>>> >> are
> > > > > > > > >> > > > >> > > >>>> >> >> >> ineligible to be leader? That
> > would
> > > > > show
> > > > > > a
> > > > > > > > >> spike
> > > > > > > > >> > > when
> > > > > > > > >> > > > a
> > > > > > > > >> > > > >> > > broker
> > > > > > > > >> > > > >> > > >>>> had an
> > > > > > > > >> > > > >> > > >>>> >> >> >> unclean restart.
> > > > > > > > >> > > > >> > > >>>> >> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> >> Ideally, we'd also have a
> metric
> > > that
> > > > > > > > indicates
> > > > > > > > >> > when
> > > > > > > > >> > > > an
> > > > > > > > >> > > > >> > > unclear
> > > > > > > > >> > > > >> > > >>>> >> leader
> > > > > > > > >> > > > >> > > >>>> >> >> >> election or a recovery
> happened.
> > > > It's a
> > > > > > bit
> > > > > > > > >> tricky
> > > > > > > > >> > > > >> because
> > > > > > > > >> > > > >> > > the
> > > > > > > > >> > > > >> > > >>>> simple
> > > > > > > > >> > > > >> > > >>>> >> >> >> thing, of tracking it per
> > > controller,
> > > > > may
> > > > > > > be
> > > > > > > > a
> > > > > > > > >> bit
> > > > > > > > >> > > > >> > confusing
> > > > > > > > >> > > > >> > > >>>> during
> > > > > > > > >> > > > >> > > >>>> >> >> >> failovers.
> > > > > > > > >> > > > >> > > >>>> >> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> >> best,
> > > > > > > > >> > > > >> > > >>>> >> >> >> Colin
> > > > > > > > >> > > > >> > > >>>> >> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> >>
> > > > > > > > >> > > > >> > > >>>> >> >> >> On Tue, Sep 12, 2023, at 14:25,
> > > Colin
> > > > > > > McCabe
> > > > > > > > >> > wrote:
> > > > > > > > >> > > > >> > > >>>> >> >> >> > Hi Calvin,
> > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > > > >> > > > >> > > >>>> >> >> >> > Thanks for the KIP. I think
> > this
> > > > is a
> > > > > > > great
> > > > > > > > >> > > > >> improvement.
> > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > > > >> > > > >> > > >>>> >> >> >> >> Additional High Watermark
> > > advance
> > > > > > > > >> requirement
> > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > > > >> > > > >> > > >>>> >> >> >> > Typo: change "advance" to
> > > > > "advancement"
> > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > > > >> > > > >> > > >>>> >> >> >> >> A bit recap of some key
> > > concepts.
> > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > > > >> > > > >> > > >>>> >> >> >> > Typo: change "bit" to "quick"
> > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > > > >> > > > >> > > >>>> >> >> >> >> Ack=1/all produce request.
> It
> > > > > defines
> > > > > > > when
> > > > > > > > >> the
> > > > > > > > >> > > > Kafka
> > > > > > > > >> > > > >> > > server
> > > > > > > > >> > > > >> > > >>>> should
> > > > > > > > >> > > > >> > > >>>> >> >> >> respond to the produce request
> > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > > > >> > > > >> > > >>>> >> >> >> > I think this section would be
> > > > clearer
> > > > > > if
> > > > > > > we
> > > > > > > > >> > talked
> > > > > > > > >> > > > >> about
> > > > > > > > >> > > > >> > > the
> > > > > > > > >> > > > >> > > >>>> new
> > > > > > > > >> > > > >> > > >>>> >> high
> > > > > > > > >> > > > >> > > >>>> >> >> >> > watermark advancement
> > requirement
> > > > > > first,
> > > > > > > > and
> > > > > > > > >> > THEN
> > > > > > > > >> > > > >> talked
> > > > > > > > >> > > > >> > > >>>> about its
> > > > > > > > >> > > > >> > > >>>> >> >> >> > impact on acks=0, acks=1, and
> > > > > > >  acks=all.
> > > > > > > > >> > > > acks=all
> > > > > > > > >> > > > >> is
> > > > > > > > >> > > > >> > of
> > > > > > > > >> > > > >> > > >>>> course
> > > > > > > > >> > > > >> > > >>>> >> the
> > > > > > > > >> > > > >> > > >>>> >> >> >> > main case we care about here,
> > so
> > > it
> > > > > > would
> > > > > > > > be
> > > > > > > > >> > good
> > > > > > > > >> > > to
> > > > > > > > >> > > > >> lead
> > > > > > > > >> > > > >> > > with
> > > > > > > > >> > > > >> > > >>>> >> that,
> > > > > > > > >> > > > >> > > >>>> >> >> >> > rather than delving into the
> > > > > > > technicalities
> > > > > > > > >> of
> > > > > > > > >> > > > acks=0/1
> > > > > > > > >> > > > >> > > first.
> > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > > > >> > > > >> > > >>>> >> >> >> >> Unclean recovery
> > > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > > > >> > > > >> > > >>>> >> >> >> > So, here you are introducing
> a
> > > new
> > > > > > > > >> > configuration,
> > > > > > > > >> > > > >> > > >>>> >> >> >> > unclean.recovery.strategy.
> The
> > > > > > difficult
> > > > > > > > >> thing
> > > > > > > > >> > > here
> > > > > > > > >> > > > is
> > > > > > > > >> > > > >> > that
> > > > > > > > >> > > > >> > > >>>> there
> > > > > > > > >> > > > >> > > >>>> >> is a
> > > > > > > > >> > > > >> > > >>>> >> >> >> > lot of overlap with
> > > > > > > > >> > > unclean.leader.election.enable.
> > > > > > > > >> > > > So
> > > > > > > > >> > > > >> we
> > > > > > > > >> > > > >> > > >>>> have 3
> > > > > > > > >> > > > >> > > >>>> >> >> >> > different settings for
> > > > > > > > >> > unclean.recovery.strategy,
> > > > > > > > >> > > > plus
> > > > > > > > >> > > > >> 2
> > > > > > > > >> > > > >> > > >>>> different
> > > > > > > > >> > > > >> > > >>>> >> >> >> > settings for
> > > > > > > > unclean.leader.election.enable,
> > > > > > > > >> > > giving
> > > > > > > > >> > > > a
> > > > > > > > >> > > > >> > cross
> > > > > > > > >> > > > >> > > >>>> >> product of
> > > > > > > > >> > > > >> > > >>>> >> >> >> > 6 different options. The
> > > following
> > > > > > > "unclean
> > > > > > > > >> > > recovery
> > > > > > > > >> > > > >> > > manager"
> > > > > > > > >> > > > >> > > >>>> >> section
> > > > > > > > >> > > > >> > > >>>> >> >> >> > on
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

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

60.  If the range query has errors for some of the partitions, do we expect
different responses when querying particular partitions?

Thanks,

Jun

On Tue, Oct 10, 2023 at 10:50 AM Calvin Liu <ca...@confluent.io.invalid>
wrote:

> Hi Jun
> 60. Yes, it is a good question. I was thinking the API could be flexible to
> query the particular partitions if the range query has errors for some of
> the partitions. Not sure whether it is a valid assumption, what do you
> think?
>
> 61. Good point, I will update them to partition level with the same limit.
>
> 62. Sure, will do.
>
> Thanks
>
> On Tue, Oct 10, 2023 at 10:12 AM Jun Rao <ju...@confluent.io.invalid> wrote:
>
> > Hi, Calvin,
> >
> > A few more minor comments on your latest update.
> >
> > 60. DescribeTopicRequest: When will the Partitions field be used? It
> seems
> > that the FirstPartitionId field is enough for AdminClient usage.
> >
> > 61. Could we make the limit for DescribeTopicRequest,
> ElectLeadersRequest,
> > GetReplicaLogInfo consistent? Currently, ElectLeadersRequest's limit is
> at
> > topic level and GetReplicaLogInfo has a different partition level limit
> > from DescribeTopicRequest.
> >
> > 62. Should ElectLeadersRequest.DesiredLeaders be at the same level as
> > ElectLeadersRequest.TopicPartitions.Partitions? In the KIP, it looks like
> > it's at the same level as ElectLeadersRequest.TopicPartitions.
> >
> > Thanks,
> >
> > Jun
> >
> > On Wed, Oct 4, 2023 at 3:55 PM Calvin Liu <ca...@confluent.io.invalid>
> > wrote:
> >
> > > Hi David,
> > > Thanks for the comments.
> > > ----
> > > I thought that a new snapshot with the downgraded MV is created in this
> > > case. Isn’t it the case?
> > > Yes, you are right, a metadata delta will be generated after the MV
> > > downgrade. Then the user can start the software downgrade.
> > > -----
> > > Could you also elaborate a bit more on the reasoning behind adding the
> > > limits to the admin RPCs? This is a new pattern in Kafka so it would be
> > > good to clear on the motivation.
> > > Thanks to Colin for bringing it up. The current MetadataRequest does
> not
> > > have a limit on the number of topics to query in a single request.
> > Massive
> > > requests can mess up the JVM. We want to have some sort of throttle on
> > the
> > > new APIs.
> > > -----
> > > Could you also explain how the client is supposed to handle the
> > > topics/partitions above the limit? I suppose that it will have to retry
> > > those, correct?
> > > Corrent. For the official admin clients, it will split the large
> request
> > > into proper pieces and query one after another.
> > > -----
> > > My understanding is that the topics/partitions above the limit will be
> > > failed with an invalid exception error. I wonder if this choice is
> > > judicious because the invalide request exception is usually fatal. It
> may
> > > be better to use an new and explicit error for this case.
> > >
> > > Thanks for bringing this up. How about "REQUEST_LIMIT_REACHED"?
> > > --------
> > > It seems that we still need to specify the changes to the admin api to
> > > accommodate the new or updated apis. Do you plan to add them?
> > > Try to cover the following
> > > 1. The admin client will use the new DescribeTopicRequest to query the
> > > topics
> > > 2. Mention the API limit and the new retriable error.
> > > 3. Output changes for the admin client when describing a topic (new
> > fields
> > > of ELR...)
> > > 4. Changes to data structures like TopicPartitionInfo to include the
> ELR.
> > > Anything else I missed?
> > >
> > > Thanks!
> > >
> > >
> > >
> > >
> > >
> > > On Wed, Oct 4, 2023 at 12:27 PM David Jacot <da...@gmail.com>
> > wrote:
> > >
> > > > Hi Calvin,
> > > >
> > > > I thought that a new snapshot with the downgraded MV is created in
> this
> > > > case. Isn’t it the case?
> > > >
> > > > Could you also elaborate a bit more on the reasoning behind adding
> the
> > > > limits to the admin RPCs? This is a new pattern in Kafka so it would
> be
> > > > good to clear on the motivation.
> > > >
> > > > Could you also explain how the client is supposed to handle the
> > > > topics/partitions above the limit? I suppose that it will have to
> retry
> > > > those, correct?
> > > >
> > > > My understanding is that the topics/partitions above the limit will
> be
> > > > failed with an invalid exception error. I wonder if this choice is
> > > > judicious because the invalide request exception is usually fatal. It
> > may
> > > > be better to use an new and explicit error for this case.
> > > >
> > > > It seems that we still need to specify the changes to the admin api
> to
> > > > accommodate the new or updated apis. Do you plan to add them?
> > > >
> > > > Best,
> > > > David
> > > >
> > > > Le mer. 4 oct. 2023 à 20:39, Calvin Liu <ca...@confluent.io.invalid>
> a
> > > > écrit :
> > > >
> > > > > Hi Jun,
> > > > > After the MV downgrade, the controller will write in the old
> version
> > of
> > > > the
> > > > > PartitionRecord/PartitionChangeRecord. If I understand correctly,
> it
> > is
> > > > > possible to downgrade the software version if the controller only
> has
> > > to
> > > > > handle old version records.
> > > > > However, the controller will not automatically rewrite the
> > > > PartitionRecord
> > > > > with the old version unless there is a partition update. Then, the
> > user
> > > > may
> > > > > have to wait an unknown amount of time before the software
> downgrades
> > > > > unless they do a roll to force update every partition. If it makes
> > > > sense, I
> > > > > can mention these steps to do a software downgrade.
> > > > > Thanks
> > > > >
> > > > > On Wed, Oct 4, 2023 at 11:20 AM Jun Rao <ju...@confluent.io.invalid>
> > > > wrote:
> > > > >
> > > > > > Hi, Calvin and Justine,
> > > > > >
> > > > > > Historically, when we change the record format in the log, we
> don't
> > > > > support
> > > > > > software version downgrading.
> > > > > >
> > > > > > For the record format change in the metadata log, have we thought
> > > about
> > > > > > forcing the write of the latest metadata records with the old
> > version
> > > > > > during MV downgrading? This will in theory allow the old version
> of
> > > the
> > > > > > software to obtain the latest metadata.
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > > On Wed, Oct 4, 2023 at 9:53 AM Justine Olshan
> > > > > <jolshan@confluent.io.invalid
> > > > > > >
> > > > > > wrote:
> > > > > >
> > > > > > > Sorry -- not MV but software version.
> > > > > > >
> > > > > > > On Wed, Oct 4, 2023 at 9:51 AM Justine Olshan <
> > > jolshan@confluent.io>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Catching up with this discussion.
> > > > > > > >
> > > > > > > > I was just curious -- have we had other instances where
> > > downgrading
> > > > > MV
> > > > > > is
> > > > > > > > not supported? I think Kafka typically tries to support
> > > downgrades,
> > > > > > and I
> > > > > > > > couldn't think of other examples.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Justine
> > > > > > > >
> > > > > > > > On Wed, Oct 4, 2023 at 9:40 AM Calvin Liu
> > > > <caliu@confluent.io.invalid
> > > > > >
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > >> Hi Jun,
> > > > > > > >> 54. Marked the software downgrading is not supported. As the
> > old
> > > > > > > >> controller
> > > > > > > >> will not understand the new PartitionRecord and
> > > > > PartitionChangeRecord.
> > > > > > > >> Thanks!
> > > > > > > >>
> > > > > > > >> On Wed, Oct 4, 2023 at 9:12 AM Jun Rao
> > <jun@confluent.io.invalid
> > > >
> > > > > > > wrote:
> > > > > > > >>
> > > > > > > >> > Hi, Calvin,
> > > > > > > >> >
> > > > > > > >> > Thanks for the reply. Just one more comment.
> > > > > > > >> >
> > > > > > > >> > 54. It seems that downgrading MV is supported. Is
> > downgrading
> > > > the
> > > > > > > >> software
> > > > > > > >> > version supported? It would be useful to document that.
> > > > > > > >> >
> > > > > > > >> > Thanks,
> > > > > > > >> >
> > > > > > > >> > Jun
> > > > > > > >> >
> > > > > > > >> > On Tue, Oct 3, 2023 at 4:55 PM Artem Livshits
> > > > > > > >> > <al...@confluent.io.invalid> wrote:
> > > > > > > >> >
> > > > > > > >> > > Hi Colin,
> > > > > > > >> > >
> > > > > > > >> > > I think in your example "do_unclean_recovery" would need
> > to
> > > do
> > > > > > > >> different
> > > > > > > >> > > things depending on the strategy.
> > > > > > > >> > >
> > > > > > > >> > > do_unclean_recovery() {
> > > > > > > >> > >    if (unclean.recovery.manager.enabled) {
> > > > > > > >> > >     if (strategy == Aggressive)
> > > > > > > >> > >       use UncleanRecoveryManager(waitLastKnownERL=false)
> > //
> > > > > just
> > > > > > > >> inspect
> > > > > > > >> > > logs from whoever is available
> > > > > > > >> > >     else
> > > > > > > >> > >       use  UncleanRecoveryManager(waitLastKnownERL=true)
> > //
> > > > > must
> > > > > > > wait
> > > > > > > >> > for
> > > > > > > >> > > at least last known ELR
> > > > > > > >> > >   } else {
> > > > > > > >> > >     if (strategy == Aggressive)
> > > > > > > >> > >       choose the last known leader if that is available,
> > or
> > > a
> > > > > > random
> > > > > > > >> > leader
> > > > > > > >> > > if not)
> > > > > > > >> > >     else
> > > > > > > >> > >       wait for last known leader to get back
> > > > > > > >> > >   }
> > > > > > > >> > > }
> > > > > > > >> > >
> > > > > > > >> > > The idea is that the Aggressive strategy would kick in
> as
> > > soon
> > > > > as
> > > > > > we
> > > > > > > >> lost
> > > > > > > >> > > the leader and would pick a leader from whoever is
> > > available;
> > > > > but
> > > > > > > the
> > > > > > > >> > > Balanced will only kick in when ELR is empty and will
> wait
> > > for
> > > > > the
> > > > > > > >> > brokers
> > > > > > > >> > > that likely have most data to be available.
> > > > > > > >> > >
> > > > > > > >> > > On Tue, Oct 3, 2023 at 3:04 PM Colin McCabe <
> > > > cmccabe@apache.org
> > > > > >
> > > > > > > >> wrote:
> > > > > > > >> > >
> > > > > > > >> > > > On Tue, Oct 3, 2023, at 10:49, Jun Rao wrote:
> > > > > > > >> > > > > Hi, Calvin,
> > > > > > > >> > > > >
> > > > > > > >> > > > > Thanks for the update KIP. A few more comments.
> > > > > > > >> > > > >
> > > > > > > >> > > > > 41. Why would a user choose the option to select a
> > > random
> > > > > > > replica
> > > > > > > >> as
> > > > > > > >> > > the
> > > > > > > >> > > > > leader instead of using
> > > > unclean.recovery.strateg=Aggressive?
> > > > > > It
> > > > > > > >> seems
> > > > > > > >> > > > that
> > > > > > > >> > > > > the latter is strictly better? If that's not the
> case,
> > > > could
> > > > > > we
> > > > > > > >> fold
> > > > > > > >> > > this
> > > > > > > >> > > > > option under unclean.recovery.strategy instead of
> > > > > introducing
> > > > > > a
> > > > > > > >> > > separate
> > > > > > > >> > > > > config?
> > > > > > > >> > > >
> > > > > > > >> > > > Hi Jun,
> > > > > > > >> > > >
> > > > > > > >> > > > I thought the flow of control was:
> > > > > > > >> > > >
> > > > > > > >> > > > If there is no leader for the partition {
> > > > > > > >> > > >   If (there are unfenced ELR members) {
> > > > > > > >> > > >     choose_an_unfenced_ELR_member
> > > > > > > >> > > >   } else if (there are fenced ELR members AND
> > > > > > > strategy=Aggressive) {
> > > > > > > >> > > >     do_unclean_recovery
> > > > > > > >> > > >   } else if (there are no ELR members AND strategy !=
> > > None)
> > > > {
> > > > > > > >> > > >     do_unclean_recovery
> > > > > > > >> > > >   } else {
> > > > > > > >> > > >     do nothing about the missing leader
> > > > > > > >> > > >   }
> > > > > > > >> > > > }
> > > > > > > >> > > >
> > > > > > > >> > > > do_unclean_recovery() {
> > > > > > > >> > > >    if (unclean.recovery.manager.enabled) {
> > > > > > > >> > > >     use UncleanRecoveryManager
> > > > > > > >> > > >   } else {
> > > > > > > >> > > >     choose the last known leader if that is available,
> > or
> > > a
> > > > > > random
> > > > > > > >> > leader
> > > > > > > >> > > > if not)
> > > > > > > >> > > >   }
> > > > > > > >> > > > }
> > > > > > > >> > > >
> > > > > > > >> > > > However, I think this could be clarified, especially
> the
> > > > > > behavior
> > > > > > > >> when
> > > > > > > >> > > > unclean.recovery.manager.enabled=false. Inuitively the
> > > goal
> > > > > for
> > > > > > > >> > > > unclean.recovery.manager.enabled=false is to be "the
> > same
> > > as
> > > > > > now,
> > > > > > > >> > mostly"
> > > > > > > >> > > > but it's very underspecified in the KIP, I agree.
> > > > > > > >> > > >
> > > > > > > >> > > > >
> > > > > > > >> > > > > 50. ElectLeadersRequest: "If more than 20 topics are
> > > > > included,
> > > > > > > >> only
> > > > > > > >> > the
> > > > > > > >> > > > > first 20 will be served. Others will be returned
> with
> > > > > > > >> > DesiredLeaders."
> > > > > > > >> > > > Hmm,
> > > > > > > >> > > > > not sure that I understand this.
> ElectLeadersResponse
> > > > > doesn't
> > > > > > > >> have a
> > > > > > > >> > > > > DesiredLeaders field.
> > > > > > > >> > > > >
> > > > > > > >> > > > > 51. GetReplicaLogInfo: "If more than 2000 partitions
> > are
> > > > > > > included,
> > > > > > > >> > only
> > > > > > > >> > > > the
> > > > > > > >> > > > > first 2000 will be served" Do we return an error for
> > the
> > > > > > > remaining
> > > > > > > >> > > > > partitions? Actually, should we include an errorCode
> > > field
> > > > > at
> > > > > > > the
> > > > > > > >> > > > partition
> > > > > > > >> > > > > level in GetReplicaLogInfoResponse to cover
> > non-existing
> > > > > > > >> partitions
> > > > > > > >> > and
> > > > > > > >> > > > no
> > > > > > > >> > > > > authorization, etc?
> > > > > > > >> > > > >
> > > > > > > >> > > > > 52. The entry should matches => The entry should
> match
> > > > > > > >> > > > >
> > > > > > > >> > > > > 53. ElectLeadersRequest.DesiredLeaders: Should it be
> > > > > nullable
> > > > > > > >> since a
> > > > > > > >> > > > user
> > > > > > > >> > > > > may not specify DesiredLeaders?
> > > > > > > >> > > > >
> > > > > > > >> > > > > 54. Downgrade: Is that indeed possible? I thought
> > > earlier
> > > > > you
> > > > > > > said
> > > > > > > >> > that
> > > > > > > >> > > > > once the new version of the records are in the
> > metadata
> > > > log,
> > > > > > one
> > > > > > > >> > can't
> > > > > > > >> > > > > downgrade since the old broker doesn't know how to
> > parse
> > > > the
> > > > > > new
> > > > > > > >> > > version
> > > > > > > >> > > > of
> > > > > > > >> > > > > the metadata records?
> > > > > > > >> > > > >
> > > > > > > >> > > >
> > > > > > > >> > > > MetadataVersion downgrade is currently broken but we
> > have
> > > > > fixing
> > > > > > > it
> > > > > > > >> on
> > > > > > > >> > > our
> > > > > > > >> > > > plate for Kafka 3.7.
> > > > > > > >> > > >
> > > > > > > >> > > > The way downgrade works is that "new features" are
> > > dropped,
> > > > > > > leaving
> > > > > > > >> > only
> > > > > > > >> > > > the old ones.
> > > > > > > >> > > >
> > > > > > > >> > > > > 55. CleanShutdownFile: Should we add a version field
> > for
> > > > > > future
> > > > > > > >> > > > extension?
> > > > > > > >> > > > >
> > > > > > > >> > > > > 56. Config changes are public facing. Could we have
> a
> > > > > separate
> > > > > > > >> > section
> > > > > > > >> > > to
> > > > > > > >> > > > > document all the config changes?
> > > > > > > >> > > >
> > > > > > > >> > > > +1. A separate section for this would be good.
> > > > > > > >> > > >
> > > > > > > >> > > > best,
> > > > > > > >> > > > Colin
> > > > > > > >> > > >
> > > > > > > >> > > > >
> > > > > > > >> > > > > Thanks,
> > > > > > > >> > > > >
> > > > > > > >> > > > > Jun
> > > > > > > >> > > > >
> > > > > > > >> > > > > On Mon, Sep 25, 2023 at 4:29 PM Calvin Liu
> > > > > > > >> > <caliu@confluent.io.invalid
> > > > > > > >> > > >
> > > > > > > >> > > > > wrote:
> > > > > > > >> > > > >
> > > > > > > >> > > > >> Hi Jun
> > > > > > > >> > > > >> Thanks for the comments.
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> 40. If we change to None, it is not guaranteed for
> no
> > > > data
> > > > > > > loss.
> > > > > > > >> For
> > > > > > > >> > > > users
> > > > > > > >> > > > >> who are not able to validate the data with external
> > > > > > resources,
> > > > > > > >> > manual
> > > > > > > >> > > > >> intervention does not give a better result but a
> loss
> > > of
> > > > > > > >> > availability.
> > > > > > > >> > > > So
> > > > > > > >> > > > >> practically speaking, the Balance mode would be a
> > > better
> > > > > > > default
> > > > > > > >> > > value.
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> 41. No, it represents how we want to do the unclean
> > > > leader
> > > > > > > >> election.
> > > > > > > >> > > If
> > > > > > > >> > > > it
> > > > > > > >> > > > >> is false, the unclean leader election will be the
> old
> > > > > random
> > > > > > > way.
> > > > > > > >> > > > >> Otherwise, the unclean recovery will be used.
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> 42. Good catch. Updated.
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> 43. Only the first 20 topics will be served. Others
> > > will
> > > > be
> > > > > > > >> returned
> > > > > > > >> > > > with
> > > > > > > >> > > > >> InvalidRequestError
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> 44. The order matters. The desired leader entries
> > match
> > > > > with
> > > > > > > the
> > > > > > > >> > topic
> > > > > > > >> > > > >> partition list by the index.
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> 45. Thanks! Updated.
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> 46. Good advice! Updated.
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> 47.1, updated the comment. Basically it will elect
> > the
> > > > > > replica
> > > > > > > in
> > > > > > > >> > the
> > > > > > > >> > > > >> desiredLeader field to be the leader
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> 47.2 We can let the admin client do the conversion.
> > > Using
> > > > > the
> > > > > > > >> > > > desiredLeader
> > > > > > > >> > > > >> field in the json format seems easier for users.
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> 48. Once the MV version is downgraded, all the ELR
> > > > related
> > > > > > > fields
> > > > > > > >> > will
> > > > > > > >> > > > be
> > > > > > > >> > > > >> removed on the next partition change. The
> controller
> > > will
> > > > > > also
> > > > > > > >> > ignore
> > > > > > > >> > > > the
> > > > > > > >> > > > >> ELR fields. Updated the KIP.
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> 49. Yes, it would be deprecated/removed.
> > > > > > > >> > > > >>
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> On Mon, Sep 25, 2023 at 3:49 PM Jun Rao
> > > > > > > <jun@confluent.io.invalid
> > > > > > > >> >
> > > > > > > >> > > > wrote:
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> > Hi, Calvin,
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >> > Thanks for the updated KIP. Made another pass. A
> > few
> > > > more
> > > > > > > >> comments
> > > > > > > >> > > > below.
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >> > 40. unclean.leader.election.enable.false ->
> > > > > > > >> > > > >> > unclean.recovery.strategy.Balanced: The Balanced
> > mode
> > > > > could
> > > > > > > >> still
> > > > > > > >> > > > lead to
> > > > > > > >> > > > >> > data loss. So, I am wondering if
> > > > > > > >> > > unclean.leader.election.enable.false
> > > > > > > >> > > > >> > should map to None?
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >> > 41. unclean.recovery.manager.enabled: I am not
> sure
> > > why
> > > > > we
> > > > > > > >> > introduce
> > > > > > > >> > > > this
> > > > > > > >> > > > >> > additional config. Is it the same as
> > > > > > > >> > unclean.recovery.strategy=None?
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >> > 42.
> > DescribeTopicResponse.TopicAuthorizedOperations:
> > > > > Should
> > > > > > > >> this
> > > > > > > >> > be
> > > > > > > >> > > at
> > > > > > > >> > > > >> the
> > > > > > > >> > > > >> > topic level?
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >> > 43. "Limit: 20 topics max per request": Could we
> > > > describe
> > > > > > > what
> > > > > > > >> > > > happens if
> > > > > > > >> > > > >> > the request includes more than 20 topics?
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >> > 44. ElectLeadersRequest.DesiredLeaders: Could we
> > > > describe
> > > > > > > >> whether
> > > > > > > >> > > the
> > > > > > > >> > > > >> > ordering matters?
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >> > 45. GetReplicaLogInfo.TopicPartitions: "about":
> > "The
> > > > > topic
> > > > > > > >> > > partitions
> > > > > > > >> > > > to
> > > > > > > >> > > > >> > elect leaders.": The description in "about" is
> > > > incorrect.
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >> > 46. GetReplicaLogInfoResponse: Should we nest
> > > > partitions
> > > > > > > under
> > > > > > > >> > > > topicId to
> > > > > > > >> > > > >> > be consistent with other types of responses?
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >> > 47. kafka-leader-election.sh:
> > > > > > > >> > > > >> > 47.1 Could we explain DESIGNATION?
> > > > > > > >> > > > >> > 47.2 desiredLeader: Should it be a list to match
> > the
> > > > > field
> > > > > > in
> > > > > > > >> > > > >> > ElectLeadersRequest?
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >> > 48. We could add a section on downgrade?
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >> > 49. LastKnownLeader: This seems only needed in
> the
> > > > first
> > > > > > > phase
> > > > > > > >> of
> > > > > > > >> > > > >> > delivering ELR. Will it be removed when the
> > complete
> > > > KIP
> > > > > is
> > > > > > > >> > > delivered?
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >> > Thanks,
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >> > Jun
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >> > On Tue, Sep 19, 2023 at 1:30 PM Colin McCabe <
> > > > > > > >> cmccabe@apache.org>
> > > > > > > >> > > > wrote:
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >> > > Hi Calvin,
> > > > > > > >> > > > >> > >
> > > > > > > >> > > > >> > > Thanks for the explanations. I like the idea of
> > > using
> > > > > > none,
> > > > > > > >> > > > balanced,
> > > > > > > >> > > > >> > > aggressive. We also had an offline discussion
> > about
> > > > why
> > > > > > it
> > > > > > > is
> > > > > > > >> > good
> > > > > > > >> > > > to
> > > > > > > >> > > > >> > use a
> > > > > > > >> > > > >> > > new config key (basically, so that we can
> > deprecate
> > > > the
> > > > > > old
> > > > > > > >> one
> > > > > > > >> > > > which
> > > > > > > >> > > > >> had
> > > > > > > >> > > > >> > > only false/true values in 4.0) With these
> > changes,
> > > I
> > > > am
> > > > > > +1.
> > > > > > > >> > > > >> > >
> > > > > > > >> > > > >> > > best,
> > > > > > > >> > > > >> > > Colin
> > > > > > > >> > > > >> > >
> > > > > > > >> > > > >> > > On Mon, Sep 18, 2023, at 15:54, Calvin Liu
> wrote:
> > > > > > > >> > > > >> > > > Hi Colin,
> > > > > > > >> > > > >> > > > Also, can we deprecate
> > > > unclean.leader.election.enable
> > > > > > in
> > > > > > > >> 4.0?
> > > > > > > >> > > > Before
> > > > > > > >> > > > >> > > that,
> > > > > > > >> > > > >> > > > we can have both the config
> > > > unclean.recovery.strategy
> > > > > > and
> > > > > > > >> > > > >> > > > unclean.leader.election.enable
> > > > > > > >> > > > >> > > > and using the unclean.recovery.Enabled to
> > > determine
> > > > > > which
> > > > > > > >> > config
> > > > > > > >> > > > to
> > > > > > > >> > > > >> use
> > > > > > > >> > > > >> > > > during the unclean leader election.
> > > > > > > >> > > > >> > > >
> > > > > > > >> > > > >> > > > On Mon, Sep 18, 2023 at 3:51 PM Calvin Liu <
> > > > > > > >> > caliu@confluent.io>
> > > > > > > >> > > > >> wrote:
> > > > > > > >> > > > >> > > >
> > > > > > > >> > > > >> > > >> Hi Colin,
> > > > > > > >> > > > >> > > >> For the unclean.recovery.strategy config
> name,
> > > how
> > > > > > about
> > > > > > > >> we
> > > > > > > >> > use
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > > >> following
> > > > > > > >> > > > >> > > >> None. It basically means no unclean recovery
> > > will
> > > > be
> > > > > > > >> > performed.
> > > > > > > >> > > > >> > > >> Aggressive. It means availability goes
> first.
> > > > > Whenever
> > > > > > > the
> > > > > > > >> > > > partition
> > > > > > > >> > > > >> > > can't
> > > > > > > >> > > > >> > > >> elect a durable replica, the controller will
> > try
> > > > the
> > > > > > > >> unclean
> > > > > > > >> > > > >> recovery.
> > > > > > > >> > > > >> > > >> Balanced. It is the balance point of the
> > > > > availability
> > > > > > > >> > > > >> > first(Aggressive)
> > > > > > > >> > > > >> > > >> and least availability(None). The controller
> > > > > performs
> > > > > > > >> unclean
> > > > > > > >> > > > >> recovery
> > > > > > > >> > > > >> > > when
> > > > > > > >> > > > >> > > >> both ISR and ELR are empty.
> > > > > > > >> > > > >> > > >>
> > > > > > > >> > > > >> > > >>
> > > > > > > >> > > > >> > > >> On Fri, Sep 15, 2023 at 11:42 AM Calvin Liu
> <
> > > > > > > >> > > caliu@confluent.io>
> > > > > > > >> > > > >> > wrote:
> > > > > > > >> > > > >> > > >>
> > > > > > > >> > > > >> > > >>> Hi Colin,
> > > > > > > >> > > > >> > > >>>
> > > > > > > >> > > > >> > > >>> > So, the proposal is that if someone sets
> > > > > > > >> > > > >> > > "unclean.leader.election.enable
> > > > > > > >> > > > >> > > >>> = true"...
> > > > > > > >> > > > >> > > >>>
> > > > > > > >> > > > >> > > >>>
> > > > > > > >> > > > >> > > >>> The idea is to use one of the
> > > > > > > >> unclean.leader.election.enable
> > > > > > > >> > > and
> > > > > > > >> > > > >> > > >>> unclean.recovery.strategy based on the
> > > > > > > >> > > > unclean.recovery.Enabled. A
> > > > > > > >> > > > >> > > possible
> > > > > > > >> > > > >> > > >>> version can be
> > > > > > > >> > > > >> > > >>>
> > > > > > > >> > > > >> > > >>> If unclean.recovery.Enabled:
> > > > > > > >> > > > >> > > >>>
> > > > > > > >> > > > >> > > >>> {
> > > > > > > >> > > > >> > > >>>
> > > > > > > >> > > > >> > > >>> Check unclean.recovery.strategy. If set,
> use
> > > it.
> > > > > > > >> Otherwise,
> > > > > > > >> > > > check
> > > > > > > >> > > > >> > > >>> unclean.leader.election.enable and
> translate
> > it
> > > > to
> > > > > > > >> > > > >> > > >>> unclean.recovery.strategy.
> > > > > > > >> > > > >> > > >>>
> > > > > > > >> > > > >> > > >>> } else {
> > > > > > > >> > > > >> > > >>>
> > > > > > > >> > > > >> > > >>> Use unclean.leader.election.enable
> > > > > > > >> > > > >> > > >>>
> > > > > > > >> > > > >> > > >>> }
> > > > > > > >> > > > >> > > >>>
> > > > > > > >> > > > >> > > >>>
> > > > > > > >> > > > >> > > >>> —--------
> > > > > > > >> > > > >> > > >>>
> > > > > > > >> > > > >> > > >>> >The configuration key should be
> > > > > > > >> > > > >> "unclean.recovery.manager.enabled",
> > > > > > > >> > > > >> > > >>> right?
> > > > > > > >> > > > >> > > >>>
> > > > > > > >> > > > >> > > >>>
> > > > > > > >> > > > >> > > >>> I think we have two ways of choosing a
> leader
> > > > > > > uncleanly,
> > > > > > > >> > > unclean
> > > > > > > >> > > > >> > leader
> > > > > > > >> > > > >> > > >>> election and unclean recovery(log
> inspection)
> > > and
> > > > > we
> > > > > > > try
> > > > > > > >> to
> > > > > > > >> > > > switch
> > > > > > > >> > > > >> > > between
> > > > > > > >> > > > >> > > >>> them.
> > > > > > > >> > > > >> > > >>>
> > > > > > > >> > > > >> > > >>> Do you mean we want to develop two ways of
> > > > > performing
> > > > > > > the
> > > > > > > >> > > > unclean
> > > > > > > >> > > > >> > > >>> recovery and one of them is using “unclean
> > > > recovery
> > > > > > > >> > manager”?
> > > > > > > >> > > I
> > > > > > > >> > > > >> guess
> > > > > > > >> > > > >> > > we
> > > > > > > >> > > > >> > > >>> haven’t discussed the second way.
> > > > > > > >> > > > >> > > >>>
> > > > > > > >> > > > >> > > >>>
> > > > > > > >> > > > >> > > >>> —-------
> > > > > > > >> > > > >> > > >>>
> > > > > > > >> > > > >> > > >>> >How do these 4 levels of overrides
> interact
> > > with
> > > > > > your
> > > > > > > >> new
> > > > > > > >> > > > >> > > >>> configurations?
> > > > > > > >> > > > >> > > >>>
> > > > > > > >> > > > >> > > >>>
> > > > > > > >> > > > >> > > >>> I do notice in the Kraft controller code,
> the
> > > > > method
> > > > > > to
> > > > > > > >> > check
> > > > > > > >> > > > >> whether
> > > > > > > >> > > > >> > > >>> perform unclean leader election is hard
> coded
> > > to
> > > > > > false
> > > > > > > >> since
> > > > > > > >> > > > >> > > >>> 2021(uncleanLeaderElectionEnabledForTopic).
> > > Isn’t
> > > > > it
> > > > > > a
> > > > > > > >> good
> > > > > > > >> > > > chance
> > > > > > > >> > > > >> to
> > > > > > > >> > > > >> > > >>> completely deprecate the
> > > > > > > unclean.leader.election.enable?
> > > > > > > >> We
> > > > > > > >> > > > don’t
> > > > > > > >> > > > >> > even
> > > > > > > >> > > > >> > > have
> > > > > > > >> > > > >> > > >>> to worry about the config conversion.
> > > > > > > >> > > > >> > > >>>
> > > > > > > >> > > > >> > > >>> On the other hand, whatever the override
> is,
> > as
> > > > > long
> > > > > > as
> > > > > > > >> the
> > > > > > > >> > > > >> > controller
> > > > > > > >> > > > >> > > >>> can have the final effective
> > > > > > > >> unclean.leader.election.enable,
> > > > > > > >> > > the
> > > > > > > >> > > > >> > topic
> > > > > > > >> > > > >> > > >>> level config unclean.recovery.strategy, the
> > > > cluster
> > > > > > > level
> > > > > > > >> > > config
> > > > > > > >> > > > >> > > >>> unclean.recovery.Enabled, the controller
> can
> > > > > > calculate
> > > > > > > >> the
> > > > > > > >> > > > correct
> > > > > > > >> > > > >> > > methods
> > > > > > > >> > > > >> > > >>> to use right?
> > > > > > > >> > > > >> > > >>>
> > > > > > > >> > > > >> > > >>>
> > > > > > > >> > > > >> > > >>> On Fri, Sep 15, 2023 at 10:02 AM Colin
> > McCabe <
> > > > > > > >> > > > cmccabe@apache.org>
> > > > > > > >> > > > >> > > wrote:
> > > > > > > >> > > > >> > > >>>
> > > > > > > >> > > > >> > > >>>> On Thu, Sep 14, 2023, at 22:23, Calvin Liu
> > > > wrote:
> > > > > > > >> > > > >> > > >>>> > Hi Colin
> > > > > > > >> > > > >> > > >>>> > 1. I think using the new config name is
> > more
> > > > > > clear.
> > > > > > > >> > > > >> > > >>>> >        a. The unclean leader election is
> > > > > actually
> > > > > > > >> removed
> > > > > > > >> > > if
> > > > > > > >> > > > >> > unclean
> > > > > > > >> > > > >> > > >>>> > recovery is in use.
> > > > > > > >> > > > >> > > >>>> >        b. Using multiple values in
> > > > > > > >> > > > >> unclean.leader.election.enable
> > > > > > > >> > > > >> > is
> > > > > > > >> > > > >> > > >>>> > confusing and it will be more confusing
> > > after
> > > > > > people
> > > > > > > >> > forget
> > > > > > > >> > > > >> about
> > > > > > > >> > > > >> > > this
> > > > > > > >> > > > >> > > >>>> > discussion.
> > > > > > > >> > > > >> > > >>>>
> > > > > > > >> > > > >> > > >>>> Hi Calvin,
> > > > > > > >> > > > >> > > >>>>
> > > > > > > >> > > > >> > > >>>> So, the proposal is that if someone sets
> > > > > > > >> > > > >> > > "unclean.leader.election.enable
> > > > > > > >> > > > >> > > >>>> = true" but then sets one of your new
> > > > > > configurations,
> > > > > > > >> the
> > > > > > > >> > > > value of
> > > > > > > >> > > > >> > > >>>> unclean.leader.election.enable is ignored?
> > > That
> > > > > > seems
> > > > > > > >> less
> > > > > > > >> > > > clear
> > > > > > > >> > > > >> to
> > > > > > > >> > > > >> > > me, not
> > > > > > > >> > > > >> > > >>>> more. Just in general, having multiple
> > > > > configuration
> > > > > > > >> keys
> > > > > > > >> > to
> > > > > > > >> > > > >> control
> > > > > > > >> > > > >> > > the
> > > > > > > >> > > > >> > > >>>> same thing confuses users. Basically, they
> > are
> > > > > > sitting
> > > > > > > >> at a
> > > > > > > >> > > > giant
> > > > > > > >> > > > >> > > control
> > > > > > > >> > > > >> > > >>>> panel, and some of the levers do nothing.
> > > > > > > >> > > > >> > > >>>>
> > > > > > > >> > > > >> > > >>>> > 2. Sorry I forgot to mention in the
> > response
> > > > > that
> > > > > > I
> > > > > > > >> did
> > > > > > > >> > add
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > > >>>> > unclean.recovery.Enabled flag.
> > > > > > > >> > > > >> > > >>>>
> > > > > > > >> > > > >> > > >>>> The configuration key should be
> > > > > > > >> > > > >> "unclean.recovery.manager.enabled",
> > > > > > > >> > > > >> > > >>>> right? Becuase we can do "unclean
> recovery"
> > > > > without
> > > > > > > the
> > > > > > > >> > > > manager.
> > > > > > > >> > > > >> > > Disabling
> > > > > > > >> > > > >> > > >>>> the manager just means we use a different
> > > > > mechanism
> > > > > > > for
> > > > > > > >> > > > recovery.
> > > > > > > >> > > > >> > > >>>>
> > > > > > > >> > > > >> > > >>>> >        c. Maybe I underestimated the
> > > challenge
> > > > > of
> > > > > > > >> > replacing
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > > >>>> config. Any
> > > > > > > >> > > > >> > > >>>> > implementation problems ahead?
> > > > > > > >> > > > >> > > >>>>
> > > > > > > >> > > > >> > > >>>> There are four levels of overrides for
> > > > > > > >> > > > >> > unclean.leader.election.enable.
> > > > > > > >> > > > >> > > >>>>
> > > > > > > >> > > > >> > > >>>> 1. static configuration for node.
> > > > > > > >> > > > >> > > >>>>     This goes in the configuration file,
> > > > typically
> > > > > > > named
> > > > > > > >> > > > >> > > >>>> server.properties
> > > > > > > >> > > > >> > > >>>>
> > > > > > > >> > > > >> > > >>>> 2. dynamic configuration for node default
> > > > > > > >> > > > >> > > >>>>   ConfigResource(type=BROKER, name="")
> > > > > > > >> > > > >> > > >>>>
> > > > > > > >> > > > >> > > >>>> 3. dynamic configuration for node
> > > > > > > >> > > > >> > > >>>>   ConfigResource(type=BROKER,
> > name=<controller
> > > > > id>)
> > > > > > > >> > > > >> > > >>>>
> > > > > > > >> > > > >> > > >>>> 4. dynamic configuration for topic
> > > > > > > >> > > > >> > > >>>>   ConfigResource(type=TOPIC,
> > > name=<topic-name>)
> > > > > > > >> > > > >> > > >>>>
> > > > > > > >> > > > >> > > >>>> How do these 4 levels of overrides
> interact
> > > with
> > > > > > your
> > > > > > > >> new
> > > > > > > >> > > > >> > > >>>> configurations? If the new configurations
> > > > dominate
> > > > > > > over
> > > > > > > >> the
> > > > > > > >> > > old
> > > > > > > >> > > > >> > ones,
> > > > > > > >> > > > >> > > it
> > > > > > > >> > > > >> > > >>>> seems like this will get a lot more
> > confusing
> > > to
> > > > > > > >> implement
> > > > > > > >> > > (and
> > > > > > > >> > > > >> also
> > > > > > > >> > > > >> > > to
> > > > > > > >> > > > >> > > >>>> use.)
> > > > > > > >> > > > >> > > >>>>
> > > > > > > >> > > > >> > > >>>> Again, I'd recommend just adding some new
> > > values
> > > > > to
> > > > > > > >> > > > >> > > >>>> unclean.leader.election.enable. It's
> simple
> > > and
> > > > > will
> > > > > > > >> > prevent
> > > > > > > >> > > > user
> > > > > > > >> > > > >> > > confusion
> > > > > > > >> > > > >> > > >>>> (as well as developer confusion.)
> > > > > > > >> > > > >> > > >>>>
> > > > > > > >> > > > >> > > >>>> best,
> > > > > > > >> > > > >> > > >>>> Colin
> > > > > > > >> > > > >> > > >>>>
> > > > > > > >> > > > >> > > >>>>
> > > > > > > >> > > > >> > > >>>> > 3. About the admin client, I mentioned 3
> > > > changes
> > > > > > in
> > > > > > > >> the
> > > > > > > >> > > > client.
> > > > > > > >> > > > >> > > >>>> Anything
> > > > > > > >> > > > >> > > >>>> > else I missed in the KIP?
> > > > > > > >> > > > >> > > >>>> >       a. The client will switch to using
> > the
> > > > new
> > > > > > RPC
> > > > > > > >> > > instead
> > > > > > > >> > > > of
> > > > > > > >> > > > >> > > >>>> > MetadataRequest for the topics.
> > > > > > > >> > > > >> > > >>>> >       b. The TopicPartitionInfo used in
> > > > > > > >> TopicDescription
> > > > > > > >> > > > needs
> > > > > > > >> > > > >> to
> > > > > > > >> > > > >> > > add
> > > > > > > >> > > > >> > > >>>> new
> > > > > > > >> > > > >> > > >>>> > fields related to the ELR.
> > > > > > > >> > > > >> > > >>>> >       c. The outputs will add the ELR
> > > related
> > > > > > > fields.
> > > > > > > >> > > > >> > > >>>> >
> > > > > > > >> > > > >> > > >>>> > On Thu, Sep 14, 2023 at 9:19 PM Colin
> > > McCabe <
> > > > > > > >> > > > >> cmccabe@apache.org>
> > > > > > > >> > > > >> > > >>>> wrote:
> > > > > > > >> > > > >> > > >>>> >
> > > > > > > >> > > > >> > > >>>> >> Hi Calvin,
> > > > > > > >> > > > >> > > >>>> >>
> > > > > > > >> > > > >> > > >>>> >> Thanks for the changes.
> > > > > > > >> > > > >> > > >>>> >>
> > > > > > > >> > > > >> > > >>>> >> 1. Earlier I commented that creating
> > > > > > > >> > > > >> "unclean.recovery.strategy "
> > > > > > > >> > > > >> > > is
> > > > > > > >> > > > >> > > >>>> not
> > > > > > > >> > > > >> > > >>>> >> necessary, and we can just reuse the
> > > existing
> > > > > > > >> > > > >> > > >>>> >> "unclean.leader.election.enable"
> > > > configuration
> > > > > > key.
> > > > > > > >> > Let's
> > > > > > > >> > > > >> discuss
> > > > > > > >> > > > >> > > >>>> that.
> > > > > > > >> > > > >> > > >>>> >>
> > > > > > > >> > > > >> > > >>>> >> 2.I also don't understand why you
> didn't
> > > add
> > > > a
> > > > > > > >> > > > configuration to
> > > > > > > >> > > > >> > > >>>> enable or
> > > > > > > >> > > > >> > > >>>> >> disable the Unclean Recovery Manager.
> > This
> > > > > seems
> > > > > > > >> like a
> > > > > > > >> > > very
> > > > > > > >> > > > >> > simple
> > > > > > > >> > > > >> > > >>>> way to
> > > > > > > >> > > > >> > > >>>> >> handle the staging issue which we
> > > discussed.
> > > > > The
> > > > > > > URM
> > > > > > > >> can
> > > > > > > >> > > > just
> > > > > > > >> > > > >> be
> > > > > > > >> > > > >> > > >>>> turned off
> > > > > > > >> > > > >> > > >>>> >> until it is production ready. Let's
> > discuss
> > > > > this.
> > > > > > > >> > > > >> > > >>>> >>
> > > > > > > >> > > > >> > > >>>> >> 3. You still need to describe the
> changes
> > > to
> > > > > > > >> AdminClient
> > > > > > > >> > > > that
> > > > > > > >> > > > >> are
> > > > > > > >> > > > >> > > >>>> needed
> > > > > > > >> > > > >> > > >>>> >> to use DescribeTopicRequest.
> > > > > > > >> > > > >> > > >>>> >>
> > > > > > > >> > > > >> > > >>>> >> Keep at it. It's looking better. :)
> > > > > > > >> > > > >> > > >>>> >>
> > > > > > > >> > > > >> > > >>>> >> best,
> > > > > > > >> > > > >> > > >>>> >> Colin
> > > > > > > >> > > > >> > > >>>> >>
> > > > > > > >> > > > >> > > >>>> >>
> > > > > > > >> > > > >> > > >>>> >> On Thu, Sep 14, 2023, at 11:03, Calvin
> > Liu
> > > > > wrote:
> > > > > > > >> > > > >> > > >>>> >> > Hi Colin
> > > > > > > >> > > > >> > > >>>> >> > Thanks for the comments!
> > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > >> > > > >> > > >>>> >> > I did the following changes
> > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > >> > > > >> > > >>>> >> >    1.
> > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > >> > > > >> > > >>>> >> >    Simplified the API spec section to
> > > only
> > > > > > > include
> > > > > > > >> the
> > > > > > > >> > > > diff.
> > > > > > > >> > > > >> > > >>>> >> >    2.
> > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > >> > > > >> > > >>>> >> >    Reordered the HWM requirement
> > section.
> > > > > > > >> > > > >> > > >>>> >> >    3.
> > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > >> > > > >> > > >>>> >> >    Removed the URM implementation
> > details
> > > > to
> > > > > > keep
> > > > > > > >> the
> > > > > > > >> > > > >> necessary
> > > > > > > >> > > > >> > > >>>> >> >    characteristics to perform the
> > unclean
> > > > > > > recovery.
> > > > > > > >> > > > >> > > >>>> >> >    1.
> > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > >> > > > >> > > >>>> >> >       When to perform the unclean
> > > recovery
> > > > > > > >> > > > >> > > >>>> >> >       2.
> > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > >> > > > >> > > >>>> >> >       Under different config, how the
> > > > unclean
> > > > > > > >> recovery
> > > > > > > >> > > > finds
> > > > > > > >> > > > >> > the
> > > > > > > >> > > > >> > > >>>> leader.
> > > > > > > >> > > > >> > > >>>> >> >       3.
> > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > >> > > > >> > > >>>> >> >       How the config
> > > > > > > unclean.leader.election.enable
> > > > > > > >> > and
> > > > > > > >> > > > >> > > >>>> >> >       unclean.recovery.strategy are
> > > > converted
> > > > > > > when
> > > > > > > >> > users
> > > > > > > >> > > > >> > > >>>> enable/disable
> > > > > > > >> > > > >> > > >>>> >> the
> > > > > > > >> > > > >> > > >>>> >> >       unclean recovery.
> > > > > > > >> > > > >> > > >>>> >> >       4.
> > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > >> > > > >> > > >>>> >> >    More details about how we change
> > admin
> > > > > > client.
> > > > > > > >> > > > >> > > >>>> >> >    5.
> > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > >> > > > >> > > >>>> >> >    API limits on the
> > > > GetReplicaLogInfoRequest
> > > > > > and
> > > > > > > >> > > > >> > > >>>> DescribeTopicRequest.
> > > > > > > >> > > > >> > > >>>> >> >    6.
> > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > >> > > > >> > > >>>> >> >    Two metrics added
> > > > > > > >> > > > >> > > >>>> >> >    1.
> > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > >> > > >  Kafka.controller.global_under_min_isr_partition_count
> > > > > > > >> > > > >> > > >>>> >> >       2.
> > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > >>  kafka.controller.unclean_recovery_finished_count
> > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > >> > > > >> > > >>>> >> > On Wed, Sep 13, 2023 at 10:46 AM
> Colin
> > > > > McCabe <
> > > > > > > >> > > > >> > > cmccabe@apache.org>
> > > > > > > >> > > > >> > > >>>> >> wrote:
> > > > > > > >> > > > >> > > >>>> >> >
> > > > > > > >> > > > >> > > >>>> >> >> On Tue, Sep 12, 2023, at 17:21,
> Calvin
> > > Liu
> > > > > > > wrote:
> > > > > > > >> > > > >> > > >>>> >> >> > Hi Colin
> > > > > > > >> > > > >> > > >>>> >> >> > Thanks for the comments!
> > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> Hi Calvin,
> > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> Thanks again for the KIP.
> > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> One meta-comment: it's usually
> better
> > to
> > > > > just
> > > > > > > do a
> > > > > > > >> > diff
> > > > > > > >> > > > on a
> > > > > > > >> > > > >> > > >>>> message
> > > > > > > >> > > > >> > > >>>> >> spec
> > > > > > > >> > > > >> > > >>>> >> >> file or java file if you're
> including
> > > > > changes
> > > > > > to
> > > > > > > >> it
> > > > > > > >> > in
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > KIP.
> > > > > > > >> > > > >> > > >>>> This is
> > > > > > > >> > > > >> > > >>>> >> >> easier to read than looking for "new
> > > > fields
> > > > > > > begin"
> > > > > > > >> > etc.
> > > > > > > >> > > > in
> > > > > > > >> > > > >> the
> > > > > > > >> > > > >> > > >>>> text, and
> > > > > > > >> > > > >> > > >>>> >> >> gracefully handles the case where
> > > existing
> > > > > > > fields
> > > > > > > >> > were
> > > > > > > >> > > > >> > changed.
> > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> > Rewrite the Additional High
> > Watermark
> > > > > > > >> advancement
> > > > > > > >> > > > >> > requirement
> > > > > > > >> > > > >> > > >>>> >> >> > There was feedback on this section
> > > that
> > > > > some
> > > > > > > >> > readers
> > > > > > > >> > > > may
> > > > > > > >> > > > >> not
> > > > > > > >> > > > >> > > be
> > > > > > > >> > > > >> > > >>>> >> familiar
> > > > > > > >> > > > >> > > >>>> >> >> > with HWM and Ack=0,1,all requests.
> > > This
> > > > > can
> > > > > > > help
> > > > > > > >> > them
> > > > > > > >> > > > >> > > understand
> > > > > > > >> > > > >> > > >>>> the
> > > > > > > >> > > > >> > > >>>> >> >> > proposal. I will rewrite this part
> > for
> > > > > more
> > > > > > > >> > > > readability.
> > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> To be clear, I wasn't suggesting
> > > dropping
> > > > > > either
> > > > > > > >> > > > section. I
> > > > > > > >> > > > >> > > agree
> > > > > > > >> > > > >> > > >>>> that
> > > > > > > >> > > > >> > > >>>> >> >> they add useful background. I was
> just
> > > > > > > suggesting
> > > > > > > >> > that
> > > > > > > >> > > we
> > > > > > > >> > > > >> > should
> > > > > > > >> > > > >> > > >>>> discuss
> > > > > > > >> > > > >> > > >>>> >> >> the "acks" setting AFTER discussing
> > the
> > > > new
> > > > > > high
> > > > > > > >> > > > watermark
> > > > > > > >> > > > >> > > >>>> advancement
> > > > > > > >> > > > >> > > >>>> >> >> conditions. We also should discuss
> > > acks=0.
> > > > > > While
> > > > > > > >> it
> > > > > > > >> > > isn't
> > > > > > > >> > > > >> > > >>>> conceptually
> > > > > > > >> > > > >> > > >>>> >> much
> > > > > > > >> > > > >> > > >>>> >> >> different than acks=1 here, its
> > omission
> > > > > from
> > > > > > > this
> > > > > > > >> > > > section
> > > > > > > >> > > > >> is
> > > > > > > >> > > > >> > > >>>> confusing.
> > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> > Unclean recovery
> > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > >> > > > >> > > >>>> >> >> > The plan is to replace the
> > > > > > > >> > > > unclean.leader.election.enable
> > > > > > > >> > > > >> > with
> > > > > > > >> > > > >> > > >>>> >> >> > unclean.recovery.strategy. If the
> > > > Unclean
> > > > > > > >> Recovery
> > > > > > > >> > is
> > > > > > > >> > > > >> > enabled
> > > > > > > >> > > > >> > > >>>> then it
> > > > > > > >> > > > >> > > >>>> >> >> deals
> > > > > > > >> > > > >> > > >>>> >> >> > with the three options in the
> > > > > > > >> > > > unclean.recovery.strategy.
> > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > >> > > > >> > > >>>> >> >> > Let’s refine the Unclean Recovery.
> > We
> > > > have
> > > > > > > >> already
> > > > > > > >> > > > taken a
> > > > > > > >> > > > >> > > lot of
> > > > > > > >> > > > >> > > >>>> >> >> > suggestions and I hope to enhance
> > the
> > > > > > > >> durability of
> > > > > > > >> > > > Kafka
> > > > > > > >> > > > >> to
> > > > > > > >> > > > >> > > the
> > > > > > > >> > > > >> > > >>>> next
> > > > > > > >> > > > >> > > >>>> >> >> level
> > > > > > > >> > > > >> > > >>>> >> >> > with this KIP.
> > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> I am OK with doing the unclean
> leader
> > > > > recovery
> > > > > > > >> > > > improvements
> > > > > > > >> > > > >> in
> > > > > > > >> > > > >> > > >>>> this KIP.
> > > > > > > >> > > > >> > > >>>> >> >> However, I think we need to really
> > work
> > > on
> > > > > the
> > > > > > > >> > > > configuration
> > > > > > > >> > > > >> > > >>>> settings.
> > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> Configuration overrides are often
> > quite
> > > > > messy.
> > > > > > > For
> > > > > > > >> > > > example,
> > > > > > > >> > > > >> > the
> > > > > > > >> > > > >> > > >>>> cases
> > > > > > > >> > > > >> > > >>>> >> >> where we have log.roll.hours and
> > > > > > > >> log.roll.segment.ms
> > > > > > > >> > ,
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > user
> > > > > > > >> > > > >> > > >>>> has to
> > > > > > > >> > > > >> > > >>>> >> >> remember which one takes precedence,
> > and
> > > > it
> > > > > is
> > > > > > > not
> > > > > > > >> > > > obvious.
> > > > > > > >> > > > >> > So,
> > > > > > > >> > > > >> > > >>>> rather
> > > > > > > >> > > > >> > > >>>> >> than
> > > > > > > >> > > > >> > > >>>> >> >> creating a new configuration, why
> not
> > > add
> > > > > > > >> additional
> > > > > > > >> > > > values
> > > > > > > >> > > > >> to
> > > > > > > >> > > > >> > > >>>> >> >> "unclean.leader.election.enable"? I
> > > think
> > > > > this
> > > > > > > >> will
> > > > > > > >> > be
> > > > > > > >> > > > >> simpler
> > > > > > > >> > > > >> > > for
> > > > > > > >> > > > >> > > >>>> >> people
> > > > > > > >> > > > >> > > >>>> >> >> to understand, and simpler in the
> code
> > > as
> > > > > > well.
> > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> What if we continued to use
> > > > > > > >> > > > "unclean.leader.election.enable"
> > > > > > > >> > > > >> > but
> > > > > > > >> > > > >> > > >>>> >> extended
> > > > > > > >> > > > >> > > >>>> >> >> it so that it took a string? Then
> the
> > > > string
> > > > > > > could
> > > > > > > >> > have
> > > > > > > >> > > > >> these
> > > > > > > >> > > > >> > > >>>> values:
> > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> never
> > > > > > > >> > > > >> > > >>>> >> >>     never automatically do an
> unclean
> > > > leader
> > > > > > > >> election
> > > > > > > >> > > > under
> > > > > > > >> > > > >> > any
> > > > > > > >> > > > >> > > >>>> >> conditions
> > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> false / default
> > > > > > > >> > > > >> > > >>>> >> >>     only do an unclean leader
> election
> > > if
> > > > > > there
> > > > > > > >> may
> > > > > > > >> > be
> > > > > > > >> > > > >> > possible
> > > > > > > >> > > > >> > > >>>> data
> > > > > > > >> > > > >> > > >>>> >> loss
> > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> true / always
> > > > > > > >> > > > >> > > >>>> >> >>     always do an unclean leader
> > election
> > > > if
> > > > > we
> > > > > > > >> can't
> > > > > > > >> > > > >> > immediately
> > > > > > > >> > > > >> > > >>>> elect a
> > > > > > > >> > > > >> > > >>>> >> >> leader
> > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> It's a bit awkward that false maps
> to
> > > > > default
> > > > > > > >> rather
> > > > > > > >> > > > than to
> > > > > > > >> > > > >> > > >>>> never. But
> > > > > > > >> > > > >> > > >>>> >> >> this awkwardness exists if we use
> two
> > > > > > different
> > > > > > > >> > > > >> configuration
> > > > > > > >> > > > >> > > keys
> > > > > > > >> > > > >> > > >>>> as
> > > > > > > >> > > > >> > > >>>> >> well.
> > > > > > > >> > > > >> > > >>>> >> >> The reason for the awkwardness is
> that
> > > we
> > > > > > simply
> > > > > > > >> > don't
> > > > > > > >> > > > want
> > > > > > > >> > > > >> > most
> > > > > > > >> > > > >> > > >>>> of the
> > > > > > > >> > > > >> > > >>>> >> >> people currently setting
> > > > > > > >> > > > >> unclean.leader.election.enable=false
> > > > > > > >> > > > >> > to
> > > > > > > >> > > > >> > > >>>> get the
> > > > > > > >> > > > >> > > >>>> >> >> "never" behavior. We have to bite
> that
> > > > > bullet.
> > > > > > > >> Better
> > > > > > > >> > > to
> > > > > > > >> > > > be
> > > > > > > >> > > > >> > > clear
> > > > > > > >> > > > >> > > >>>> and
> > > > > > > >> > > > >> > > >>>> >> >> explicit than hide it.
> > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> Another thing that's a bit awkward
> is
> > > > having
> > > > > > two
> > > > > > > >> > > > different
> > > > > > > >> > > > >> > ways
> > > > > > > >> > > > >> > > to
> > > > > > > >> > > > >> > > >>>> do
> > > > > > > >> > > > >> > > >>>> >> >> unclean leader election specified in
> > the
> > > > > KIP.
> > > > > > > You
> > > > > > > >> > > > descirbe
> > > > > > > >> > > > >> two
> > > > > > > >> > > > >> > > >>>> methods:
> > > > > > > >> > > > >> > > >>>> >> the
> > > > > > > >> > > > >> > > >>>> >> >> simple "choose the last leader"
> > method,
> > > > and
> > > > > > the
> > > > > > > >> > > "unclean
> > > > > > > >> > > > >> > > recovery
> > > > > > > >> > > > >> > > >>>> >> manager"
> > > > > > > >> > > > >> > > >>>> >> >> method. I understand why you did it
> > this
> > > > way
> > > > > > --
> > > > > > > >> > "choose
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > last
> > > > > > > >> > > > >> > > >>>> >> leader" is
> > > > > > > >> > > > >> > > >>>> >> >> simple, and will help us deliver an
> > > > > > > implementation
> > > > > > > >> > > > quickly,
> > > > > > > >> > > > >> > > while
> > > > > > > >> > > > >> > > >>>> the
> > > > > > > >> > > > >> > > >>>> >> URM
> > > > > > > >> > > > >> > > >>>> >> >> is preferable in the long term. My
> > > > > suggestion
> > > > > > > >> here is
> > > > > > > >> > > to
> > > > > > > >> > > > >> > > separate
> > > > > > > >> > > > >> > > >>>> the
> > > > > > > >> > > > >> > > >>>> >> >> decision of HOW to do unclean leader
> > > > > election
> > > > > > > from
> > > > > > > >> > the
> > > > > > > >> > > > >> > decision
> > > > > > > >> > > > >> > > of
> > > > > > > >> > > > >> > > >>>> WHEN
> > > > > > > >> > > > >> > > >>>> >> to
> > > > > > > >> > > > >> > > >>>> >> >> do it.
> > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> So in other words, have
> > > > > > > >> > > "unclean.leader.election.enable"
> > > > > > > >> > > > >> > specify
> > > > > > > >> > > > >> > > >>>> when we
> > > > > > > >> > > > >> > > >>>> >> >> do unclean leader election, and
> have a
> > > new
> > > > > > > >> > > configuration
> > > > > > > >> > > > >> like
> > > > > > > >> > > > >> > > >>>> >> >> "unclean.recovery.manager.enable" to
> > > > > determine
> > > > > > > if
> > > > > > > >> we
> > > > > > > >> > > use
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > > URM.
> > > > > > > >> > > > >> > > >>>> >> >> Presumably the URM will take some
> time
> > > to
> > > > > get
> > > > > > > >> fully
> > > > > > > >> > > > stable,
> > > > > > > >> > > > >> so
> > > > > > > >> > > > >> > > >>>> this can
> > > > > > > >> > > > >> > > >>>> >> >> default to false for a while, and we
> > can
> > > > > flip
> > > > > > > the
> > > > > > > >> > > > default to
> > > > > > > >> > > > >> > > true
> > > > > > > >> > > > >> > > >>>> when
> > > > > > > >> > > > >> > > >>>> >> we
> > > > > > > >> > > > >> > > >>>> >> >> feel ready.
> > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> The URM is somewhat under-described
> > > here.
> > > > I
> > > > > > > think
> > > > > > > >> we
> > > > > > > >> > > > need a
> > > > > > > >> > > > >> > few
> > > > > > > >> > > > >> > > >>>> >> >> configurations here for it. For
> > example,
> > > > we
> > > > > > > need a
> > > > > > > >> > > > >> > > configuration to
> > > > > > > >> > > > >> > > >>>> >> specify
> > > > > > > >> > > > >> > > >>>> >> >> how long it should wait for a broker
> > to
> > > > > > respond
> > > > > > > to
> > > > > > > >> > its
> > > > > > > >> > > > RPCs
> > > > > > > >> > > > >> > > before
> > > > > > > >> > > > >> > > >>>> >> moving
> > > > > > > >> > > > >> > > >>>> >> >> on. We also need to understand how
> the
> > > URM
> > > > > > > >> interacts
> > > > > > > >> > > with
> > > > > > > >> > > > >> > > >>>> >> >>
> > unclean.leader.election.enable=always. I
> > > > > > assume
> > > > > > > >> that
> > > > > > > >> > > with
> > > > > > > >> > > > >> > > "always"
> > > > > > > >> > > > >> > > >>>> we
> > > > > > > >> > > > >> > > >>>> >> will
> > > > > > > >> > > > >> > > >>>> >> >> just unconditionally use the URM
> > rather
> > > > than
> > > > > > > >> choosing
> > > > > > > >> > > > >> > randomly.
> > > > > > > >> > > > >> > > >>>> But this
> > > > > > > >> > > > >> > > >>>> >> >> should be spelled out in the KIP.
> > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > >> > > > >> > > >>>> >> >> > DescribeTopicRequest
> > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > >> > > > >> > > >>>> >> >> >    1.
> > > > > > > >> > > > >> > > >>>> >> >> >    Yes, the plan is to replace the
> > > > > > > >> MetadataRequest
> > > > > > > >> > > with
> > > > > > > >> > > > >> the
> > > > > > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest for the
> > admin
> > > > > > clients.
> > > > > > > >> Will
> > > > > > > >> > > > check
> > > > > > > >> > > > >> > the
> > > > > > > >> > > > >> > > >>>> details.
> > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> Sounds good. But as I said, you need
> > to
> > > > > > specify
> > > > > > > >> how
> > > > > > > >> > > > >> > AdminClient
> > > > > > > >> > > > >> > > >>>> >> interacts
> > > > > > > >> > > > >> > > >>>> >> >> with the new request. This will
> > involve
> > > > > adding
> > > > > > > >> some
> > > > > > > >> > > > fields
> > > > > > > >> > > > >> to
> > > > > > > >> > > > >> > > >>>> >> >> TopicDescription.java. And you need
> to
> > > > > specify
> > > > > > > the
> > > > > > > >> > > > changes
> > > > > > > >> > > > >> to
> > > > > > > >> > > > >> > > the
> > > > > > > >> > > > >> > > >>>> >> >> kafka-topics.sh command line tool.
> > > > Otherwise
> > > > > > we
> > > > > > > >> > cannot
> > > > > > > >> > > > use
> > > > > > > >> > > > >> the
> > > > > > > >> > > > >> > > >>>> tool to
> > > > > > > >> > > > >> > > >>>> >> see
> > > > > > > >> > > > >> > > >>>> >> >> the new information.
> > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> The new requests,
> DescribeTopicRequest
> > > and
> > > > > > > >> > > > >> > > >>>> GetReplicaLogInfoRequest,
> > > > > > > >> > > > >> > > >>>> >> need
> > > > > > > >> > > > >> > > >>>> >> >> to have limits placed on them so
> that
> > > > their
> > > > > > size
> > > > > > > >> > can't
> > > > > > > >> > > be
> > > > > > > >> > > > >> > > >>>> infinite. We
> > > > > > > >> > > > >> > > >>>> >> >> don't want to propagate the current
> > > > problems
> > > > > > of
> > > > > > > >> > > > >> > MetadataRequest,
> > > > > > > >> > > > >> > > >>>> where
> > > > > > > >> > > > >> > > >>>> >> >> clients can request massive
> responses
> > > that
> > > > > can
> > > > > > > >> mess
> > > > > > > >> > up
> > > > > > > >> > > > the
> > > > > > > >> > > > >> JVM
> > > > > > > >> > > > >> > > when
> > > > > > > >> > > > >> > > >>>> >> handled.
> > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> Adding limits is simple for
> > > > > > > >> GetReplicaLogInfoRequest
> > > > > > > >> > --
> > > > > > > >> > > > we
> > > > > > > >> > > > >> can
> > > > > > > >> > > > >> > > >>>> just say
> > > > > > > >> > > > >> > > >>>> >> >> that only 2000 partitions at a time
> > can
> > > be
> > > > > > > >> requested.
> > > > > > > >> > > For
> > > > > > > >> > > > >> > > >>>> >> >> DescribeTopicRequest we can probably
> > > just
> > > > > > limit
> > > > > > > >> to 20
> > > > > > > >> > > > topics
> > > > > > > >> > > > >> > or
> > > > > > > >> > > > >> > > >>>> >> something
> > > > > > > >> > > > >> > > >>>> >> >> like that, to avoid the complexity
> of
> > > > doing
> > > > > > > >> > pagination
> > > > > > > >> > > in
> > > > > > > >> > > > >> this
> > > > > > > >> > > > >> > > KIP.
> > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> >    2.
> > > > > > > >> > > > >> > > >>>> >> >> >    I can let the broker load the
> ELR
> > > > info
> > > > > so
> > > > > > > >> that
> > > > > > > >> > > they
> > > > > > > >> > > > can
> > > > > > > >> > > > >> > > serve
> > > > > > > >> > > > >> > > >>>> the
> > > > > > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest as well.
> > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> Yes, it's fine to add to
> > MetadataCache.
> > > In
> > > > > > fact,
> > > > > > > >> > you'll
> > > > > > > >> > > > be
> > > > > > > >> > > > >> > > loading
> > > > > > > >> > > > >> > > >>>> it
> > > > > > > >> > > > >> > > >>>> >> >> anyway once it's added to
> > > PartitionImage.
> > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> >    3.
> > > > > > > >> > > > >> > > >>>> >> >> >    Yeah, it does not make sense to
> > > have
> > > > > the
> > > > > > > >> topic
> > > > > > > >> > id
> > > > > > > >> > > if
> > > > > > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest is only
> used
> > > by
> > > > > the
> > > > > > > >> admin
> > > > > > > >> > > > client.
> > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> OK. That makes things simpler. We
> can
> > > > always
> > > > > > > >> create a
> > > > > > > >> > > new
> > > > > > > >> > > > >> API
> > > > > > > >> > > > >> > > later
> > > > > > > >> > > > >> > > >>>> >> >> (hopefully not in this KIP!) to
> query
> > by
> > > > > topic
> > > > > > > ID.
> > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > >> > > > >> > > >>>> >> >> > Metrics
> > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > >> > > > >> > > >>>> >> >> > As for overall cluster health
> > > metrics, I
> > > > > > think
> > > > > > > >> > > > >> under-min-ISR
> > > > > > > >> > > > >> > > is
> > > > > > > >> > > > >> > > >>>> still
> > > > > > > >> > > > >> > > >>>> >> a
> > > > > > > >> > > > >> > > >>>> >> >> > useful one. ELR is more like a
> > safety
> > > > > belt.
> > > > > > > When
> > > > > > > >> > the
> > > > > > > >> > > > ELR
> > > > > > > >> > > > >> is
> > > > > > > >> > > > >> > > >>>> used, the
> > > > > > > >> > > > >> > > >>>> >> >> > cluster availability has already
> > been
> > > > > > > impacted.
> > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > >> > > > >> > > >>>> >> >> > Maybe we can have a metric to
> count
> > > the
> > > > > > > >> partitions
> > > > > > > >> > > that
> > > > > > > >> > > > >> > > sum(ISR,
> > > > > > > >> > > > >> > > >>>> ELR)
> > > > > > > >> > > > >> > > >>>> >> <
> > > > > > > >> > > > >> > > >>>> >> >> min
> > > > > > > >> > > > >> > > >>>> >> >> > ISR. What do you think?
> > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> How about:
> > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> A.  a metric for the totoal number
> of
> > > > > > > >> under-min-isr
> > > > > > > >> > > > >> > partitions?
> > > > > > > >> > > > >> > > We
> > > > > > > >> > > > >> > > >>>> don't
> > > > > > > >> > > > >> > > >>>> >> >> have that in Apache Kafka at the
> > moment.
> > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> B. a metric for the number of
> unclean
> > > > leader
> > > > > > > >> > elections
> > > > > > > >> > > we
> > > > > > > >> > > > >> did
> > > > > > > >> > > > >> > > (for
> > > > > > > >> > > > >> > > >>>> >> >> simplicity, it can reset to 0 on
> > > > controller
> > > > > > > >> restart:
> > > > > > > >> > we
> > > > > > > >> > > > >> expect
> > > > > > > >> > > > >> > > >>>> people to
> > > > > > > >> > > > >> > > >>>> >> >> monitor the change over time anyway)
> > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> best,
> > > > > > > >> > > > >> > > >>>> >> >> Colin
> > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > >> > > > >> > > >>>> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > >> > > > >> > > >>>> >> >> > Yeah, for the ongoing unclean
> > > > recoveries,
> > > > > > the
> > > > > > > >> > > > controller
> > > > > > > >> > > > >> can
> > > > > > > >> > > > >> > > >>>> keep an
> > > > > > > >> > > > >> > > >>>> >> >> > accurate count through failover
> > > because
> > > > > > > >> partition
> > > > > > > >> > > > >> > registration
> > > > > > > >> > > > >> > > >>>> can
> > > > > > > >> > > > >> > > >>>> >> >> indicate
> > > > > > > >> > > > >> > > >>>> >> >> > whether a recovery is needed.
> > However,
> > > > for
> > > > > > the
> > > > > > > >> > > happened
> > > > > > > >> > > > >> > ones,
> > > > > > > >> > > > >> > > >>>> unless
> > > > > > > >> > > > >> > > >>>> >> we
> > > > > > > >> > > > >> > > >>>> >> >> > want to persist the number
> > somewhere,
> > > we
> > > > > can
> > > > > > > >> only
> > > > > > > >> > > > figure
> > > > > > > >> > > > >> it
> > > > > > > >> > > > >> > > out
> > > > > > > >> > > > >> > > >>>> from
> > > > > > > >> > > > >> > > >>>> >> the
> > > > > > > >> > > > >> > > >>>> >> >> > log.
> > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > >> > > > >> > > >>>> >> >> > On Tue, Sep 12, 2023 at 3:16 PM
> > Colin
> > > > > > McCabe <
> > > > > > > >> > > > >> > > cmccabe@apache.org
> > > > > > > >> > > > >> > > >>>> >
> > > > > > > >> > > > >> > > >>>> >> wrote:
> > > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > > >> > > > >> > > >>>> >> >> >> Also, we should have metrics that
> > > show
> > > > > what
> > > > > > > is
> > > > > > > >> > going
> > > > > > > >> > > > on
> > > > > > > >> > > > >> > with
> > > > > > > >> > > > >> > > >>>> regard
> > > > > > > >> > > > >> > > >>>> >> to
> > > > > > > >> > > > >> > > >>>> >> >> the
> > > > > > > >> > > > >> > > >>>> >> >> >> eligible replica set. I'm not
> sure
> > > > > exactly
> > > > > > > >> what to
> > > > > > > >> > > > >> suggest,
> > > > > > > >> > > > >> > > but
> > > > > > > >> > > > >> > > >>>> >> >> something
> > > > > > > >> > > > >> > > >>>> >> >> >> that could identify when things
> are
> > > > going
> > > > > > > >> wrong in
> > > > > > > >> > > the
> > > > > > > >> > > > >> > > clsuter.
> > > > > > > >> > > > >> > > >>>> >> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> >> For example, maybe a metric for
> > > > > partitions
> > > > > > > >> > > containing
> > > > > > > >> > > > >> > > replicas
> > > > > > > >> > > > >> > > >>>> that
> > > > > > > >> > > > >> > > >>>> >> are
> > > > > > > >> > > > >> > > >>>> >> >> >> ineligible to be leader? That
> would
> > > > show
> > > > > a
> > > > > > > >> spike
> > > > > > > >> > > when
> > > > > > > >> > > > a
> > > > > > > >> > > > >> > > broker
> > > > > > > >> > > > >> > > >>>> had an
> > > > > > > >> > > > >> > > >>>> >> >> >> unclean restart.
> > > > > > > >> > > > >> > > >>>> >> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> >> Ideally, we'd also have a metric
> > that
> > > > > > > indicates
> > > > > > > >> > when
> > > > > > > >> > > > an
> > > > > > > >> > > > >> > > unclear
> > > > > > > >> > > > >> > > >>>> >> leader
> > > > > > > >> > > > >> > > >>>> >> >> >> election or a recovery happened.
> > > It's a
> > > > > bit
> > > > > > > >> tricky
> > > > > > > >> > > > >> because
> > > > > > > >> > > > >> > > the
> > > > > > > >> > > > >> > > >>>> simple
> > > > > > > >> > > > >> > > >>>> >> >> >> thing, of tracking it per
> > controller,
> > > > may
> > > > > > be
> > > > > > > a
> > > > > > > >> bit
> > > > > > > >> > > > >> > confusing
> > > > > > > >> > > > >> > > >>>> during
> > > > > > > >> > > > >> > > >>>> >> >> >> failovers.
> > > > > > > >> > > > >> > > >>>> >> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> >> best,
> > > > > > > >> > > > >> > > >>>> >> >> >> Colin
> > > > > > > >> > > > >> > > >>>> >> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> >>
> > > > > > > >> > > > >> > > >>>> >> >> >> On Tue, Sep 12, 2023, at 14:25,
> > Colin
> > > > > > McCabe
> > > > > > > >> > wrote:
> > > > > > > >> > > > >> > > >>>> >> >> >> > Hi Calvin,
> > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > > >> > > > >> > > >>>> >> >> >> > Thanks for the KIP. I think
> this
> > > is a
> > > > > > great
> > > > > > > >> > > > >> improvement.
> > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > > >> > > > >> > > >>>> >> >> >> >> Additional High Watermark
> > advance
> > > > > > > >> requirement
> > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > > >> > > > >> > > >>>> >> >> >> > Typo: change "advance" to
> > > > "advancement"
> > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > > >> > > > >> > > >>>> >> >> >> >> A bit recap of some key
> > concepts.
> > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > > >> > > > >> > > >>>> >> >> >> > Typo: change "bit" to "quick"
> > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > > >> > > > >> > > >>>> >> >> >> >> Ack=1/all produce request. It
> > > > defines
> > > > > > when
> > > > > > > >> the
> > > > > > > >> > > > Kafka
> > > > > > > >> > > > >> > > server
> > > > > > > >> > > > >> > > >>>> should
> > > > > > > >> > > > >> > > >>>> >> >> >> respond to the produce request
> > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > > >> > > > >> > > >>>> >> >> >> > I think this section would be
> > > clearer
> > > > > if
> > > > > > we
> > > > > > > >> > talked
> > > > > > > >> > > > >> about
> > > > > > > >> > > > >> > > the
> > > > > > > >> > > > >> > > >>>> new
> > > > > > > >> > > > >> > > >>>> >> high
> > > > > > > >> > > > >> > > >>>> >> >> >> > watermark advancement
> requirement
> > > > > first,
> > > > > > > and
> > > > > > > >> > THEN
> > > > > > > >> > > > >> talked
> > > > > > > >> > > > >> > > >>>> about its
> > > > > > > >> > > > >> > > >>>> >> >> >> > impact on acks=0, acks=1, and
> > > > > >  acks=all.
> > > > > > > >> > > > acks=all
> > > > > > > >> > > > >> is
> > > > > > > >> > > > >> > of
> > > > > > > >> > > > >> > > >>>> course
> > > > > > > >> > > > >> > > >>>> >> the
> > > > > > > >> > > > >> > > >>>> >> >> >> > main case we care about here,
> so
> > it
> > > > > would
> > > > > > > be
> > > > > > > >> > good
> > > > > > > >> > > to
> > > > > > > >> > > > >> lead
> > > > > > > >> > > > >> > > with
> > > > > > > >> > > > >> > > >>>> >> that,
> > > > > > > >> > > > >> > > >>>> >> >> >> > rather than delving into the
> > > > > > technicalities
> > > > > > > >> of
> > > > > > > >> > > > acks=0/1
> > > > > > > >> > > > >> > > first.
> > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > > >> > > > >> > > >>>> >> >> >> >> Unclean recovery
> > > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > > >> > > > >> > > >>>> >> >> >> > So, here you are introducing a
> > new
> > > > > > > >> > configuration,
> > > > > > > >> > > > >> > > >>>> >> >> >> > unclean.recovery.strategy. The
> > > > > difficult
> > > > > > > >> thing
> > > > > > > >> > > here
> > > > > > > >> > > > is
> > > > > > > >> > > > >> > that
> > > > > > > >> > > > >> > > >>>> there
> > > > > > > >> > > > >> > > >>>> >> is a
> > > > > > > >> > > > >> > > >>>> >> >> >> > lot of overlap with
> > > > > > > >> > > unclean.leader.election.enable.
> > > > > > > >> > > > So
> > > > > > > >> > > > >> we
> > > > > > > >> > > > >> > > >>>> have 3
> > > > > > > >> > > > >> > > >>>> >> >> >> > different settings for
> > > > > > > >> > unclean.recovery.strategy,
> > > > > > > >> > > > plus
> > > > > > > >> > > > >> 2
> > > > > > > >> > > > >> > > >>>> different
> > > > > > > >> > > > >> > > >>>> >> >> >> > settings for
> > > > > > > unclean.leader.election.enable,
> > > > > > > >> > > giving
> > > > > > > >> > > > a
> > > > > > > >> > > > >> > cross
> > > > > > > >> > > > >> > > >>>> >> product of
> > > > > > > >> > > > >> > > >>>> >> >> >> > 6 different options. The
> > following
> > > > > > "unclean
> > > > > > > >> > > recovery
> > > > > > > >> > > > >> > > manager"
> > > > > > > >> > > > >> > > >>>> >> section
> > > > > > > >> > > > >> > > >>>> >> >> >> > on
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Calvin Liu <ca...@confluent.io.INVALID>.
Hi Jun
60. Yes, it is a good question. I was thinking the API could be flexible to
query the particular partitions if the range query has errors for some of
the partitions. Not sure whether it is a valid assumption, what do you
think?

61. Good point, I will update them to partition level with the same limit.

62. Sure, will do.

Thanks

On Tue, Oct 10, 2023 at 10:12 AM Jun Rao <ju...@confluent.io.invalid> wrote:

> Hi, Calvin,
>
> A few more minor comments on your latest update.
>
> 60. DescribeTopicRequest: When will the Partitions field be used? It seems
> that the FirstPartitionId field is enough for AdminClient usage.
>
> 61. Could we make the limit for DescribeTopicRequest, ElectLeadersRequest,
> GetReplicaLogInfo consistent? Currently, ElectLeadersRequest's limit is at
> topic level and GetReplicaLogInfo has a different partition level limit
> from DescribeTopicRequest.
>
> 62. Should ElectLeadersRequest.DesiredLeaders be at the same level as
> ElectLeadersRequest.TopicPartitions.Partitions? In the KIP, it looks like
> it's at the same level as ElectLeadersRequest.TopicPartitions.
>
> Thanks,
>
> Jun
>
> On Wed, Oct 4, 2023 at 3:55 PM Calvin Liu <ca...@confluent.io.invalid>
> wrote:
>
> > Hi David,
> > Thanks for the comments.
> > ----
> > I thought that a new snapshot with the downgraded MV is created in this
> > case. Isn’t it the case?
> > Yes, you are right, a metadata delta will be generated after the MV
> > downgrade. Then the user can start the software downgrade.
> > -----
> > Could you also elaborate a bit more on the reasoning behind adding the
> > limits to the admin RPCs? This is a new pattern in Kafka so it would be
> > good to clear on the motivation.
> > Thanks to Colin for bringing it up. The current MetadataRequest does not
> > have a limit on the number of topics to query in a single request.
> Massive
> > requests can mess up the JVM. We want to have some sort of throttle on
> the
> > new APIs.
> > -----
> > Could you also explain how the client is supposed to handle the
> > topics/partitions above the limit? I suppose that it will have to retry
> > those, correct?
> > Corrent. For the official admin clients, it will split the large request
> > into proper pieces and query one after another.
> > -----
> > My understanding is that the topics/partitions above the limit will be
> > failed with an invalid exception error. I wonder if this choice is
> > judicious because the invalide request exception is usually fatal. It may
> > be better to use an new and explicit error for this case.
> >
> > Thanks for bringing this up. How about "REQUEST_LIMIT_REACHED"?
> > --------
> > It seems that we still need to specify the changes to the admin api to
> > accommodate the new or updated apis. Do you plan to add them?
> > Try to cover the following
> > 1. The admin client will use the new DescribeTopicRequest to query the
> > topics
> > 2. Mention the API limit and the new retriable error.
> > 3. Output changes for the admin client when describing a topic (new
> fields
> > of ELR...)
> > 4. Changes to data structures like TopicPartitionInfo to include the ELR.
> > Anything else I missed?
> >
> > Thanks!
> >
> >
> >
> >
> >
> > On Wed, Oct 4, 2023 at 12:27 PM David Jacot <da...@gmail.com>
> wrote:
> >
> > > Hi Calvin,
> > >
> > > I thought that a new snapshot with the downgraded MV is created in this
> > > case. Isn’t it the case?
> > >
> > > Could you also elaborate a bit more on the reasoning behind adding the
> > > limits to the admin RPCs? This is a new pattern in Kafka so it would be
> > > good to clear on the motivation.
> > >
> > > Could you also explain how the client is supposed to handle the
> > > topics/partitions above the limit? I suppose that it will have to retry
> > > those, correct?
> > >
> > > My understanding is that the topics/partitions above the limit will be
> > > failed with an invalid exception error. I wonder if this choice is
> > > judicious because the invalide request exception is usually fatal. It
> may
> > > be better to use an new and explicit error for this case.
> > >
> > > It seems that we still need to specify the changes to the admin api to
> > > accommodate the new or updated apis. Do you plan to add them?
> > >
> > > Best,
> > > David
> > >
> > > Le mer. 4 oct. 2023 à 20:39, Calvin Liu <ca...@confluent.io.invalid> a
> > > écrit :
> > >
> > > > Hi Jun,
> > > > After the MV downgrade, the controller will write in the old version
> of
> > > the
> > > > PartitionRecord/PartitionChangeRecord. If I understand correctly, it
> is
> > > > possible to downgrade the software version if the controller only has
> > to
> > > > handle old version records.
> > > > However, the controller will not automatically rewrite the
> > > PartitionRecord
> > > > with the old version unless there is a partition update. Then, the
> user
> > > may
> > > > have to wait an unknown amount of time before the software downgrades
> > > > unless they do a roll to force update every partition. If it makes
> > > sense, I
> > > > can mention these steps to do a software downgrade.
> > > > Thanks
> > > >
> > > > On Wed, Oct 4, 2023 at 11:20 AM Jun Rao <ju...@confluent.io.invalid>
> > > wrote:
> > > >
> > > > > Hi, Calvin and Justine,
> > > > >
> > > > > Historically, when we change the record format in the log, we don't
> > > > support
> > > > > software version downgrading.
> > > > >
> > > > > For the record format change in the metadata log, have we thought
> > about
> > > > > forcing the write of the latest metadata records with the old
> version
> > > > > during MV downgrading? This will in theory allow the old version of
> > the
> > > > > software to obtain the latest metadata.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > > On Wed, Oct 4, 2023 at 9:53 AM Justine Olshan
> > > > <jolshan@confluent.io.invalid
> > > > > >
> > > > > wrote:
> > > > >
> > > > > > Sorry -- not MV but software version.
> > > > > >
> > > > > > On Wed, Oct 4, 2023 at 9:51 AM Justine Olshan <
> > jolshan@confluent.io>
> > > > > > wrote:
> > > > > >
> > > > > > > Catching up with this discussion.
> > > > > > >
> > > > > > > I was just curious -- have we had other instances where
> > downgrading
> > > > MV
> > > > > is
> > > > > > > not supported? I think Kafka typically tries to support
> > downgrades,
> > > > > and I
> > > > > > > couldn't think of other examples.
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Justine
> > > > > > >
> > > > > > > On Wed, Oct 4, 2023 at 9:40 AM Calvin Liu
> > > <caliu@confluent.io.invalid
> > > > >
> > > > > > > wrote:
> > > > > > >
> > > > > > >> Hi Jun,
> > > > > > >> 54. Marked the software downgrading is not supported. As the
> old
> > > > > > >> controller
> > > > > > >> will not understand the new PartitionRecord and
> > > > PartitionChangeRecord.
> > > > > > >> Thanks!
> > > > > > >>
> > > > > > >> On Wed, Oct 4, 2023 at 9:12 AM Jun Rao
> <jun@confluent.io.invalid
> > >
> > > > > > wrote:
> > > > > > >>
> > > > > > >> > Hi, Calvin,
> > > > > > >> >
> > > > > > >> > Thanks for the reply. Just one more comment.
> > > > > > >> >
> > > > > > >> > 54. It seems that downgrading MV is supported. Is
> downgrading
> > > the
> > > > > > >> software
> > > > > > >> > version supported? It would be useful to document that.
> > > > > > >> >
> > > > > > >> > Thanks,
> > > > > > >> >
> > > > > > >> > Jun
> > > > > > >> >
> > > > > > >> > On Tue, Oct 3, 2023 at 4:55 PM Artem Livshits
> > > > > > >> > <al...@confluent.io.invalid> wrote:
> > > > > > >> >
> > > > > > >> > > Hi Colin,
> > > > > > >> > >
> > > > > > >> > > I think in your example "do_unclean_recovery" would need
> to
> > do
> > > > > > >> different
> > > > > > >> > > things depending on the strategy.
> > > > > > >> > >
> > > > > > >> > > do_unclean_recovery() {
> > > > > > >> > >    if (unclean.recovery.manager.enabled) {
> > > > > > >> > >     if (strategy == Aggressive)
> > > > > > >> > >       use UncleanRecoveryManager(waitLastKnownERL=false)
> //
> > > > just
> > > > > > >> inspect
> > > > > > >> > > logs from whoever is available
> > > > > > >> > >     else
> > > > > > >> > >       use  UncleanRecoveryManager(waitLastKnownERL=true)
> //
> > > > must
> > > > > > wait
> > > > > > >> > for
> > > > > > >> > > at least last known ELR
> > > > > > >> > >   } else {
> > > > > > >> > >     if (strategy == Aggressive)
> > > > > > >> > >       choose the last known leader if that is available,
> or
> > a
> > > > > random
> > > > > > >> > leader
> > > > > > >> > > if not)
> > > > > > >> > >     else
> > > > > > >> > >       wait for last known leader to get back
> > > > > > >> > >   }
> > > > > > >> > > }
> > > > > > >> > >
> > > > > > >> > > The idea is that the Aggressive strategy would kick in as
> > soon
> > > > as
> > > > > we
> > > > > > >> lost
> > > > > > >> > > the leader and would pick a leader from whoever is
> > available;
> > > > but
> > > > > > the
> > > > > > >> > > Balanced will only kick in when ELR is empty and will wait
> > for
> > > > the
> > > > > > >> > brokers
> > > > > > >> > > that likely have most data to be available.
> > > > > > >> > >
> > > > > > >> > > On Tue, Oct 3, 2023 at 3:04 PM Colin McCabe <
> > > cmccabe@apache.org
> > > > >
> > > > > > >> wrote:
> > > > > > >> > >
> > > > > > >> > > > On Tue, Oct 3, 2023, at 10:49, Jun Rao wrote:
> > > > > > >> > > > > Hi, Calvin,
> > > > > > >> > > > >
> > > > > > >> > > > > Thanks for the update KIP. A few more comments.
> > > > > > >> > > > >
> > > > > > >> > > > > 41. Why would a user choose the option to select a
> > random
> > > > > > replica
> > > > > > >> as
> > > > > > >> > > the
> > > > > > >> > > > > leader instead of using
> > > unclean.recovery.strateg=Aggressive?
> > > > > It
> > > > > > >> seems
> > > > > > >> > > > that
> > > > > > >> > > > > the latter is strictly better? If that's not the case,
> > > could
> > > > > we
> > > > > > >> fold
> > > > > > >> > > this
> > > > > > >> > > > > option under unclean.recovery.strategy instead of
> > > > introducing
> > > > > a
> > > > > > >> > > separate
> > > > > > >> > > > > config?
> > > > > > >> > > >
> > > > > > >> > > > Hi Jun,
> > > > > > >> > > >
> > > > > > >> > > > I thought the flow of control was:
> > > > > > >> > > >
> > > > > > >> > > > If there is no leader for the partition {
> > > > > > >> > > >   If (there are unfenced ELR members) {
> > > > > > >> > > >     choose_an_unfenced_ELR_member
> > > > > > >> > > >   } else if (there are fenced ELR members AND
> > > > > > strategy=Aggressive) {
> > > > > > >> > > >     do_unclean_recovery
> > > > > > >> > > >   } else if (there are no ELR members AND strategy !=
> > None)
> > > {
> > > > > > >> > > >     do_unclean_recovery
> > > > > > >> > > >   } else {
> > > > > > >> > > >     do nothing about the missing leader
> > > > > > >> > > >   }
> > > > > > >> > > > }
> > > > > > >> > > >
> > > > > > >> > > > do_unclean_recovery() {
> > > > > > >> > > >    if (unclean.recovery.manager.enabled) {
> > > > > > >> > > >     use UncleanRecoveryManager
> > > > > > >> > > >   } else {
> > > > > > >> > > >     choose the last known leader if that is available,
> or
> > a
> > > > > random
> > > > > > >> > leader
> > > > > > >> > > > if not)
> > > > > > >> > > >   }
> > > > > > >> > > > }
> > > > > > >> > > >
> > > > > > >> > > > However, I think this could be clarified, especially the
> > > > > behavior
> > > > > > >> when
> > > > > > >> > > > unclean.recovery.manager.enabled=false. Inuitively the
> > goal
> > > > for
> > > > > > >> > > > unclean.recovery.manager.enabled=false is to be "the
> same
> > as
> > > > > now,
> > > > > > >> > mostly"
> > > > > > >> > > > but it's very underspecified in the KIP, I agree.
> > > > > > >> > > >
> > > > > > >> > > > >
> > > > > > >> > > > > 50. ElectLeadersRequest: "If more than 20 topics are
> > > > included,
> > > > > > >> only
> > > > > > >> > the
> > > > > > >> > > > > first 20 will be served. Others will be returned with
> > > > > > >> > DesiredLeaders."
> > > > > > >> > > > Hmm,
> > > > > > >> > > > > not sure that I understand this. ElectLeadersResponse
> > > > doesn't
> > > > > > >> have a
> > > > > > >> > > > > DesiredLeaders field.
> > > > > > >> > > > >
> > > > > > >> > > > > 51. GetReplicaLogInfo: "If more than 2000 partitions
> are
> > > > > > included,
> > > > > > >> > only
> > > > > > >> > > > the
> > > > > > >> > > > > first 2000 will be served" Do we return an error for
> the
> > > > > > remaining
> > > > > > >> > > > > partitions? Actually, should we include an errorCode
> > field
> > > > at
> > > > > > the
> > > > > > >> > > > partition
> > > > > > >> > > > > level in GetReplicaLogInfoResponse to cover
> non-existing
> > > > > > >> partitions
> > > > > > >> > and
> > > > > > >> > > > no
> > > > > > >> > > > > authorization, etc?
> > > > > > >> > > > >
> > > > > > >> > > > > 52. The entry should matches => The entry should match
> > > > > > >> > > > >
> > > > > > >> > > > > 53. ElectLeadersRequest.DesiredLeaders: Should it be
> > > > nullable
> > > > > > >> since a
> > > > > > >> > > > user
> > > > > > >> > > > > may not specify DesiredLeaders?
> > > > > > >> > > > >
> > > > > > >> > > > > 54. Downgrade: Is that indeed possible? I thought
> > earlier
> > > > you
> > > > > > said
> > > > > > >> > that
> > > > > > >> > > > > once the new version of the records are in the
> metadata
> > > log,
> > > > > one
> > > > > > >> > can't
> > > > > > >> > > > > downgrade since the old broker doesn't know how to
> parse
> > > the
> > > > > new
> > > > > > >> > > version
> > > > > > >> > > > of
> > > > > > >> > > > > the metadata records?
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > > > MetadataVersion downgrade is currently broken but we
> have
> > > > fixing
> > > > > > it
> > > > > > >> on
> > > > > > >> > > our
> > > > > > >> > > > plate for Kafka 3.7.
> > > > > > >> > > >
> > > > > > >> > > > The way downgrade works is that "new features" are
> > dropped,
> > > > > > leaving
> > > > > > >> > only
> > > > > > >> > > > the old ones.
> > > > > > >> > > >
> > > > > > >> > > > > 55. CleanShutdownFile: Should we add a version field
> for
> > > > > future
> > > > > > >> > > > extension?
> > > > > > >> > > > >
> > > > > > >> > > > > 56. Config changes are public facing. Could we have a
> > > > separate
> > > > > > >> > section
> > > > > > >> > > to
> > > > > > >> > > > > document all the config changes?
> > > > > > >> > > >
> > > > > > >> > > > +1. A separate section for this would be good.
> > > > > > >> > > >
> > > > > > >> > > > best,
> > > > > > >> > > > Colin
> > > > > > >> > > >
> > > > > > >> > > > >
> > > > > > >> > > > > Thanks,
> > > > > > >> > > > >
> > > > > > >> > > > > Jun
> > > > > > >> > > > >
> > > > > > >> > > > > On Mon, Sep 25, 2023 at 4:29 PM Calvin Liu
> > > > > > >> > <caliu@confluent.io.invalid
> > > > > > >> > > >
> > > > > > >> > > > > wrote:
> > > > > > >> > > > >
> > > > > > >> > > > >> Hi Jun
> > > > > > >> > > > >> Thanks for the comments.
> > > > > > >> > > > >>
> > > > > > >> > > > >> 40. If we change to None, it is not guaranteed for no
> > > data
> > > > > > loss.
> > > > > > >> For
> > > > > > >> > > > users
> > > > > > >> > > > >> who are not able to validate the data with external
> > > > > resources,
> > > > > > >> > manual
> > > > > > >> > > > >> intervention does not give a better result but a loss
> > of
> > > > > > >> > availability.
> > > > > > >> > > > So
> > > > > > >> > > > >> practically speaking, the Balance mode would be a
> > better
> > > > > > default
> > > > > > >> > > value.
> > > > > > >> > > > >>
> > > > > > >> > > > >> 41. No, it represents how we want to do the unclean
> > > leader
> > > > > > >> election.
> > > > > > >> > > If
> > > > > > >> > > > it
> > > > > > >> > > > >> is false, the unclean leader election will be the old
> > > > random
> > > > > > way.
> > > > > > >> > > > >> Otherwise, the unclean recovery will be used.
> > > > > > >> > > > >>
> > > > > > >> > > > >> 42. Good catch. Updated.
> > > > > > >> > > > >>
> > > > > > >> > > > >> 43. Only the first 20 topics will be served. Others
> > will
> > > be
> > > > > > >> returned
> > > > > > >> > > > with
> > > > > > >> > > > >> InvalidRequestError
> > > > > > >> > > > >>
> > > > > > >> > > > >> 44. The order matters. The desired leader entries
> match
> > > > with
> > > > > > the
> > > > > > >> > topic
> > > > > > >> > > > >> partition list by the index.
> > > > > > >> > > > >>
> > > > > > >> > > > >> 45. Thanks! Updated.
> > > > > > >> > > > >>
> > > > > > >> > > > >> 46. Good advice! Updated.
> > > > > > >> > > > >>
> > > > > > >> > > > >> 47.1, updated the comment. Basically it will elect
> the
> > > > > replica
> > > > > > in
> > > > > > >> > the
> > > > > > >> > > > >> desiredLeader field to be the leader
> > > > > > >> > > > >>
> > > > > > >> > > > >> 47.2 We can let the admin client do the conversion.
> > Using
> > > > the
> > > > > > >> > > > desiredLeader
> > > > > > >> > > > >> field in the json format seems easier for users.
> > > > > > >> > > > >>
> > > > > > >> > > > >> 48. Once the MV version is downgraded, all the ELR
> > > related
> > > > > > fields
> > > > > > >> > will
> > > > > > >> > > > be
> > > > > > >> > > > >> removed on the next partition change. The controller
> > will
> > > > > also
> > > > > > >> > ignore
> > > > > > >> > > > the
> > > > > > >> > > > >> ELR fields. Updated the KIP.
> > > > > > >> > > > >>
> > > > > > >> > > > >> 49. Yes, it would be deprecated/removed.
> > > > > > >> > > > >>
> > > > > > >> > > > >>
> > > > > > >> > > > >> On Mon, Sep 25, 2023 at 3:49 PM Jun Rao
> > > > > > <jun@confluent.io.invalid
> > > > > > >> >
> > > > > > >> > > > wrote:
> > > > > > >> > > > >>
> > > > > > >> > > > >> > Hi, Calvin,
> > > > > > >> > > > >> >
> > > > > > >> > > > >> > Thanks for the updated KIP. Made another pass. A
> few
> > > more
> > > > > > >> comments
> > > > > > >> > > > below.
> > > > > > >> > > > >> >
> > > > > > >> > > > >> > 40. unclean.leader.election.enable.false ->
> > > > > > >> > > > >> > unclean.recovery.strategy.Balanced: The Balanced
> mode
> > > > could
> > > > > > >> still
> > > > > > >> > > > lead to
> > > > > > >> > > > >> > data loss. So, I am wondering if
> > > > > > >> > > unclean.leader.election.enable.false
> > > > > > >> > > > >> > should map to None?
> > > > > > >> > > > >> >
> > > > > > >> > > > >> > 41. unclean.recovery.manager.enabled: I am not sure
> > why
> > > > we
> > > > > > >> > introduce
> > > > > > >> > > > this
> > > > > > >> > > > >> > additional config. Is it the same as
> > > > > > >> > unclean.recovery.strategy=None?
> > > > > > >> > > > >> >
> > > > > > >> > > > >> > 42.
> DescribeTopicResponse.TopicAuthorizedOperations:
> > > > Should
> > > > > > >> this
> > > > > > >> > be
> > > > > > >> > > at
> > > > > > >> > > > >> the
> > > > > > >> > > > >> > topic level?
> > > > > > >> > > > >> >
> > > > > > >> > > > >> > 43. "Limit: 20 topics max per request": Could we
> > > describe
> > > > > > what
> > > > > > >> > > > happens if
> > > > > > >> > > > >> > the request includes more than 20 topics?
> > > > > > >> > > > >> >
> > > > > > >> > > > >> > 44. ElectLeadersRequest.DesiredLeaders: Could we
> > > describe
> > > > > > >> whether
> > > > > > >> > > the
> > > > > > >> > > > >> > ordering matters?
> > > > > > >> > > > >> >
> > > > > > >> > > > >> > 45. GetReplicaLogInfo.TopicPartitions: "about":
> "The
> > > > topic
> > > > > > >> > > partitions
> > > > > > >> > > > to
> > > > > > >> > > > >> > elect leaders.": The description in "about" is
> > > incorrect.
> > > > > > >> > > > >> >
> > > > > > >> > > > >> > 46. GetReplicaLogInfoResponse: Should we nest
> > > partitions
> > > > > > under
> > > > > > >> > > > topicId to
> > > > > > >> > > > >> > be consistent with other types of responses?
> > > > > > >> > > > >> >
> > > > > > >> > > > >> > 47. kafka-leader-election.sh:
> > > > > > >> > > > >> > 47.1 Could we explain DESIGNATION?
> > > > > > >> > > > >> > 47.2 desiredLeader: Should it be a list to match
> the
> > > > field
> > > > > in
> > > > > > >> > > > >> > ElectLeadersRequest?
> > > > > > >> > > > >> >
> > > > > > >> > > > >> > 48. We could add a section on downgrade?
> > > > > > >> > > > >> >
> > > > > > >> > > > >> > 49. LastKnownLeader: This seems only needed in the
> > > first
> > > > > > phase
> > > > > > >> of
> > > > > > >> > > > >> > delivering ELR. Will it be removed when the
> complete
> > > KIP
> > > > is
> > > > > > >> > > delivered?
> > > > > > >> > > > >> >
> > > > > > >> > > > >> > Thanks,
> > > > > > >> > > > >> >
> > > > > > >> > > > >> > Jun
> > > > > > >> > > > >> >
> > > > > > >> > > > >> > On Tue, Sep 19, 2023 at 1:30 PM Colin McCabe <
> > > > > > >> cmccabe@apache.org>
> > > > > > >> > > > wrote:
> > > > > > >> > > > >> >
> > > > > > >> > > > >> > > Hi Calvin,
> > > > > > >> > > > >> > >
> > > > > > >> > > > >> > > Thanks for the explanations. I like the idea of
> > using
> > > > > none,
> > > > > > >> > > > balanced,
> > > > > > >> > > > >> > > aggressive. We also had an offline discussion
> about
> > > why
> > > > > it
> > > > > > is
> > > > > > >> > good
> > > > > > >> > > > to
> > > > > > >> > > > >> > use a
> > > > > > >> > > > >> > > new config key (basically, so that we can
> deprecate
> > > the
> > > > > old
> > > > > > >> one
> > > > > > >> > > > which
> > > > > > >> > > > >> had
> > > > > > >> > > > >> > > only false/true values in 4.0) With these
> changes,
> > I
> > > am
> > > > > +1.
> > > > > > >> > > > >> > >
> > > > > > >> > > > >> > > best,
> > > > > > >> > > > >> > > Colin
> > > > > > >> > > > >> > >
> > > > > > >> > > > >> > > On Mon, Sep 18, 2023, at 15:54, Calvin Liu wrote:
> > > > > > >> > > > >> > > > Hi Colin,
> > > > > > >> > > > >> > > > Also, can we deprecate
> > > unclean.leader.election.enable
> > > > > in
> > > > > > >> 4.0?
> > > > > > >> > > > Before
> > > > > > >> > > > >> > > that,
> > > > > > >> > > > >> > > > we can have both the config
> > > unclean.recovery.strategy
> > > > > and
> > > > > > >> > > > >> > > > unclean.leader.election.enable
> > > > > > >> > > > >> > > > and using the unclean.recovery.Enabled to
> > determine
> > > > > which
> > > > > > >> > config
> > > > > > >> > > > to
> > > > > > >> > > > >> use
> > > > > > >> > > > >> > > > during the unclean leader election.
> > > > > > >> > > > >> > > >
> > > > > > >> > > > >> > > > On Mon, Sep 18, 2023 at 3:51 PM Calvin Liu <
> > > > > > >> > caliu@confluent.io>
> > > > > > >> > > > >> wrote:
> > > > > > >> > > > >> > > >
> > > > > > >> > > > >> > > >> Hi Colin,
> > > > > > >> > > > >> > > >> For the unclean.recovery.strategy config name,
> > how
> > > > > about
> > > > > > >> we
> > > > > > >> > use
> > > > > > >> > > > the
> > > > > > >> > > > >> > > >> following
> > > > > > >> > > > >> > > >> None. It basically means no unclean recovery
> > will
> > > be
> > > > > > >> > performed.
> > > > > > >> > > > >> > > >> Aggressive. It means availability goes first.
> > > > Whenever
> > > > > > the
> > > > > > >> > > > partition
> > > > > > >> > > > >> > > can't
> > > > > > >> > > > >> > > >> elect a durable replica, the controller will
> try
> > > the
> > > > > > >> unclean
> > > > > > >> > > > >> recovery.
> > > > > > >> > > > >> > > >> Balanced. It is the balance point of the
> > > > availability
> > > > > > >> > > > >> > first(Aggressive)
> > > > > > >> > > > >> > > >> and least availability(None). The controller
> > > > performs
> > > > > > >> unclean
> > > > > > >> > > > >> recovery
> > > > > > >> > > > >> > > when
> > > > > > >> > > > >> > > >> both ISR and ELR are empty.
> > > > > > >> > > > >> > > >>
> > > > > > >> > > > >> > > >>
> > > > > > >> > > > >> > > >> On Fri, Sep 15, 2023 at 11:42 AM Calvin Liu <
> > > > > > >> > > caliu@confluent.io>
> > > > > > >> > > > >> > wrote:
> > > > > > >> > > > >> > > >>
> > > > > > >> > > > >> > > >>> Hi Colin,
> > > > > > >> > > > >> > > >>>
> > > > > > >> > > > >> > > >>> > So, the proposal is that if someone sets
> > > > > > >> > > > >> > > "unclean.leader.election.enable
> > > > > > >> > > > >> > > >>> = true"...
> > > > > > >> > > > >> > > >>>
> > > > > > >> > > > >> > > >>>
> > > > > > >> > > > >> > > >>> The idea is to use one of the
> > > > > > >> unclean.leader.election.enable
> > > > > > >> > > and
> > > > > > >> > > > >> > > >>> unclean.recovery.strategy based on the
> > > > > > >> > > > unclean.recovery.Enabled. A
> > > > > > >> > > > >> > > possible
> > > > > > >> > > > >> > > >>> version can be
> > > > > > >> > > > >> > > >>>
> > > > > > >> > > > >> > > >>> If unclean.recovery.Enabled:
> > > > > > >> > > > >> > > >>>
> > > > > > >> > > > >> > > >>> {
> > > > > > >> > > > >> > > >>>
> > > > > > >> > > > >> > > >>> Check unclean.recovery.strategy. If set, use
> > it.
> > > > > > >> Otherwise,
> > > > > > >> > > > check
> > > > > > >> > > > >> > > >>> unclean.leader.election.enable and translate
> it
> > > to
> > > > > > >> > > > >> > > >>> unclean.recovery.strategy.
> > > > > > >> > > > >> > > >>>
> > > > > > >> > > > >> > > >>> } else {
> > > > > > >> > > > >> > > >>>
> > > > > > >> > > > >> > > >>> Use unclean.leader.election.enable
> > > > > > >> > > > >> > > >>>
> > > > > > >> > > > >> > > >>> }
> > > > > > >> > > > >> > > >>>
> > > > > > >> > > > >> > > >>>
> > > > > > >> > > > >> > > >>> —--------
> > > > > > >> > > > >> > > >>>
> > > > > > >> > > > >> > > >>> >The configuration key should be
> > > > > > >> > > > >> "unclean.recovery.manager.enabled",
> > > > > > >> > > > >> > > >>> right?
> > > > > > >> > > > >> > > >>>
> > > > > > >> > > > >> > > >>>
> > > > > > >> > > > >> > > >>> I think we have two ways of choosing a leader
> > > > > > uncleanly,
> > > > > > >> > > unclean
> > > > > > >> > > > >> > leader
> > > > > > >> > > > >> > > >>> election and unclean recovery(log inspection)
> > and
> > > > we
> > > > > > try
> > > > > > >> to
> > > > > > >> > > > switch
> > > > > > >> > > > >> > > between
> > > > > > >> > > > >> > > >>> them.
> > > > > > >> > > > >> > > >>>
> > > > > > >> > > > >> > > >>> Do you mean we want to develop two ways of
> > > > performing
> > > > > > the
> > > > > > >> > > > unclean
> > > > > > >> > > > >> > > >>> recovery and one of them is using “unclean
> > > recovery
> > > > > > >> > manager”?
> > > > > > >> > > I
> > > > > > >> > > > >> guess
> > > > > > >> > > > >> > > we
> > > > > > >> > > > >> > > >>> haven’t discussed the second way.
> > > > > > >> > > > >> > > >>>
> > > > > > >> > > > >> > > >>>
> > > > > > >> > > > >> > > >>> —-------
> > > > > > >> > > > >> > > >>>
> > > > > > >> > > > >> > > >>> >How do these 4 levels of overrides interact
> > with
> > > > > your
> > > > > > >> new
> > > > > > >> > > > >> > > >>> configurations?
> > > > > > >> > > > >> > > >>>
> > > > > > >> > > > >> > > >>>
> > > > > > >> > > > >> > > >>> I do notice in the Kraft controller code, the
> > > > method
> > > > > to
> > > > > > >> > check
> > > > > > >> > > > >> whether
> > > > > > >> > > > >> > > >>> perform unclean leader election is hard coded
> > to
> > > > > false
> > > > > > >> since
> > > > > > >> > > > >> > > >>> 2021(uncleanLeaderElectionEnabledForTopic).
> > Isn’t
> > > > it
> > > > > a
> > > > > > >> good
> > > > > > >> > > > chance
> > > > > > >> > > > >> to
> > > > > > >> > > > >> > > >>> completely deprecate the
> > > > > > unclean.leader.election.enable?
> > > > > > >> We
> > > > > > >> > > > don’t
> > > > > > >> > > > >> > even
> > > > > > >> > > > >> > > have
> > > > > > >> > > > >> > > >>> to worry about the config conversion.
> > > > > > >> > > > >> > > >>>
> > > > > > >> > > > >> > > >>> On the other hand, whatever the override is,
> as
> > > > long
> > > > > as
> > > > > > >> the
> > > > > > >> > > > >> > controller
> > > > > > >> > > > >> > > >>> can have the final effective
> > > > > > >> unclean.leader.election.enable,
> > > > > > >> > > the
> > > > > > >> > > > >> > topic
> > > > > > >> > > > >> > > >>> level config unclean.recovery.strategy, the
> > > cluster
> > > > > > level
> > > > > > >> > > config
> > > > > > >> > > > >> > > >>> unclean.recovery.Enabled, the controller can
> > > > > calculate
> > > > > > >> the
> > > > > > >> > > > correct
> > > > > > >> > > > >> > > methods
> > > > > > >> > > > >> > > >>> to use right?
> > > > > > >> > > > >> > > >>>
> > > > > > >> > > > >> > > >>>
> > > > > > >> > > > >> > > >>> On Fri, Sep 15, 2023 at 10:02 AM Colin
> McCabe <
> > > > > > >> > > > cmccabe@apache.org>
> > > > > > >> > > > >> > > wrote:
> > > > > > >> > > > >> > > >>>
> > > > > > >> > > > >> > > >>>> On Thu, Sep 14, 2023, at 22:23, Calvin Liu
> > > wrote:
> > > > > > >> > > > >> > > >>>> > Hi Colin
> > > > > > >> > > > >> > > >>>> > 1. I think using the new config name is
> more
> > > > > clear.
> > > > > > >> > > > >> > > >>>> >        a. The unclean leader election is
> > > > actually
> > > > > > >> removed
> > > > > > >> > > if
> > > > > > >> > > > >> > unclean
> > > > > > >> > > > >> > > >>>> > recovery is in use.
> > > > > > >> > > > >> > > >>>> >        b. Using multiple values in
> > > > > > >> > > > >> unclean.leader.election.enable
> > > > > > >> > > > >> > is
> > > > > > >> > > > >> > > >>>> > confusing and it will be more confusing
> > after
> > > > > people
> > > > > > >> > forget
> > > > > > >> > > > >> about
> > > > > > >> > > > >> > > this
> > > > > > >> > > > >> > > >>>> > discussion.
> > > > > > >> > > > >> > > >>>>
> > > > > > >> > > > >> > > >>>> Hi Calvin,
> > > > > > >> > > > >> > > >>>>
> > > > > > >> > > > >> > > >>>> So, the proposal is that if someone sets
> > > > > > >> > > > >> > > "unclean.leader.election.enable
> > > > > > >> > > > >> > > >>>> = true" but then sets one of your new
> > > > > configurations,
> > > > > > >> the
> > > > > > >> > > > value of
> > > > > > >> > > > >> > > >>>> unclean.leader.election.enable is ignored?
> > That
> > > > > seems
> > > > > > >> less
> > > > > > >> > > > clear
> > > > > > >> > > > >> to
> > > > > > >> > > > >> > > me, not
> > > > > > >> > > > >> > > >>>> more. Just in general, having multiple
> > > > configuration
> > > > > > >> keys
> > > > > > >> > to
> > > > > > >> > > > >> control
> > > > > > >> > > > >> > > the
> > > > > > >> > > > >> > > >>>> same thing confuses users. Basically, they
> are
> > > > > sitting
> > > > > > >> at a
> > > > > > >> > > > giant
> > > > > > >> > > > >> > > control
> > > > > > >> > > > >> > > >>>> panel, and some of the levers do nothing.
> > > > > > >> > > > >> > > >>>>
> > > > > > >> > > > >> > > >>>> > 2. Sorry I forgot to mention in the
> response
> > > > that
> > > > > I
> > > > > > >> did
> > > > > > >> > add
> > > > > > >> > > > the
> > > > > > >> > > > >> > > >>>> > unclean.recovery.Enabled flag.
> > > > > > >> > > > >> > > >>>>
> > > > > > >> > > > >> > > >>>> The configuration key should be
> > > > > > >> > > > >> "unclean.recovery.manager.enabled",
> > > > > > >> > > > >> > > >>>> right? Becuase we can do "unclean recovery"
> > > > without
> > > > > > the
> > > > > > >> > > > manager.
> > > > > > >> > > > >> > > Disabling
> > > > > > >> > > > >> > > >>>> the manager just means we use a different
> > > > mechanism
> > > > > > for
> > > > > > >> > > > recovery.
> > > > > > >> > > > >> > > >>>>
> > > > > > >> > > > >> > > >>>> >        c. Maybe I underestimated the
> > challenge
> > > > of
> > > > > > >> > replacing
> > > > > > >> > > > the
> > > > > > >> > > > >> > > >>>> config. Any
> > > > > > >> > > > >> > > >>>> > implementation problems ahead?
> > > > > > >> > > > >> > > >>>>
> > > > > > >> > > > >> > > >>>> There are four levels of overrides for
> > > > > > >> > > > >> > unclean.leader.election.enable.
> > > > > > >> > > > >> > > >>>>
> > > > > > >> > > > >> > > >>>> 1. static configuration for node.
> > > > > > >> > > > >> > > >>>>     This goes in the configuration file,
> > > typically
> > > > > > named
> > > > > > >> > > > >> > > >>>> server.properties
> > > > > > >> > > > >> > > >>>>
> > > > > > >> > > > >> > > >>>> 2. dynamic configuration for node default
> > > > > > >> > > > >> > > >>>>   ConfigResource(type=BROKER, name="")
> > > > > > >> > > > >> > > >>>>
> > > > > > >> > > > >> > > >>>> 3. dynamic configuration for node
> > > > > > >> > > > >> > > >>>>   ConfigResource(type=BROKER,
> name=<controller
> > > > id>)
> > > > > > >> > > > >> > > >>>>
> > > > > > >> > > > >> > > >>>> 4. dynamic configuration for topic
> > > > > > >> > > > >> > > >>>>   ConfigResource(type=TOPIC,
> > name=<topic-name>)
> > > > > > >> > > > >> > > >>>>
> > > > > > >> > > > >> > > >>>> How do these 4 levels of overrides interact
> > with
> > > > > your
> > > > > > >> new
> > > > > > >> > > > >> > > >>>> configurations? If the new configurations
> > > dominate
> > > > > > over
> > > > > > >> the
> > > > > > >> > > old
> > > > > > >> > > > >> > ones,
> > > > > > >> > > > >> > > it
> > > > > > >> > > > >> > > >>>> seems like this will get a lot more
> confusing
> > to
> > > > > > >> implement
> > > > > > >> > > (and
> > > > > > >> > > > >> also
> > > > > > >> > > > >> > > to
> > > > > > >> > > > >> > > >>>> use.)
> > > > > > >> > > > >> > > >>>>
> > > > > > >> > > > >> > > >>>> Again, I'd recommend just adding some new
> > values
> > > > to
> > > > > > >> > > > >> > > >>>> unclean.leader.election.enable. It's simple
> > and
> > > > will
> > > > > > >> > prevent
> > > > > > >> > > > user
> > > > > > >> > > > >> > > confusion
> > > > > > >> > > > >> > > >>>> (as well as developer confusion.)
> > > > > > >> > > > >> > > >>>>
> > > > > > >> > > > >> > > >>>> best,
> > > > > > >> > > > >> > > >>>> Colin
> > > > > > >> > > > >> > > >>>>
> > > > > > >> > > > >> > > >>>>
> > > > > > >> > > > >> > > >>>> > 3. About the admin client, I mentioned 3
> > > changes
> > > > > in
> > > > > > >> the
> > > > > > >> > > > client.
> > > > > > >> > > > >> > > >>>> Anything
> > > > > > >> > > > >> > > >>>> > else I missed in the KIP?
> > > > > > >> > > > >> > > >>>> >       a. The client will switch to using
> the
> > > new
> > > > > RPC
> > > > > > >> > > instead
> > > > > > >> > > > of
> > > > > > >> > > > >> > > >>>> > MetadataRequest for the topics.
> > > > > > >> > > > >> > > >>>> >       b. The TopicPartitionInfo used in
> > > > > > >> TopicDescription
> > > > > > >> > > > needs
> > > > > > >> > > > >> to
> > > > > > >> > > > >> > > add
> > > > > > >> > > > >> > > >>>> new
> > > > > > >> > > > >> > > >>>> > fields related to the ELR.
> > > > > > >> > > > >> > > >>>> >       c. The outputs will add the ELR
> > related
> > > > > > fields.
> > > > > > >> > > > >> > > >>>> >
> > > > > > >> > > > >> > > >>>> > On Thu, Sep 14, 2023 at 9:19 PM Colin
> > McCabe <
> > > > > > >> > > > >> cmccabe@apache.org>
> > > > > > >> > > > >> > > >>>> wrote:
> > > > > > >> > > > >> > > >>>> >
> > > > > > >> > > > >> > > >>>> >> Hi Calvin,
> > > > > > >> > > > >> > > >>>> >>
> > > > > > >> > > > >> > > >>>> >> Thanks for the changes.
> > > > > > >> > > > >> > > >>>> >>
> > > > > > >> > > > >> > > >>>> >> 1. Earlier I commented that creating
> > > > > > >> > > > >> "unclean.recovery.strategy "
> > > > > > >> > > > >> > > is
> > > > > > >> > > > >> > > >>>> not
> > > > > > >> > > > >> > > >>>> >> necessary, and we can just reuse the
> > existing
> > > > > > >> > > > >> > > >>>> >> "unclean.leader.election.enable"
> > > configuration
> > > > > key.
> > > > > > >> > Let's
> > > > > > >> > > > >> discuss
> > > > > > >> > > > >> > > >>>> that.
> > > > > > >> > > > >> > > >>>> >>
> > > > > > >> > > > >> > > >>>> >> 2.I also don't understand why you didn't
> > add
> > > a
> > > > > > >> > > > configuration to
> > > > > > >> > > > >> > > >>>> enable or
> > > > > > >> > > > >> > > >>>> >> disable the Unclean Recovery Manager.
> This
> > > > seems
> > > > > > >> like a
> > > > > > >> > > very
> > > > > > >> > > > >> > simple
> > > > > > >> > > > >> > > >>>> way to
> > > > > > >> > > > >> > > >>>> >> handle the staging issue which we
> > discussed.
> > > > The
> > > > > > URM
> > > > > > >> can
> > > > > > >> > > > just
> > > > > > >> > > > >> be
> > > > > > >> > > > >> > > >>>> turned off
> > > > > > >> > > > >> > > >>>> >> until it is production ready. Let's
> discuss
> > > > this.
> > > > > > >> > > > >> > > >>>> >>
> > > > > > >> > > > >> > > >>>> >> 3. You still need to describe the changes
> > to
> > > > > > >> AdminClient
> > > > > > >> > > > that
> > > > > > >> > > > >> are
> > > > > > >> > > > >> > > >>>> needed
> > > > > > >> > > > >> > > >>>> >> to use DescribeTopicRequest.
> > > > > > >> > > > >> > > >>>> >>
> > > > > > >> > > > >> > > >>>> >> Keep at it. It's looking better. :)
> > > > > > >> > > > >> > > >>>> >>
> > > > > > >> > > > >> > > >>>> >> best,
> > > > > > >> > > > >> > > >>>> >> Colin
> > > > > > >> > > > >> > > >>>> >>
> > > > > > >> > > > >> > > >>>> >>
> > > > > > >> > > > >> > > >>>> >> On Thu, Sep 14, 2023, at 11:03, Calvin
> Liu
> > > > wrote:
> > > > > > >> > > > >> > > >>>> >> > Hi Colin
> > > > > > >> > > > >> > > >>>> >> > Thanks for the comments!
> > > > > > >> > > > >> > > >>>> >> >
> > > > > > >> > > > >> > > >>>> >> > I did the following changes
> > > > > > >> > > > >> > > >>>> >> >
> > > > > > >> > > > >> > > >>>> >> >    1.
> > > > > > >> > > > >> > > >>>> >> >
> > > > > > >> > > > >> > > >>>> >> >    Simplified the API spec section to
> > only
> > > > > > include
> > > > > > >> the
> > > > > > >> > > > diff.
> > > > > > >> > > > >> > > >>>> >> >    2.
> > > > > > >> > > > >> > > >>>> >> >
> > > > > > >> > > > >> > > >>>> >> >    Reordered the HWM requirement
> section.
> > > > > > >> > > > >> > > >>>> >> >    3.
> > > > > > >> > > > >> > > >>>> >> >
> > > > > > >> > > > >> > > >>>> >> >    Removed the URM implementation
> details
> > > to
> > > > > keep
> > > > > > >> the
> > > > > > >> > > > >> necessary
> > > > > > >> > > > >> > > >>>> >> >    characteristics to perform the
> unclean
> > > > > > recovery.
> > > > > > >> > > > >> > > >>>> >> >    1.
> > > > > > >> > > > >> > > >>>> >> >
> > > > > > >> > > > >> > > >>>> >> >       When to perform the unclean
> > recovery
> > > > > > >> > > > >> > > >>>> >> >       2.
> > > > > > >> > > > >> > > >>>> >> >
> > > > > > >> > > > >> > > >>>> >> >       Under different config, how the
> > > unclean
> > > > > > >> recovery
> > > > > > >> > > > finds
> > > > > > >> > > > >> > the
> > > > > > >> > > > >> > > >>>> leader.
> > > > > > >> > > > >> > > >>>> >> >       3.
> > > > > > >> > > > >> > > >>>> >> >
> > > > > > >> > > > >> > > >>>> >> >       How the config
> > > > > > unclean.leader.election.enable
> > > > > > >> > and
> > > > > > >> > > > >> > > >>>> >> >       unclean.recovery.strategy are
> > > converted
> > > > > > when
> > > > > > >> > users
> > > > > > >> > > > >> > > >>>> enable/disable
> > > > > > >> > > > >> > > >>>> >> the
> > > > > > >> > > > >> > > >>>> >> >       unclean recovery.
> > > > > > >> > > > >> > > >>>> >> >       4.
> > > > > > >> > > > >> > > >>>> >> >
> > > > > > >> > > > >> > > >>>> >> >    More details about how we change
> admin
> > > > > client.
> > > > > > >> > > > >> > > >>>> >> >    5.
> > > > > > >> > > > >> > > >>>> >> >
> > > > > > >> > > > >> > > >>>> >> >    API limits on the
> > > GetReplicaLogInfoRequest
> > > > > and
> > > > > > >> > > > >> > > >>>> DescribeTopicRequest.
> > > > > > >> > > > >> > > >>>> >> >    6.
> > > > > > >> > > > >> > > >>>> >> >
> > > > > > >> > > > >> > > >>>> >> >    Two metrics added
> > > > > > >> > > > >> > > >>>> >> >    1.
> > > > > > >> > > > >> > > >>>> >> >
> > > > > > >> > > > >> > > >>>> >> >
> > > > > > >> > > >  Kafka.controller.global_under_min_isr_partition_count
> > > > > > >> > > > >> > > >>>> >> >       2.
> > > > > > >> > > > >> > > >>>> >> >
> > > > > > >> > > > >> > > >>>> >> >
> > > > > > >>  kafka.controller.unclean_recovery_finished_count
> > > > > > >> > > > >> > > >>>> >> >
> > > > > > >> > > > >> > > >>>> >> >
> > > > > > >> > > > >> > > >>>> >> > On Wed, Sep 13, 2023 at 10:46 AM Colin
> > > > McCabe <
> > > > > > >> > > > >> > > cmccabe@apache.org>
> > > > > > >> > > > >> > > >>>> >> wrote:
> > > > > > >> > > > >> > > >>>> >> >
> > > > > > >> > > > >> > > >>>> >> >> On Tue, Sep 12, 2023, at 17:21, Calvin
> > Liu
> > > > > > wrote:
> > > > > > >> > > > >> > > >>>> >> >> > Hi Colin
> > > > > > >> > > > >> > > >>>> >> >> > Thanks for the comments!
> > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > >> > > > >> > > >>>> >> >>
> > > > > > >> > > > >> > > >>>> >> >> Hi Calvin,
> > > > > > >> > > > >> > > >>>> >> >>
> > > > > > >> > > > >> > > >>>> >> >> Thanks again for the KIP.
> > > > > > >> > > > >> > > >>>> >> >>
> > > > > > >> > > > >> > > >>>> >> >> One meta-comment: it's usually better
> to
> > > > just
> > > > > > do a
> > > > > > >> > diff
> > > > > > >> > > > on a
> > > > > > >> > > > >> > > >>>> message
> > > > > > >> > > > >> > > >>>> >> spec
> > > > > > >> > > > >> > > >>>> >> >> file or java file if you're including
> > > > changes
> > > > > to
> > > > > > >> it
> > > > > > >> > in
> > > > > > >> > > > the
> > > > > > >> > > > >> > KIP.
> > > > > > >> > > > >> > > >>>> This is
> > > > > > >> > > > >> > > >>>> >> >> easier to read than looking for "new
> > > fields
> > > > > > begin"
> > > > > > >> > etc.
> > > > > > >> > > > in
> > > > > > >> > > > >> the
> > > > > > >> > > > >> > > >>>> text, and
> > > > > > >> > > > >> > > >>>> >> >> gracefully handles the case where
> > existing
> > > > > > fields
> > > > > > >> > were
> > > > > > >> > > > >> > changed.
> > > > > > >> > > > >> > > >>>> >> >>
> > > > > > >> > > > >> > > >>>> >> >> > Rewrite the Additional High
> Watermark
> > > > > > >> advancement
> > > > > > >> > > > >> > requirement
> > > > > > >> > > > >> > > >>>> >> >> > There was feedback on this section
> > that
> > > > some
> > > > > > >> > readers
> > > > > > >> > > > may
> > > > > > >> > > > >> not
> > > > > > >> > > > >> > > be
> > > > > > >> > > > >> > > >>>> >> familiar
> > > > > > >> > > > >> > > >>>> >> >> > with HWM and Ack=0,1,all requests.
> > This
> > > > can
> > > > > > help
> > > > > > >> > them
> > > > > > >> > > > >> > > understand
> > > > > > >> > > > >> > > >>>> the
> > > > > > >> > > > >> > > >>>> >> >> > proposal. I will rewrite this part
> for
> > > > more
> > > > > > >> > > > readability.
> > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > >> > > > >> > > >>>> >> >>
> > > > > > >> > > > >> > > >>>> >> >> To be clear, I wasn't suggesting
> > dropping
> > > > > either
> > > > > > >> > > > section. I
> > > > > > >> > > > >> > > agree
> > > > > > >> > > > >> > > >>>> that
> > > > > > >> > > > >> > > >>>> >> >> they add useful background. I was just
> > > > > > suggesting
> > > > > > >> > that
> > > > > > >> > > we
> > > > > > >> > > > >> > should
> > > > > > >> > > > >> > > >>>> discuss
> > > > > > >> > > > >> > > >>>> >> >> the "acks" setting AFTER discussing
> the
> > > new
> > > > > high
> > > > > > >> > > > watermark
> > > > > > >> > > > >> > > >>>> advancement
> > > > > > >> > > > >> > > >>>> >> >> conditions. We also should discuss
> > acks=0.
> > > > > While
> > > > > > >> it
> > > > > > >> > > isn't
> > > > > > >> > > > >> > > >>>> conceptually
> > > > > > >> > > > >> > > >>>> >> much
> > > > > > >> > > > >> > > >>>> >> >> different than acks=1 here, its
> omission
> > > > from
> > > > > > this
> > > > > > >> > > > section
> > > > > > >> > > > >> is
> > > > > > >> > > > >> > > >>>> confusing.
> > > > > > >> > > > >> > > >>>> >> >>
> > > > > > >> > > > >> > > >>>> >> >> > Unclean recovery
> > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > >> > > > >> > > >>>> >> >> > The plan is to replace the
> > > > > > >> > > > unclean.leader.election.enable
> > > > > > >> > > > >> > with
> > > > > > >> > > > >> > > >>>> >> >> > unclean.recovery.strategy. If the
> > > Unclean
> > > > > > >> Recovery
> > > > > > >> > is
> > > > > > >> > > > >> > enabled
> > > > > > >> > > > >> > > >>>> then it
> > > > > > >> > > > >> > > >>>> >> >> deals
> > > > > > >> > > > >> > > >>>> >> >> > with the three options in the
> > > > > > >> > > > unclean.recovery.strategy.
> > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > >> > > > >> > > >>>> >> >> > Let’s refine the Unclean Recovery.
> We
> > > have
> > > > > > >> already
> > > > > > >> > > > taken a
> > > > > > >> > > > >> > > lot of
> > > > > > >> > > > >> > > >>>> >> >> > suggestions and I hope to enhance
> the
> > > > > > >> durability of
> > > > > > >> > > > Kafka
> > > > > > >> > > > >> to
> > > > > > >> > > > >> > > the
> > > > > > >> > > > >> > > >>>> next
> > > > > > >> > > > >> > > >>>> >> >> level
> > > > > > >> > > > >> > > >>>> >> >> > with this KIP.
> > > > > > >> > > > >> > > >>>> >> >>
> > > > > > >> > > > >> > > >>>> >> >> I am OK with doing the unclean leader
> > > > recovery
> > > > > > >> > > > improvements
> > > > > > >> > > > >> in
> > > > > > >> > > > >> > > >>>> this KIP.
> > > > > > >> > > > >> > > >>>> >> >> However, I think we need to really
> work
> > on
> > > > the
> > > > > > >> > > > configuration
> > > > > > >> > > > >> > > >>>> settings.
> > > > > > >> > > > >> > > >>>> >> >>
> > > > > > >> > > > >> > > >>>> >> >> Configuration overrides are often
> quite
> > > > messy.
> > > > > > For
> > > > > > >> > > > example,
> > > > > > >> > > > >> > the
> > > > > > >> > > > >> > > >>>> cases
> > > > > > >> > > > >> > > >>>> >> >> where we have log.roll.hours and
> > > > > > >> log.roll.segment.ms
> > > > > > >> > ,
> > > > > > >> > > > the
> > > > > > >> > > > >> > user
> > > > > > >> > > > >> > > >>>> has to
> > > > > > >> > > > >> > > >>>> >> >> remember which one takes precedence,
> and
> > > it
> > > > is
> > > > > > not
> > > > > > >> > > > obvious.
> > > > > > >> > > > >> > So,
> > > > > > >> > > > >> > > >>>> rather
> > > > > > >> > > > >> > > >>>> >> than
> > > > > > >> > > > >> > > >>>> >> >> creating a new configuration, why not
> > add
> > > > > > >> additional
> > > > > > >> > > > values
> > > > > > >> > > > >> to
> > > > > > >> > > > >> > > >>>> >> >> "unclean.leader.election.enable"? I
> > think
> > > > this
> > > > > > >> will
> > > > > > >> > be
> > > > > > >> > > > >> simpler
> > > > > > >> > > > >> > > for
> > > > > > >> > > > >> > > >>>> >> people
> > > > > > >> > > > >> > > >>>> >> >> to understand, and simpler in the code
> > as
> > > > > well.
> > > > > > >> > > > >> > > >>>> >> >>
> > > > > > >> > > > >> > > >>>> >> >> What if we continued to use
> > > > > > >> > > > "unclean.leader.election.enable"
> > > > > > >> > > > >> > but
> > > > > > >> > > > >> > > >>>> >> extended
> > > > > > >> > > > >> > > >>>> >> >> it so that it took a string? Then the
> > > string
> > > > > > could
> > > > > > >> > have
> > > > > > >> > > > >> these
> > > > > > >> > > > >> > > >>>> values:
> > > > > > >> > > > >> > > >>>> >> >>
> > > > > > >> > > > >> > > >>>> >> >> never
> > > > > > >> > > > >> > > >>>> >> >>     never automatically do an unclean
> > > leader
> > > > > > >> election
> > > > > > >> > > > under
> > > > > > >> > > > >> > any
> > > > > > >> > > > >> > > >>>> >> conditions
> > > > > > >> > > > >> > > >>>> >> >>
> > > > > > >> > > > >> > > >>>> >> >> false / default
> > > > > > >> > > > >> > > >>>> >> >>     only do an unclean leader election
> > if
> > > > > there
> > > > > > >> may
> > > > > > >> > be
> > > > > > >> > > > >> > possible
> > > > > > >> > > > >> > > >>>> data
> > > > > > >> > > > >> > > >>>> >> loss
> > > > > > >> > > > >> > > >>>> >> >>
> > > > > > >> > > > >> > > >>>> >> >> true / always
> > > > > > >> > > > >> > > >>>> >> >>     always do an unclean leader
> election
> > > if
> > > > we
> > > > > > >> can't
> > > > > > >> > > > >> > immediately
> > > > > > >> > > > >> > > >>>> elect a
> > > > > > >> > > > >> > > >>>> >> >> leader
> > > > > > >> > > > >> > > >>>> >> >>
> > > > > > >> > > > >> > > >>>> >> >> It's a bit awkward that false maps to
> > > > default
> > > > > > >> rather
> > > > > > >> > > > than to
> > > > > > >> > > > >> > > >>>> never. But
> > > > > > >> > > > >> > > >>>> >> >> this awkwardness exists if we use two
> > > > > different
> > > > > > >> > > > >> configuration
> > > > > > >> > > > >> > > keys
> > > > > > >> > > > >> > > >>>> as
> > > > > > >> > > > >> > > >>>> >> well.
> > > > > > >> > > > >> > > >>>> >> >> The reason for the awkwardness is that
> > we
> > > > > simply
> > > > > > >> > don't
> > > > > > >> > > > want
> > > > > > >> > > > >> > most
> > > > > > >> > > > >> > > >>>> of the
> > > > > > >> > > > >> > > >>>> >> >> people currently setting
> > > > > > >> > > > >> unclean.leader.election.enable=false
> > > > > > >> > > > >> > to
> > > > > > >> > > > >> > > >>>> get the
> > > > > > >> > > > >> > > >>>> >> >> "never" behavior. We have to bite that
> > > > bullet.
> > > > > > >> Better
> > > > > > >> > > to
> > > > > > >> > > > be
> > > > > > >> > > > >> > > clear
> > > > > > >> > > > >> > > >>>> and
> > > > > > >> > > > >> > > >>>> >> >> explicit than hide it.
> > > > > > >> > > > >> > > >>>> >> >>
> > > > > > >> > > > >> > > >>>> >> >> Another thing that's a bit awkward is
> > > having
> > > > > two
> > > > > > >> > > > different
> > > > > > >> > > > >> > ways
> > > > > > >> > > > >> > > to
> > > > > > >> > > > >> > > >>>> do
> > > > > > >> > > > >> > > >>>> >> >> unclean leader election specified in
> the
> > > > KIP.
> > > > > > You
> > > > > > >> > > > descirbe
> > > > > > >> > > > >> two
> > > > > > >> > > > >> > > >>>> methods:
> > > > > > >> > > > >> > > >>>> >> the
> > > > > > >> > > > >> > > >>>> >> >> simple "choose the last leader"
> method,
> > > and
> > > > > the
> > > > > > >> > > "unclean
> > > > > > >> > > > >> > > recovery
> > > > > > >> > > > >> > > >>>> >> manager"
> > > > > > >> > > > >> > > >>>> >> >> method. I understand why you did it
> this
> > > way
> > > > > --
> > > > > > >> > "choose
> > > > > > >> > > > the
> > > > > > >> > > > >> > last
> > > > > > >> > > > >> > > >>>> >> leader" is
> > > > > > >> > > > >> > > >>>> >> >> simple, and will help us deliver an
> > > > > > implementation
> > > > > > >> > > > quickly,
> > > > > > >> > > > >> > > while
> > > > > > >> > > > >> > > >>>> the
> > > > > > >> > > > >> > > >>>> >> URM
> > > > > > >> > > > >> > > >>>> >> >> is preferable in the long term. My
> > > > suggestion
> > > > > > >> here is
> > > > > > >> > > to
> > > > > > >> > > > >> > > separate
> > > > > > >> > > > >> > > >>>> the
> > > > > > >> > > > >> > > >>>> >> >> decision of HOW to do unclean leader
> > > > election
> > > > > > from
> > > > > > >> > the
> > > > > > >> > > > >> > decision
> > > > > > >> > > > >> > > of
> > > > > > >> > > > >> > > >>>> WHEN
> > > > > > >> > > > >> > > >>>> >> to
> > > > > > >> > > > >> > > >>>> >> >> do it.
> > > > > > >> > > > >> > > >>>> >> >>
> > > > > > >> > > > >> > > >>>> >> >> So in other words, have
> > > > > > >> > > "unclean.leader.election.enable"
> > > > > > >> > > > >> > specify
> > > > > > >> > > > >> > > >>>> when we
> > > > > > >> > > > >> > > >>>> >> >> do unclean leader election, and have a
> > new
> > > > > > >> > > configuration
> > > > > > >> > > > >> like
> > > > > > >> > > > >> > > >>>> >> >> "unclean.recovery.manager.enable" to
> > > > determine
> > > > > > if
> > > > > > >> we
> > > > > > >> > > use
> > > > > > >> > > > the
> > > > > > >> > > > >> > > URM.
> > > > > > >> > > > >> > > >>>> >> >> Presumably the URM will take some time
> > to
> > > > get
> > > > > > >> fully
> > > > > > >> > > > stable,
> > > > > > >> > > > >> so
> > > > > > >> > > > >> > > >>>> this can
> > > > > > >> > > > >> > > >>>> >> >> default to false for a while, and we
> can
> > > > flip
> > > > > > the
> > > > > > >> > > > default to
> > > > > > >> > > > >> > > true
> > > > > > >> > > > >> > > >>>> when
> > > > > > >> > > > >> > > >>>> >> we
> > > > > > >> > > > >> > > >>>> >> >> feel ready.
> > > > > > >> > > > >> > > >>>> >> >>
> > > > > > >> > > > >> > > >>>> >> >> The URM is somewhat under-described
> > here.
> > > I
> > > > > > think
> > > > > > >> we
> > > > > > >> > > > need a
> > > > > > >> > > > >> > few
> > > > > > >> > > > >> > > >>>> >> >> configurations here for it. For
> example,
> > > we
> > > > > > need a
> > > > > > >> > > > >> > > configuration to
> > > > > > >> > > > >> > > >>>> >> specify
> > > > > > >> > > > >> > > >>>> >> >> how long it should wait for a broker
> to
> > > > > respond
> > > > > > to
> > > > > > >> > its
> > > > > > >> > > > RPCs
> > > > > > >> > > > >> > > before
> > > > > > >> > > > >> > > >>>> >> moving
> > > > > > >> > > > >> > > >>>> >> >> on. We also need to understand how the
> > URM
> > > > > > >> interacts
> > > > > > >> > > with
> > > > > > >> > > > >> > > >>>> >> >>
> unclean.leader.election.enable=always. I
> > > > > assume
> > > > > > >> that
> > > > > > >> > > with
> > > > > > >> > > > >> > > "always"
> > > > > > >> > > > >> > > >>>> we
> > > > > > >> > > > >> > > >>>> >> will
> > > > > > >> > > > >> > > >>>> >> >> just unconditionally use the URM
> rather
> > > than
> > > > > > >> choosing
> > > > > > >> > > > >> > randomly.
> > > > > > >> > > > >> > > >>>> But this
> > > > > > >> > > > >> > > >>>> >> >> should be spelled out in the KIP.
> > > > > > >> > > > >> > > >>>> >> >>
> > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > >> > > > >> > > >>>> >> >> > DescribeTopicRequest
> > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > >> > > > >> > > >>>> >> >> >    1.
> > > > > > >> > > > >> > > >>>> >> >> >    Yes, the plan is to replace the
> > > > > > >> MetadataRequest
> > > > > > >> > > with
> > > > > > >> > > > >> the
> > > > > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest for the
> admin
> > > > > clients.
> > > > > > >> Will
> > > > > > >> > > > check
> > > > > > >> > > > >> > the
> > > > > > >> > > > >> > > >>>> details.
> > > > > > >> > > > >> > > >>>> >> >>
> > > > > > >> > > > >> > > >>>> >> >> Sounds good. But as I said, you need
> to
> > > > > specify
> > > > > > >> how
> > > > > > >> > > > >> > AdminClient
> > > > > > >> > > > >> > > >>>> >> interacts
> > > > > > >> > > > >> > > >>>> >> >> with the new request. This will
> involve
> > > > adding
> > > > > > >> some
> > > > > > >> > > > fields
> > > > > > >> > > > >> to
> > > > > > >> > > > >> > > >>>> >> >> TopicDescription.java. And you need to
> > > > specify
> > > > > > the
> > > > > > >> > > > changes
> > > > > > >> > > > >> to
> > > > > > >> > > > >> > > the
> > > > > > >> > > > >> > > >>>> >> >> kafka-topics.sh command line tool.
> > > Otherwise
> > > > > we
> > > > > > >> > cannot
> > > > > > >> > > > use
> > > > > > >> > > > >> the
> > > > > > >> > > > >> > > >>>> tool to
> > > > > > >> > > > >> > > >>>> >> see
> > > > > > >> > > > >> > > >>>> >> >> the new information.
> > > > > > >> > > > >> > > >>>> >> >>
> > > > > > >> > > > >> > > >>>> >> >> The new requests, DescribeTopicRequest
> > and
> > > > > > >> > > > >> > > >>>> GetReplicaLogInfoRequest,
> > > > > > >> > > > >> > > >>>> >> need
> > > > > > >> > > > >> > > >>>> >> >> to have limits placed on them so that
> > > their
> > > > > size
> > > > > > >> > can't
> > > > > > >> > > be
> > > > > > >> > > > >> > > >>>> infinite. We
> > > > > > >> > > > >> > > >>>> >> >> don't want to propagate the current
> > > problems
> > > > > of
> > > > > > >> > > > >> > MetadataRequest,
> > > > > > >> > > > >> > > >>>> where
> > > > > > >> > > > >> > > >>>> >> >> clients can request massive responses
> > that
> > > > can
> > > > > > >> mess
> > > > > > >> > up
> > > > > > >> > > > the
> > > > > > >> > > > >> JVM
> > > > > > >> > > > >> > > when
> > > > > > >> > > > >> > > >>>> >> handled.
> > > > > > >> > > > >> > > >>>> >> >>
> > > > > > >> > > > >> > > >>>> >> >> Adding limits is simple for
> > > > > > >> GetReplicaLogInfoRequest
> > > > > > >> > --
> > > > > > >> > > > we
> > > > > > >> > > > >> can
> > > > > > >> > > > >> > > >>>> just say
> > > > > > >> > > > >> > > >>>> >> >> that only 2000 partitions at a time
> can
> > be
> > > > > > >> requested.
> > > > > > >> > > For
> > > > > > >> > > > >> > > >>>> >> >> DescribeTopicRequest we can probably
> > just
> > > > > limit
> > > > > > >> to 20
> > > > > > >> > > > topics
> > > > > > >> > > > >> > or
> > > > > > >> > > > >> > > >>>> >> something
> > > > > > >> > > > >> > > >>>> >> >> like that, to avoid the complexity of
> > > doing
> > > > > > >> > pagination
> > > > > > >> > > in
> > > > > > >> > > > >> this
> > > > > > >> > > > >> > > KIP.
> > > > > > >> > > > >> > > >>>> >> >>
> > > > > > >> > > > >> > > >>>> >> >> >    2.
> > > > > > >> > > > >> > > >>>> >> >> >    I can let the broker load the ELR
> > > info
> > > > so
> > > > > > >> that
> > > > > > >> > > they
> > > > > > >> > > > can
> > > > > > >> > > > >> > > serve
> > > > > > >> > > > >> > > >>>> the
> > > > > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest as well.
> > > > > > >> > > > >> > > >>>> >> >>
> > > > > > >> > > > >> > > >>>> >> >> Yes, it's fine to add to
> MetadataCache.
> > In
> > > > > fact,
> > > > > > >> > you'll
> > > > > > >> > > > be
> > > > > > >> > > > >> > > loading
> > > > > > >> > > > >> > > >>>> it
> > > > > > >> > > > >> > > >>>> >> >> anyway once it's added to
> > PartitionImage.
> > > > > > >> > > > >> > > >>>> >> >>
> > > > > > >> > > > >> > > >>>> >> >> >    3.
> > > > > > >> > > > >> > > >>>> >> >> >    Yeah, it does not make sense to
> > have
> > > > the
> > > > > > >> topic
> > > > > > >> > id
> > > > > > >> > > if
> > > > > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest is only used
> > by
> > > > the
> > > > > > >> admin
> > > > > > >> > > > client.
> > > > > > >> > > > >> > > >>>> >> >>
> > > > > > >> > > > >> > > >>>> >> >> OK. That makes things simpler. We can
> > > always
> > > > > > >> create a
> > > > > > >> > > new
> > > > > > >> > > > >> API
> > > > > > >> > > > >> > > later
> > > > > > >> > > > >> > > >>>> >> >> (hopefully not in this KIP!) to query
> by
> > > > topic
> > > > > > ID.
> > > > > > >> > > > >> > > >>>> >> >>
> > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > >> > > > >> > > >>>> >> >> > Metrics
> > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > >> > > > >> > > >>>> >> >> > As for overall cluster health
> > metrics, I
> > > > > think
> > > > > > >> > > > >> under-min-ISR
> > > > > > >> > > > >> > > is
> > > > > > >> > > > >> > > >>>> still
> > > > > > >> > > > >> > > >>>> >> a
> > > > > > >> > > > >> > > >>>> >> >> > useful one. ELR is more like a
> safety
> > > > belt.
> > > > > > When
> > > > > > >> > the
> > > > > > >> > > > ELR
> > > > > > >> > > > >> is
> > > > > > >> > > > >> > > >>>> used, the
> > > > > > >> > > > >> > > >>>> >> >> > cluster availability has already
> been
> > > > > > impacted.
> > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > >> > > > >> > > >>>> >> >> > Maybe we can have a metric to count
> > the
> > > > > > >> partitions
> > > > > > >> > > that
> > > > > > >> > > > >> > > sum(ISR,
> > > > > > >> > > > >> > > >>>> ELR)
> > > > > > >> > > > >> > > >>>> >> <
> > > > > > >> > > > >> > > >>>> >> >> min
> > > > > > >> > > > >> > > >>>> >> >> > ISR. What do you think?
> > > > > > >> > > > >> > > >>>> >> >>
> > > > > > >> > > > >> > > >>>> >> >> How about:
> > > > > > >> > > > >> > > >>>> >> >>
> > > > > > >> > > > >> > > >>>> >> >> A.  a metric for the totoal number of
> > > > > > >> under-min-isr
> > > > > > >> > > > >> > partitions?
> > > > > > >> > > > >> > > We
> > > > > > >> > > > >> > > >>>> don't
> > > > > > >> > > > >> > > >>>> >> >> have that in Apache Kafka at the
> moment.
> > > > > > >> > > > >> > > >>>> >> >>
> > > > > > >> > > > >> > > >>>> >> >> B. a metric for the number of unclean
> > > leader
> > > > > > >> > elections
> > > > > > >> > > we
> > > > > > >> > > > >> did
> > > > > > >> > > > >> > > (for
> > > > > > >> > > > >> > > >>>> >> >> simplicity, it can reset to 0 on
> > > controller
> > > > > > >> restart:
> > > > > > >> > we
> > > > > > >> > > > >> expect
> > > > > > >> > > > >> > > >>>> people to
> > > > > > >> > > > >> > > >>>> >> >> monitor the change over time anyway)
> > > > > > >> > > > >> > > >>>> >> >>
> > > > > > >> > > > >> > > >>>> >> >> best,
> > > > > > >> > > > >> > > >>>> >> >> Colin
> > > > > > >> > > > >> > > >>>> >> >>
> > > > > > >> > > > >> > > >>>> >> >>
> > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > >> > > > >> > > >>>> >> >> > Yeah, for the ongoing unclean
> > > recoveries,
> > > > > the
> > > > > > >> > > > controller
> > > > > > >> > > > >> can
> > > > > > >> > > > >> > > >>>> keep an
> > > > > > >> > > > >> > > >>>> >> >> > accurate count through failover
> > because
> > > > > > >> partition
> > > > > > >> > > > >> > registration
> > > > > > >> > > > >> > > >>>> can
> > > > > > >> > > > >> > > >>>> >> >> indicate
> > > > > > >> > > > >> > > >>>> >> >> > whether a recovery is needed.
> However,
> > > for
> > > > > the
> > > > > > >> > > happened
> > > > > > >> > > > >> > ones,
> > > > > > >> > > > >> > > >>>> unless
> > > > > > >> > > > >> > > >>>> >> we
> > > > > > >> > > > >> > > >>>> >> >> > want to persist the number
> somewhere,
> > we
> > > > can
> > > > > > >> only
> > > > > > >> > > > figure
> > > > > > >> > > > >> it
> > > > > > >> > > > >> > > out
> > > > > > >> > > > >> > > >>>> from
> > > > > > >> > > > >> > > >>>> >> the
> > > > > > >> > > > >> > > >>>> >> >> > log.
> > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > >> > > > >> > > >>>> >> >> > On Tue, Sep 12, 2023 at 3:16 PM
> Colin
> > > > > McCabe <
> > > > > > >> > > > >> > > cmccabe@apache.org
> > > > > > >> > > > >> > > >>>> >
> > > > > > >> > > > >> > > >>>> >> wrote:
> > > > > > >> > > > >> > > >>>> >> >> >
> > > > > > >> > > > >> > > >>>> >> >> >> Also, we should have metrics that
> > show
> > > > what
> > > > > > is
> > > > > > >> > going
> > > > > > >> > > > on
> > > > > > >> > > > >> > with
> > > > > > >> > > > >> > > >>>> regard
> > > > > > >> > > > >> > > >>>> >> to
> > > > > > >> > > > >> > > >>>> >> >> the
> > > > > > >> > > > >> > > >>>> >> >> >> eligible replica set. I'm not sure
> > > > exactly
> > > > > > >> what to
> > > > > > >> > > > >> suggest,
> > > > > > >> > > > >> > > but
> > > > > > >> > > > >> > > >>>> >> >> something
> > > > > > >> > > > >> > > >>>> >> >> >> that could identify when things are
> > > going
> > > > > > >> wrong in
> > > > > > >> > > the
> > > > > > >> > > > >> > > clsuter.
> > > > > > >> > > > >> > > >>>> >> >> >>
> > > > > > >> > > > >> > > >>>> >> >> >> For example, maybe a metric for
> > > > partitions
> > > > > > >> > > containing
> > > > > > >> > > > >> > > replicas
> > > > > > >> > > > >> > > >>>> that
> > > > > > >> > > > >> > > >>>> >> are
> > > > > > >> > > > >> > > >>>> >> >> >> ineligible to be leader? That would
> > > show
> > > > a
> > > > > > >> spike
> > > > > > >> > > when
> > > > > > >> > > > a
> > > > > > >> > > > >> > > broker
> > > > > > >> > > > >> > > >>>> had an
> > > > > > >> > > > >> > > >>>> >> >> >> unclean restart.
> > > > > > >> > > > >> > > >>>> >> >> >>
> > > > > > >> > > > >> > > >>>> >> >> >> Ideally, we'd also have a metric
> that
> > > > > > indicates
> > > > > > >> > when
> > > > > > >> > > > an
> > > > > > >> > > > >> > > unclear
> > > > > > >> > > > >> > > >>>> >> leader
> > > > > > >> > > > >> > > >>>> >> >> >> election or a recovery happened.
> > It's a
> > > > bit
> > > > > > >> tricky
> > > > > > >> > > > >> because
> > > > > > >> > > > >> > > the
> > > > > > >> > > > >> > > >>>> simple
> > > > > > >> > > > >> > > >>>> >> >> >> thing, of tracking it per
> controller,
> > > may
> > > > > be
> > > > > > a
> > > > > > >> bit
> > > > > > >> > > > >> > confusing
> > > > > > >> > > > >> > > >>>> during
> > > > > > >> > > > >> > > >>>> >> >> >> failovers.
> > > > > > >> > > > >> > > >>>> >> >> >>
> > > > > > >> > > > >> > > >>>> >> >> >> best,
> > > > > > >> > > > >> > > >>>> >> >> >> Colin
> > > > > > >> > > > >> > > >>>> >> >> >>
> > > > > > >> > > > >> > > >>>> >> >> >>
> > > > > > >> > > > >> > > >>>> >> >> >> On Tue, Sep 12, 2023, at 14:25,
> Colin
> > > > > McCabe
> > > > > > >> > wrote:
> > > > > > >> > > > >> > > >>>> >> >> >> > Hi Calvin,
> > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > >> > > > >> > > >>>> >> >> >> > Thanks for the KIP. I think this
> > is a
> > > > > great
> > > > > > >> > > > >> improvement.
> > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > >> > > > >> > > >>>> >> >> >> >> Additional High Watermark
> advance
> > > > > > >> requirement
> > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > >> > > > >> > > >>>> >> >> >> > Typo: change "advance" to
> > > "advancement"
> > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > >> > > > >> > > >>>> >> >> >> >> A bit recap of some key
> concepts.
> > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > >> > > > >> > > >>>> >> >> >> > Typo: change "bit" to "quick"
> > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > >> > > > >> > > >>>> >> >> >> >> Ack=1/all produce request. It
> > > defines
> > > > > when
> > > > > > >> the
> > > > > > >> > > > Kafka
> > > > > > >> > > > >> > > server
> > > > > > >> > > > >> > > >>>> should
> > > > > > >> > > > >> > > >>>> >> >> >> respond to the produce request
> > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > >> > > > >> > > >>>> >> >> >> > I think this section would be
> > clearer
> > > > if
> > > > > we
> > > > > > >> > talked
> > > > > > >> > > > >> about
> > > > > > >> > > > >> > > the
> > > > > > >> > > > >> > > >>>> new
> > > > > > >> > > > >> > > >>>> >> high
> > > > > > >> > > > >> > > >>>> >> >> >> > watermark advancement requirement
> > > > first,
> > > > > > and
> > > > > > >> > THEN
> > > > > > >> > > > >> talked
> > > > > > >> > > > >> > > >>>> about its
> > > > > > >> > > > >> > > >>>> >> >> >> > impact on acks=0, acks=1, and
> > > > >  acks=all.
> > > > > > >> > > > acks=all
> > > > > > >> > > > >> is
> > > > > > >> > > > >> > of
> > > > > > >> > > > >> > > >>>> course
> > > > > > >> > > > >> > > >>>> >> the
> > > > > > >> > > > >> > > >>>> >> >> >> > main case we care about here, so
> it
> > > > would
> > > > > > be
> > > > > > >> > good
> > > > > > >> > > to
> > > > > > >> > > > >> lead
> > > > > > >> > > > >> > > with
> > > > > > >> > > > >> > > >>>> >> that,
> > > > > > >> > > > >> > > >>>> >> >> >> > rather than delving into the
> > > > > technicalities
> > > > > > >> of
> > > > > > >> > > > acks=0/1
> > > > > > >> > > > >> > > first.
> > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > >> > > > >> > > >>>> >> >> >> >> Unclean recovery
> > > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > > >> > > > >> > > >>>> >> >> >> > So, here you are introducing a
> new
> > > > > > >> > configuration,
> > > > > > >> > > > >> > > >>>> >> >> >> > unclean.recovery.strategy. The
> > > > difficult
> > > > > > >> thing
> > > > > > >> > > here
> > > > > > >> > > > is
> > > > > > >> > > > >> > that
> > > > > > >> > > > >> > > >>>> there
> > > > > > >> > > > >> > > >>>> >> is a
> > > > > > >> > > > >> > > >>>> >> >> >> > lot of overlap with
> > > > > > >> > > unclean.leader.election.enable.
> > > > > > >> > > > So
> > > > > > >> > > > >> we
> > > > > > >> > > > >> > > >>>> have 3
> > > > > > >> > > > >> > > >>>> >> >> >> > different settings for
> > > > > > >> > unclean.recovery.strategy,
> > > > > > >> > > > plus
> > > > > > >> > > > >> 2
> > > > > > >> > > > >> > > >>>> different
> > > > > > >> > > > >> > > >>>> >> >> >> > settings for
> > > > > > unclean.leader.election.enable,
> > > > > > >> > > giving
> > > > > > >> > > > a
> > > > > > >> > > > >> > cross
> > > > > > >> > > > >> > > >>>> >> product of
> > > > > > >> > > > >> > > >>>> >> >> >> > 6 different options. The
> following
> > > > > "unclean
> > > > > > >> > > recovery
> > > > > > >> > > > >> > > manager"
> > > > > > >> > > > >> > > >>>> >> section
> > > > > > >> > > > >> > > >>>> >> >> >> > on
> > >
> >
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

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

A few more minor comments on your latest update.

60. DescribeTopicRequest: When will the Partitions field be used? It seems
that the FirstPartitionId field is enough for AdminClient usage.

61. Could we make the limit for DescribeTopicRequest, ElectLeadersRequest,
GetReplicaLogInfo consistent? Currently, ElectLeadersRequest's limit is at
topic level and GetReplicaLogInfo has a different partition level limit
from DescribeTopicRequest.

62. Should ElectLeadersRequest.DesiredLeaders be at the same level as
ElectLeadersRequest.TopicPartitions.Partitions? In the KIP, it looks like
it's at the same level as ElectLeadersRequest.TopicPartitions.

Thanks,

Jun

On Wed, Oct 4, 2023 at 3:55 PM Calvin Liu <ca...@confluent.io.invalid>
wrote:

> Hi David,
> Thanks for the comments.
> ----
> I thought that a new snapshot with the downgraded MV is created in this
> case. Isn’t it the case?
> Yes, you are right, a metadata delta will be generated after the MV
> downgrade. Then the user can start the software downgrade.
> -----
> Could you also elaborate a bit more on the reasoning behind adding the
> limits to the admin RPCs? This is a new pattern in Kafka so it would be
> good to clear on the motivation.
> Thanks to Colin for bringing it up. The current MetadataRequest does not
> have a limit on the number of topics to query in a single request. Massive
> requests can mess up the JVM. We want to have some sort of throttle on the
> new APIs.
> -----
> Could you also explain how the client is supposed to handle the
> topics/partitions above the limit? I suppose that it will have to retry
> those, correct?
> Corrent. For the official admin clients, it will split the large request
> into proper pieces and query one after another.
> -----
> My understanding is that the topics/partitions above the limit will be
> failed with an invalid exception error. I wonder if this choice is
> judicious because the invalide request exception is usually fatal. It may
> be better to use an new and explicit error for this case.
>
> Thanks for bringing this up. How about "REQUEST_LIMIT_REACHED"?
> --------
> It seems that we still need to specify the changes to the admin api to
> accommodate the new or updated apis. Do you plan to add them?
> Try to cover the following
> 1. The admin client will use the new DescribeTopicRequest to query the
> topics
> 2. Mention the API limit and the new retriable error.
> 3. Output changes for the admin client when describing a topic (new fields
> of ELR...)
> 4. Changes to data structures like TopicPartitionInfo to include the ELR.
> Anything else I missed?
>
> Thanks!
>
>
>
>
>
> On Wed, Oct 4, 2023 at 12:27 PM David Jacot <da...@gmail.com> wrote:
>
> > Hi Calvin,
> >
> > I thought that a new snapshot with the downgraded MV is created in this
> > case. Isn’t it the case?
> >
> > Could you also elaborate a bit more on the reasoning behind adding the
> > limits to the admin RPCs? This is a new pattern in Kafka so it would be
> > good to clear on the motivation.
> >
> > Could you also explain how the client is supposed to handle the
> > topics/partitions above the limit? I suppose that it will have to retry
> > those, correct?
> >
> > My understanding is that the topics/partitions above the limit will be
> > failed with an invalid exception error. I wonder if this choice is
> > judicious because the invalide request exception is usually fatal. It may
> > be better to use an new and explicit error for this case.
> >
> > It seems that we still need to specify the changes to the admin api to
> > accommodate the new or updated apis. Do you plan to add them?
> >
> > Best,
> > David
> >
> > Le mer. 4 oct. 2023 à 20:39, Calvin Liu <ca...@confluent.io.invalid> a
> > écrit :
> >
> > > Hi Jun,
> > > After the MV downgrade, the controller will write in the old version of
> > the
> > > PartitionRecord/PartitionChangeRecord. If I understand correctly, it is
> > > possible to downgrade the software version if the controller only has
> to
> > > handle old version records.
> > > However, the controller will not automatically rewrite the
> > PartitionRecord
> > > with the old version unless there is a partition update. Then, the user
> > may
> > > have to wait an unknown amount of time before the software downgrades
> > > unless they do a roll to force update every partition. If it makes
> > sense, I
> > > can mention these steps to do a software downgrade.
> > > Thanks
> > >
> > > On Wed, Oct 4, 2023 at 11:20 AM Jun Rao <ju...@confluent.io.invalid>
> > wrote:
> > >
> > > > Hi, Calvin and Justine,
> > > >
> > > > Historically, when we change the record format in the log, we don't
> > > support
> > > > software version downgrading.
> > > >
> > > > For the record format change in the metadata log, have we thought
> about
> > > > forcing the write of the latest metadata records with the old version
> > > > during MV downgrading? This will in theory allow the old version of
> the
> > > > software to obtain the latest metadata.
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > > On Wed, Oct 4, 2023 at 9:53 AM Justine Olshan
> > > <jolshan@confluent.io.invalid
> > > > >
> > > > wrote:
> > > >
> > > > > Sorry -- not MV but software version.
> > > > >
> > > > > On Wed, Oct 4, 2023 at 9:51 AM Justine Olshan <
> jolshan@confluent.io>
> > > > > wrote:
> > > > >
> > > > > > Catching up with this discussion.
> > > > > >
> > > > > > I was just curious -- have we had other instances where
> downgrading
> > > MV
> > > > is
> > > > > > not supported? I think Kafka typically tries to support
> downgrades,
> > > > and I
> > > > > > couldn't think of other examples.
> > > > > >
> > > > > > Thanks,
> > > > > > Justine
> > > > > >
> > > > > > On Wed, Oct 4, 2023 at 9:40 AM Calvin Liu
> > <caliu@confluent.io.invalid
> > > >
> > > > > > wrote:
> > > > > >
> > > > > >> Hi Jun,
> > > > > >> 54. Marked the software downgrading is not supported. As the old
> > > > > >> controller
> > > > > >> will not understand the new PartitionRecord and
> > > PartitionChangeRecord.
> > > > > >> Thanks!
> > > > > >>
> > > > > >> On Wed, Oct 4, 2023 at 9:12 AM Jun Rao <jun@confluent.io.invalid
> >
> > > > > wrote:
> > > > > >>
> > > > > >> > Hi, Calvin,
> > > > > >> >
> > > > > >> > Thanks for the reply. Just one more comment.
> > > > > >> >
> > > > > >> > 54. It seems that downgrading MV is supported. Is downgrading
> > the
> > > > > >> software
> > > > > >> > version supported? It would be useful to document that.
> > > > > >> >
> > > > > >> > Thanks,
> > > > > >> >
> > > > > >> > Jun
> > > > > >> >
> > > > > >> > On Tue, Oct 3, 2023 at 4:55 PM Artem Livshits
> > > > > >> > <al...@confluent.io.invalid> wrote:
> > > > > >> >
> > > > > >> > > Hi Colin,
> > > > > >> > >
> > > > > >> > > I think in your example "do_unclean_recovery" would need to
> do
> > > > > >> different
> > > > > >> > > things depending on the strategy.
> > > > > >> > >
> > > > > >> > > do_unclean_recovery() {
> > > > > >> > >    if (unclean.recovery.manager.enabled) {
> > > > > >> > >     if (strategy == Aggressive)
> > > > > >> > >       use UncleanRecoveryManager(waitLastKnownERL=false)  //
> > > just
> > > > > >> inspect
> > > > > >> > > logs from whoever is available
> > > > > >> > >     else
> > > > > >> > >       use  UncleanRecoveryManager(waitLastKnownERL=true)  //
> > > must
> > > > > wait
> > > > > >> > for
> > > > > >> > > at least last known ELR
> > > > > >> > >   } else {
> > > > > >> > >     if (strategy == Aggressive)
> > > > > >> > >       choose the last known leader if that is available, or
> a
> > > > random
> > > > > >> > leader
> > > > > >> > > if not)
> > > > > >> > >     else
> > > > > >> > >       wait for last known leader to get back
> > > > > >> > >   }
> > > > > >> > > }
> > > > > >> > >
> > > > > >> > > The idea is that the Aggressive strategy would kick in as
> soon
> > > as
> > > > we
> > > > > >> lost
> > > > > >> > > the leader and would pick a leader from whoever is
> available;
> > > but
> > > > > the
> > > > > >> > > Balanced will only kick in when ELR is empty and will wait
> for
> > > the
> > > > > >> > brokers
> > > > > >> > > that likely have most data to be available.
> > > > > >> > >
> > > > > >> > > On Tue, Oct 3, 2023 at 3:04 PM Colin McCabe <
> > cmccabe@apache.org
> > > >
> > > > > >> wrote:
> > > > > >> > >
> > > > > >> > > > On Tue, Oct 3, 2023, at 10:49, Jun Rao wrote:
> > > > > >> > > > > Hi, Calvin,
> > > > > >> > > > >
> > > > > >> > > > > Thanks for the update KIP. A few more comments.
> > > > > >> > > > >
> > > > > >> > > > > 41. Why would a user choose the option to select a
> random
> > > > > replica
> > > > > >> as
> > > > > >> > > the
> > > > > >> > > > > leader instead of using
> > unclean.recovery.strateg=Aggressive?
> > > > It
> > > > > >> seems
> > > > > >> > > > that
> > > > > >> > > > > the latter is strictly better? If that's not the case,
> > could
> > > > we
> > > > > >> fold
> > > > > >> > > this
> > > > > >> > > > > option under unclean.recovery.strategy instead of
> > > introducing
> > > > a
> > > > > >> > > separate
> > > > > >> > > > > config?
> > > > > >> > > >
> > > > > >> > > > Hi Jun,
> > > > > >> > > >
> > > > > >> > > > I thought the flow of control was:
> > > > > >> > > >
> > > > > >> > > > If there is no leader for the partition {
> > > > > >> > > >   If (there are unfenced ELR members) {
> > > > > >> > > >     choose_an_unfenced_ELR_member
> > > > > >> > > >   } else if (there are fenced ELR members AND
> > > > > strategy=Aggressive) {
> > > > > >> > > >     do_unclean_recovery
> > > > > >> > > >   } else if (there are no ELR members AND strategy !=
> None)
> > {
> > > > > >> > > >     do_unclean_recovery
> > > > > >> > > >   } else {
> > > > > >> > > >     do nothing about the missing leader
> > > > > >> > > >   }
> > > > > >> > > > }
> > > > > >> > > >
> > > > > >> > > > do_unclean_recovery() {
> > > > > >> > > >    if (unclean.recovery.manager.enabled) {
> > > > > >> > > >     use UncleanRecoveryManager
> > > > > >> > > >   } else {
> > > > > >> > > >     choose the last known leader if that is available, or
> a
> > > > random
> > > > > >> > leader
> > > > > >> > > > if not)
> > > > > >> > > >   }
> > > > > >> > > > }
> > > > > >> > > >
> > > > > >> > > > However, I think this could be clarified, especially the
> > > > behavior
> > > > > >> when
> > > > > >> > > > unclean.recovery.manager.enabled=false. Inuitively the
> goal
> > > for
> > > > > >> > > > unclean.recovery.manager.enabled=false is to be "the same
> as
> > > > now,
> > > > > >> > mostly"
> > > > > >> > > > but it's very underspecified in the KIP, I agree.
> > > > > >> > > >
> > > > > >> > > > >
> > > > > >> > > > > 50. ElectLeadersRequest: "If more than 20 topics are
> > > included,
> > > > > >> only
> > > > > >> > the
> > > > > >> > > > > first 20 will be served. Others will be returned with
> > > > > >> > DesiredLeaders."
> > > > > >> > > > Hmm,
> > > > > >> > > > > not sure that I understand this. ElectLeadersResponse
> > > doesn't
> > > > > >> have a
> > > > > >> > > > > DesiredLeaders field.
> > > > > >> > > > >
> > > > > >> > > > > 51. GetReplicaLogInfo: "If more than 2000 partitions are
> > > > > included,
> > > > > >> > only
> > > > > >> > > > the
> > > > > >> > > > > first 2000 will be served" Do we return an error for the
> > > > > remaining
> > > > > >> > > > > partitions? Actually, should we include an errorCode
> field
> > > at
> > > > > the
> > > > > >> > > > partition
> > > > > >> > > > > level in GetReplicaLogInfoResponse to cover non-existing
> > > > > >> partitions
> > > > > >> > and
> > > > > >> > > > no
> > > > > >> > > > > authorization, etc?
> > > > > >> > > > >
> > > > > >> > > > > 52. The entry should matches => The entry should match
> > > > > >> > > > >
> > > > > >> > > > > 53. ElectLeadersRequest.DesiredLeaders: Should it be
> > > nullable
> > > > > >> since a
> > > > > >> > > > user
> > > > > >> > > > > may not specify DesiredLeaders?
> > > > > >> > > > >
> > > > > >> > > > > 54. Downgrade: Is that indeed possible? I thought
> earlier
> > > you
> > > > > said
> > > > > >> > that
> > > > > >> > > > > once the new version of the records are in the metadata
> > log,
> > > > one
> > > > > >> > can't
> > > > > >> > > > > downgrade since the old broker doesn't know how to parse
> > the
> > > > new
> > > > > >> > > version
> > > > > >> > > > of
> > > > > >> > > > > the metadata records?
> > > > > >> > > > >
> > > > > >> > > >
> > > > > >> > > > MetadataVersion downgrade is currently broken but we have
> > > fixing
> > > > > it
> > > > > >> on
> > > > > >> > > our
> > > > > >> > > > plate for Kafka 3.7.
> > > > > >> > > >
> > > > > >> > > > The way downgrade works is that "new features" are
> dropped,
> > > > > leaving
> > > > > >> > only
> > > > > >> > > > the old ones.
> > > > > >> > > >
> > > > > >> > > > > 55. CleanShutdownFile: Should we add a version field for
> > > > future
> > > > > >> > > > extension?
> > > > > >> > > > >
> > > > > >> > > > > 56. Config changes are public facing. Could we have a
> > > separate
> > > > > >> > section
> > > > > >> > > to
> > > > > >> > > > > document all the config changes?
> > > > > >> > > >
> > > > > >> > > > +1. A separate section for this would be good.
> > > > > >> > > >
> > > > > >> > > > best,
> > > > > >> > > > Colin
> > > > > >> > > >
> > > > > >> > > > >
> > > > > >> > > > > Thanks,
> > > > > >> > > > >
> > > > > >> > > > > Jun
> > > > > >> > > > >
> > > > > >> > > > > On Mon, Sep 25, 2023 at 4:29 PM Calvin Liu
> > > > > >> > <caliu@confluent.io.invalid
> > > > > >> > > >
> > > > > >> > > > > wrote:
> > > > > >> > > > >
> > > > > >> > > > >> Hi Jun
> > > > > >> > > > >> Thanks for the comments.
> > > > > >> > > > >>
> > > > > >> > > > >> 40. If we change to None, it is not guaranteed for no
> > data
> > > > > loss.
> > > > > >> For
> > > > > >> > > > users
> > > > > >> > > > >> who are not able to validate the data with external
> > > > resources,
> > > > > >> > manual
> > > > > >> > > > >> intervention does not give a better result but a loss
> of
> > > > > >> > availability.
> > > > > >> > > > So
> > > > > >> > > > >> practically speaking, the Balance mode would be a
> better
> > > > > default
> > > > > >> > > value.
> > > > > >> > > > >>
> > > > > >> > > > >> 41. No, it represents how we want to do the unclean
> > leader
> > > > > >> election.
> > > > > >> > > If
> > > > > >> > > > it
> > > > > >> > > > >> is false, the unclean leader election will be the old
> > > random
> > > > > way.
> > > > > >> > > > >> Otherwise, the unclean recovery will be used.
> > > > > >> > > > >>
> > > > > >> > > > >> 42. Good catch. Updated.
> > > > > >> > > > >>
> > > > > >> > > > >> 43. Only the first 20 topics will be served. Others
> will
> > be
> > > > > >> returned
> > > > > >> > > > with
> > > > > >> > > > >> InvalidRequestError
> > > > > >> > > > >>
> > > > > >> > > > >> 44. The order matters. The desired leader entries match
> > > with
> > > > > the
> > > > > >> > topic
> > > > > >> > > > >> partition list by the index.
> > > > > >> > > > >>
> > > > > >> > > > >> 45. Thanks! Updated.
> > > > > >> > > > >>
> > > > > >> > > > >> 46. Good advice! Updated.
> > > > > >> > > > >>
> > > > > >> > > > >> 47.1, updated the comment. Basically it will elect the
> > > > replica
> > > > > in
> > > > > >> > the
> > > > > >> > > > >> desiredLeader field to be the leader
> > > > > >> > > > >>
> > > > > >> > > > >> 47.2 We can let the admin client do the conversion.
> Using
> > > the
> > > > > >> > > > desiredLeader
> > > > > >> > > > >> field in the json format seems easier for users.
> > > > > >> > > > >>
> > > > > >> > > > >> 48. Once the MV version is downgraded, all the ELR
> > related
> > > > > fields
> > > > > >> > will
> > > > > >> > > > be
> > > > > >> > > > >> removed on the next partition change. The controller
> will
> > > > also
> > > > > >> > ignore
> > > > > >> > > > the
> > > > > >> > > > >> ELR fields. Updated the KIP.
> > > > > >> > > > >>
> > > > > >> > > > >> 49. Yes, it would be deprecated/removed.
> > > > > >> > > > >>
> > > > > >> > > > >>
> > > > > >> > > > >> On Mon, Sep 25, 2023 at 3:49 PM Jun Rao
> > > > > <jun@confluent.io.invalid
> > > > > >> >
> > > > > >> > > > wrote:
> > > > > >> > > > >>
> > > > > >> > > > >> > Hi, Calvin,
> > > > > >> > > > >> >
> > > > > >> > > > >> > Thanks for the updated KIP. Made another pass. A few
> > more
> > > > > >> comments
> > > > > >> > > > below.
> > > > > >> > > > >> >
> > > > > >> > > > >> > 40. unclean.leader.election.enable.false ->
> > > > > >> > > > >> > unclean.recovery.strategy.Balanced: The Balanced mode
> > > could
> > > > > >> still
> > > > > >> > > > lead to
> > > > > >> > > > >> > data loss. So, I am wondering if
> > > > > >> > > unclean.leader.election.enable.false
> > > > > >> > > > >> > should map to None?
> > > > > >> > > > >> >
> > > > > >> > > > >> > 41. unclean.recovery.manager.enabled: I am not sure
> why
> > > we
> > > > > >> > introduce
> > > > > >> > > > this
> > > > > >> > > > >> > additional config. Is it the same as
> > > > > >> > unclean.recovery.strategy=None?
> > > > > >> > > > >> >
> > > > > >> > > > >> > 42. DescribeTopicResponse.TopicAuthorizedOperations:
> > > Should
> > > > > >> this
> > > > > >> > be
> > > > > >> > > at
> > > > > >> > > > >> the
> > > > > >> > > > >> > topic level?
> > > > > >> > > > >> >
> > > > > >> > > > >> > 43. "Limit: 20 topics max per request": Could we
> > describe
> > > > > what
> > > > > >> > > > happens if
> > > > > >> > > > >> > the request includes more than 20 topics?
> > > > > >> > > > >> >
> > > > > >> > > > >> > 44. ElectLeadersRequest.DesiredLeaders: Could we
> > describe
> > > > > >> whether
> > > > > >> > > the
> > > > > >> > > > >> > ordering matters?
> > > > > >> > > > >> >
> > > > > >> > > > >> > 45. GetReplicaLogInfo.TopicPartitions: "about": "The
> > > topic
> > > > > >> > > partitions
> > > > > >> > > > to
> > > > > >> > > > >> > elect leaders.": The description in "about" is
> > incorrect.
> > > > > >> > > > >> >
> > > > > >> > > > >> > 46. GetReplicaLogInfoResponse: Should we nest
> > partitions
> > > > > under
> > > > > >> > > > topicId to
> > > > > >> > > > >> > be consistent with other types of responses?
> > > > > >> > > > >> >
> > > > > >> > > > >> > 47. kafka-leader-election.sh:
> > > > > >> > > > >> > 47.1 Could we explain DESIGNATION?
> > > > > >> > > > >> > 47.2 desiredLeader: Should it be a list to match the
> > > field
> > > > in
> > > > > >> > > > >> > ElectLeadersRequest?
> > > > > >> > > > >> >
> > > > > >> > > > >> > 48. We could add a section on downgrade?
> > > > > >> > > > >> >
> > > > > >> > > > >> > 49. LastKnownLeader: This seems only needed in the
> > first
> > > > > phase
> > > > > >> of
> > > > > >> > > > >> > delivering ELR. Will it be removed when the complete
> > KIP
> > > is
> > > > > >> > > delivered?
> > > > > >> > > > >> >
> > > > > >> > > > >> > Thanks,
> > > > > >> > > > >> >
> > > > > >> > > > >> > Jun
> > > > > >> > > > >> >
> > > > > >> > > > >> > On Tue, Sep 19, 2023 at 1:30 PM Colin McCabe <
> > > > > >> cmccabe@apache.org>
> > > > > >> > > > wrote:
> > > > > >> > > > >> >
> > > > > >> > > > >> > > Hi Calvin,
> > > > > >> > > > >> > >
> > > > > >> > > > >> > > Thanks for the explanations. I like the idea of
> using
> > > > none,
> > > > > >> > > > balanced,
> > > > > >> > > > >> > > aggressive. We also had an offline discussion about
> > why
> > > > it
> > > > > is
> > > > > >> > good
> > > > > >> > > > to
> > > > > >> > > > >> > use a
> > > > > >> > > > >> > > new config key (basically, so that we can deprecate
> > the
> > > > old
> > > > > >> one
> > > > > >> > > > which
> > > > > >> > > > >> had
> > > > > >> > > > >> > > only false/true values in 4.0) With these changes,
> I
> > am
> > > > +1.
> > > > > >> > > > >> > >
> > > > > >> > > > >> > > best,
> > > > > >> > > > >> > > Colin
> > > > > >> > > > >> > >
> > > > > >> > > > >> > > On Mon, Sep 18, 2023, at 15:54, Calvin Liu wrote:
> > > > > >> > > > >> > > > Hi Colin,
> > > > > >> > > > >> > > > Also, can we deprecate
> > unclean.leader.election.enable
> > > > in
> > > > > >> 4.0?
> > > > > >> > > > Before
> > > > > >> > > > >> > > that,
> > > > > >> > > > >> > > > we can have both the config
> > unclean.recovery.strategy
> > > > and
> > > > > >> > > > >> > > > unclean.leader.election.enable
> > > > > >> > > > >> > > > and using the unclean.recovery.Enabled to
> determine
> > > > which
> > > > > >> > config
> > > > > >> > > > to
> > > > > >> > > > >> use
> > > > > >> > > > >> > > > during the unclean leader election.
> > > > > >> > > > >> > > >
> > > > > >> > > > >> > > > On Mon, Sep 18, 2023 at 3:51 PM Calvin Liu <
> > > > > >> > caliu@confluent.io>
> > > > > >> > > > >> wrote:
> > > > > >> > > > >> > > >
> > > > > >> > > > >> > > >> Hi Colin,
> > > > > >> > > > >> > > >> For the unclean.recovery.strategy config name,
> how
> > > > about
> > > > > >> we
> > > > > >> > use
> > > > > >> > > > the
> > > > > >> > > > >> > > >> following
> > > > > >> > > > >> > > >> None. It basically means no unclean recovery
> will
> > be
> > > > > >> > performed.
> > > > > >> > > > >> > > >> Aggressive. It means availability goes first.
> > > Whenever
> > > > > the
> > > > > >> > > > partition
> > > > > >> > > > >> > > can't
> > > > > >> > > > >> > > >> elect a durable replica, the controller will try
> > the
> > > > > >> unclean
> > > > > >> > > > >> recovery.
> > > > > >> > > > >> > > >> Balanced. It is the balance point of the
> > > availability
> > > > > >> > > > >> > first(Aggressive)
> > > > > >> > > > >> > > >> and least availability(None). The controller
> > > performs
> > > > > >> unclean
> > > > > >> > > > >> recovery
> > > > > >> > > > >> > > when
> > > > > >> > > > >> > > >> both ISR and ELR are empty.
> > > > > >> > > > >> > > >>
> > > > > >> > > > >> > > >>
> > > > > >> > > > >> > > >> On Fri, Sep 15, 2023 at 11:42 AM Calvin Liu <
> > > > > >> > > caliu@confluent.io>
> > > > > >> > > > >> > wrote:
> > > > > >> > > > >> > > >>
> > > > > >> > > > >> > > >>> Hi Colin,
> > > > > >> > > > >> > > >>>
> > > > > >> > > > >> > > >>> > So, the proposal is that if someone sets
> > > > > >> > > > >> > > "unclean.leader.election.enable
> > > > > >> > > > >> > > >>> = true"...
> > > > > >> > > > >> > > >>>
> > > > > >> > > > >> > > >>>
> > > > > >> > > > >> > > >>> The idea is to use one of the
> > > > > >> unclean.leader.election.enable
> > > > > >> > > and
> > > > > >> > > > >> > > >>> unclean.recovery.strategy based on the
> > > > > >> > > > unclean.recovery.Enabled. A
> > > > > >> > > > >> > > possible
> > > > > >> > > > >> > > >>> version can be
> > > > > >> > > > >> > > >>>
> > > > > >> > > > >> > > >>> If unclean.recovery.Enabled:
> > > > > >> > > > >> > > >>>
> > > > > >> > > > >> > > >>> {
> > > > > >> > > > >> > > >>>
> > > > > >> > > > >> > > >>> Check unclean.recovery.strategy. If set, use
> it.
> > > > > >> Otherwise,
> > > > > >> > > > check
> > > > > >> > > > >> > > >>> unclean.leader.election.enable and translate it
> > to
> > > > > >> > > > >> > > >>> unclean.recovery.strategy.
> > > > > >> > > > >> > > >>>
> > > > > >> > > > >> > > >>> } else {
> > > > > >> > > > >> > > >>>
> > > > > >> > > > >> > > >>> Use unclean.leader.election.enable
> > > > > >> > > > >> > > >>>
> > > > > >> > > > >> > > >>> }
> > > > > >> > > > >> > > >>>
> > > > > >> > > > >> > > >>>
> > > > > >> > > > >> > > >>> —--------
> > > > > >> > > > >> > > >>>
> > > > > >> > > > >> > > >>> >The configuration key should be
> > > > > >> > > > >> "unclean.recovery.manager.enabled",
> > > > > >> > > > >> > > >>> right?
> > > > > >> > > > >> > > >>>
> > > > > >> > > > >> > > >>>
> > > > > >> > > > >> > > >>> I think we have two ways of choosing a leader
> > > > > uncleanly,
> > > > > >> > > unclean
> > > > > >> > > > >> > leader
> > > > > >> > > > >> > > >>> election and unclean recovery(log inspection)
> and
> > > we
> > > > > try
> > > > > >> to
> > > > > >> > > > switch
> > > > > >> > > > >> > > between
> > > > > >> > > > >> > > >>> them.
> > > > > >> > > > >> > > >>>
> > > > > >> > > > >> > > >>> Do you mean we want to develop two ways of
> > > performing
> > > > > the
> > > > > >> > > > unclean
> > > > > >> > > > >> > > >>> recovery and one of them is using “unclean
> > recovery
> > > > > >> > manager”?
> > > > > >> > > I
> > > > > >> > > > >> guess
> > > > > >> > > > >> > > we
> > > > > >> > > > >> > > >>> haven’t discussed the second way.
> > > > > >> > > > >> > > >>>
> > > > > >> > > > >> > > >>>
> > > > > >> > > > >> > > >>> —-------
> > > > > >> > > > >> > > >>>
> > > > > >> > > > >> > > >>> >How do these 4 levels of overrides interact
> with
> > > > your
> > > > > >> new
> > > > > >> > > > >> > > >>> configurations?
> > > > > >> > > > >> > > >>>
> > > > > >> > > > >> > > >>>
> > > > > >> > > > >> > > >>> I do notice in the Kraft controller code, the
> > > method
> > > > to
> > > > > >> > check
> > > > > >> > > > >> whether
> > > > > >> > > > >> > > >>> perform unclean leader election is hard coded
> to
> > > > false
> > > > > >> since
> > > > > >> > > > >> > > >>> 2021(uncleanLeaderElectionEnabledForTopic).
> Isn’t
> > > it
> > > > a
> > > > > >> good
> > > > > >> > > > chance
> > > > > >> > > > >> to
> > > > > >> > > > >> > > >>> completely deprecate the
> > > > > unclean.leader.election.enable?
> > > > > >> We
> > > > > >> > > > don’t
> > > > > >> > > > >> > even
> > > > > >> > > > >> > > have
> > > > > >> > > > >> > > >>> to worry about the config conversion.
> > > > > >> > > > >> > > >>>
> > > > > >> > > > >> > > >>> On the other hand, whatever the override is, as
> > > long
> > > > as
> > > > > >> the
> > > > > >> > > > >> > controller
> > > > > >> > > > >> > > >>> can have the final effective
> > > > > >> unclean.leader.election.enable,
> > > > > >> > > the
> > > > > >> > > > >> > topic
> > > > > >> > > > >> > > >>> level config unclean.recovery.strategy, the
> > cluster
> > > > > level
> > > > > >> > > config
> > > > > >> > > > >> > > >>> unclean.recovery.Enabled, the controller can
> > > > calculate
> > > > > >> the
> > > > > >> > > > correct
> > > > > >> > > > >> > > methods
> > > > > >> > > > >> > > >>> to use right?
> > > > > >> > > > >> > > >>>
> > > > > >> > > > >> > > >>>
> > > > > >> > > > >> > > >>> On Fri, Sep 15, 2023 at 10:02 AM Colin McCabe <
> > > > > >> > > > cmccabe@apache.org>
> > > > > >> > > > >> > > wrote:
> > > > > >> > > > >> > > >>>
> > > > > >> > > > >> > > >>>> On Thu, Sep 14, 2023, at 22:23, Calvin Liu
> > wrote:
> > > > > >> > > > >> > > >>>> > Hi Colin
> > > > > >> > > > >> > > >>>> > 1. I think using the new config name is more
> > > > clear.
> > > > > >> > > > >> > > >>>> >        a. The unclean leader election is
> > > actually
> > > > > >> removed
> > > > > >> > > if
> > > > > >> > > > >> > unclean
> > > > > >> > > > >> > > >>>> > recovery is in use.
> > > > > >> > > > >> > > >>>> >        b. Using multiple values in
> > > > > >> > > > >> unclean.leader.election.enable
> > > > > >> > > > >> > is
> > > > > >> > > > >> > > >>>> > confusing and it will be more confusing
> after
> > > > people
> > > > > >> > forget
> > > > > >> > > > >> about
> > > > > >> > > > >> > > this
> > > > > >> > > > >> > > >>>> > discussion.
> > > > > >> > > > >> > > >>>>
> > > > > >> > > > >> > > >>>> Hi Calvin,
> > > > > >> > > > >> > > >>>>
> > > > > >> > > > >> > > >>>> So, the proposal is that if someone sets
> > > > > >> > > > >> > > "unclean.leader.election.enable
> > > > > >> > > > >> > > >>>> = true" but then sets one of your new
> > > > configurations,
> > > > > >> the
> > > > > >> > > > value of
> > > > > >> > > > >> > > >>>> unclean.leader.election.enable is ignored?
> That
> > > > seems
> > > > > >> less
> > > > > >> > > > clear
> > > > > >> > > > >> to
> > > > > >> > > > >> > > me, not
> > > > > >> > > > >> > > >>>> more. Just in general, having multiple
> > > configuration
> > > > > >> keys
> > > > > >> > to
> > > > > >> > > > >> control
> > > > > >> > > > >> > > the
> > > > > >> > > > >> > > >>>> same thing confuses users. Basically, they are
> > > > sitting
> > > > > >> at a
> > > > > >> > > > giant
> > > > > >> > > > >> > > control
> > > > > >> > > > >> > > >>>> panel, and some of the levers do nothing.
> > > > > >> > > > >> > > >>>>
> > > > > >> > > > >> > > >>>> > 2. Sorry I forgot to mention in the response
> > > that
> > > > I
> > > > > >> did
> > > > > >> > add
> > > > > >> > > > the
> > > > > >> > > > >> > > >>>> > unclean.recovery.Enabled flag.
> > > > > >> > > > >> > > >>>>
> > > > > >> > > > >> > > >>>> The configuration key should be
> > > > > >> > > > >> "unclean.recovery.manager.enabled",
> > > > > >> > > > >> > > >>>> right? Becuase we can do "unclean recovery"
> > > without
> > > > > the
> > > > > >> > > > manager.
> > > > > >> > > > >> > > Disabling
> > > > > >> > > > >> > > >>>> the manager just means we use a different
> > > mechanism
> > > > > for
> > > > > >> > > > recovery.
> > > > > >> > > > >> > > >>>>
> > > > > >> > > > >> > > >>>> >        c. Maybe I underestimated the
> challenge
> > > of
> > > > > >> > replacing
> > > > > >> > > > the
> > > > > >> > > > >> > > >>>> config. Any
> > > > > >> > > > >> > > >>>> > implementation problems ahead?
> > > > > >> > > > >> > > >>>>
> > > > > >> > > > >> > > >>>> There are four levels of overrides for
> > > > > >> > > > >> > unclean.leader.election.enable.
> > > > > >> > > > >> > > >>>>
> > > > > >> > > > >> > > >>>> 1. static configuration for node.
> > > > > >> > > > >> > > >>>>     This goes in the configuration file,
> > typically
> > > > > named
> > > > > >> > > > >> > > >>>> server.properties
> > > > > >> > > > >> > > >>>>
> > > > > >> > > > >> > > >>>> 2. dynamic configuration for node default
> > > > > >> > > > >> > > >>>>   ConfigResource(type=BROKER, name="")
> > > > > >> > > > >> > > >>>>
> > > > > >> > > > >> > > >>>> 3. dynamic configuration for node
> > > > > >> > > > >> > > >>>>   ConfigResource(type=BROKER, name=<controller
> > > id>)
> > > > > >> > > > >> > > >>>>
> > > > > >> > > > >> > > >>>> 4. dynamic configuration for topic
> > > > > >> > > > >> > > >>>>   ConfigResource(type=TOPIC,
> name=<topic-name>)
> > > > > >> > > > >> > > >>>>
> > > > > >> > > > >> > > >>>> How do these 4 levels of overrides interact
> with
> > > > your
> > > > > >> new
> > > > > >> > > > >> > > >>>> configurations? If the new configurations
> > dominate
> > > > > over
> > > > > >> the
> > > > > >> > > old
> > > > > >> > > > >> > ones,
> > > > > >> > > > >> > > it
> > > > > >> > > > >> > > >>>> seems like this will get a lot more confusing
> to
> > > > > >> implement
> > > > > >> > > (and
> > > > > >> > > > >> also
> > > > > >> > > > >> > > to
> > > > > >> > > > >> > > >>>> use.)
> > > > > >> > > > >> > > >>>>
> > > > > >> > > > >> > > >>>> Again, I'd recommend just adding some new
> values
> > > to
> > > > > >> > > > >> > > >>>> unclean.leader.election.enable. It's simple
> and
> > > will
> > > > > >> > prevent
> > > > > >> > > > user
> > > > > >> > > > >> > > confusion
> > > > > >> > > > >> > > >>>> (as well as developer confusion.)
> > > > > >> > > > >> > > >>>>
> > > > > >> > > > >> > > >>>> best,
> > > > > >> > > > >> > > >>>> Colin
> > > > > >> > > > >> > > >>>>
> > > > > >> > > > >> > > >>>>
> > > > > >> > > > >> > > >>>> > 3. About the admin client, I mentioned 3
> > changes
> > > > in
> > > > > >> the
> > > > > >> > > > client.
> > > > > >> > > > >> > > >>>> Anything
> > > > > >> > > > >> > > >>>> > else I missed in the KIP?
> > > > > >> > > > >> > > >>>> >       a. The client will switch to using the
> > new
> > > > RPC
> > > > > >> > > instead
> > > > > >> > > > of
> > > > > >> > > > >> > > >>>> > MetadataRequest for the topics.
> > > > > >> > > > >> > > >>>> >       b. The TopicPartitionInfo used in
> > > > > >> TopicDescription
> > > > > >> > > > needs
> > > > > >> > > > >> to
> > > > > >> > > > >> > > add
> > > > > >> > > > >> > > >>>> new
> > > > > >> > > > >> > > >>>> > fields related to the ELR.
> > > > > >> > > > >> > > >>>> >       c. The outputs will add the ELR
> related
> > > > > fields.
> > > > > >> > > > >> > > >>>> >
> > > > > >> > > > >> > > >>>> > On Thu, Sep 14, 2023 at 9:19 PM Colin
> McCabe <
> > > > > >> > > > >> cmccabe@apache.org>
> > > > > >> > > > >> > > >>>> wrote:
> > > > > >> > > > >> > > >>>> >
> > > > > >> > > > >> > > >>>> >> Hi Calvin,
> > > > > >> > > > >> > > >>>> >>
> > > > > >> > > > >> > > >>>> >> Thanks for the changes.
> > > > > >> > > > >> > > >>>> >>
> > > > > >> > > > >> > > >>>> >> 1. Earlier I commented that creating
> > > > > >> > > > >> "unclean.recovery.strategy "
> > > > > >> > > > >> > > is
> > > > > >> > > > >> > > >>>> not
> > > > > >> > > > >> > > >>>> >> necessary, and we can just reuse the
> existing
> > > > > >> > > > >> > > >>>> >> "unclean.leader.election.enable"
> > configuration
> > > > key.
> > > > > >> > Let's
> > > > > >> > > > >> discuss
> > > > > >> > > > >> > > >>>> that.
> > > > > >> > > > >> > > >>>> >>
> > > > > >> > > > >> > > >>>> >> 2.I also don't understand why you didn't
> add
> > a
> > > > > >> > > > configuration to
> > > > > >> > > > >> > > >>>> enable or
> > > > > >> > > > >> > > >>>> >> disable the Unclean Recovery Manager. This
> > > seems
> > > > > >> like a
> > > > > >> > > very
> > > > > >> > > > >> > simple
> > > > > >> > > > >> > > >>>> way to
> > > > > >> > > > >> > > >>>> >> handle the staging issue which we
> discussed.
> > > The
> > > > > URM
> > > > > >> can
> > > > > >> > > > just
> > > > > >> > > > >> be
> > > > > >> > > > >> > > >>>> turned off
> > > > > >> > > > >> > > >>>> >> until it is production ready. Let's discuss
> > > this.
> > > > > >> > > > >> > > >>>> >>
> > > > > >> > > > >> > > >>>> >> 3. You still need to describe the changes
> to
> > > > > >> AdminClient
> > > > > >> > > > that
> > > > > >> > > > >> are
> > > > > >> > > > >> > > >>>> needed
> > > > > >> > > > >> > > >>>> >> to use DescribeTopicRequest.
> > > > > >> > > > >> > > >>>> >>
> > > > > >> > > > >> > > >>>> >> Keep at it. It's looking better. :)
> > > > > >> > > > >> > > >>>> >>
> > > > > >> > > > >> > > >>>> >> best,
> > > > > >> > > > >> > > >>>> >> Colin
> > > > > >> > > > >> > > >>>> >>
> > > > > >> > > > >> > > >>>> >>
> > > > > >> > > > >> > > >>>> >> On Thu, Sep 14, 2023, at 11:03, Calvin Liu
> > > wrote:
> > > > > >> > > > >> > > >>>> >> > Hi Colin
> > > > > >> > > > >> > > >>>> >> > Thanks for the comments!
> > > > > >> > > > >> > > >>>> >> >
> > > > > >> > > > >> > > >>>> >> > I did the following changes
> > > > > >> > > > >> > > >>>> >> >
> > > > > >> > > > >> > > >>>> >> >    1.
> > > > > >> > > > >> > > >>>> >> >
> > > > > >> > > > >> > > >>>> >> >    Simplified the API spec section to
> only
> > > > > include
> > > > > >> the
> > > > > >> > > > diff.
> > > > > >> > > > >> > > >>>> >> >    2.
> > > > > >> > > > >> > > >>>> >> >
> > > > > >> > > > >> > > >>>> >> >    Reordered the HWM requirement section.
> > > > > >> > > > >> > > >>>> >> >    3.
> > > > > >> > > > >> > > >>>> >> >
> > > > > >> > > > >> > > >>>> >> >    Removed the URM implementation details
> > to
> > > > keep
> > > > > >> the
> > > > > >> > > > >> necessary
> > > > > >> > > > >> > > >>>> >> >    characteristics to perform the unclean
> > > > > recovery.
> > > > > >> > > > >> > > >>>> >> >    1.
> > > > > >> > > > >> > > >>>> >> >
> > > > > >> > > > >> > > >>>> >> >       When to perform the unclean
> recovery
> > > > > >> > > > >> > > >>>> >> >       2.
> > > > > >> > > > >> > > >>>> >> >
> > > > > >> > > > >> > > >>>> >> >       Under different config, how the
> > unclean
> > > > > >> recovery
> > > > > >> > > > finds
> > > > > >> > > > >> > the
> > > > > >> > > > >> > > >>>> leader.
> > > > > >> > > > >> > > >>>> >> >       3.
> > > > > >> > > > >> > > >>>> >> >
> > > > > >> > > > >> > > >>>> >> >       How the config
> > > > > unclean.leader.election.enable
> > > > > >> > and
> > > > > >> > > > >> > > >>>> >> >       unclean.recovery.strategy are
> > converted
> > > > > when
> > > > > >> > users
> > > > > >> > > > >> > > >>>> enable/disable
> > > > > >> > > > >> > > >>>> >> the
> > > > > >> > > > >> > > >>>> >> >       unclean recovery.
> > > > > >> > > > >> > > >>>> >> >       4.
> > > > > >> > > > >> > > >>>> >> >
> > > > > >> > > > >> > > >>>> >> >    More details about how we change admin
> > > > client.
> > > > > >> > > > >> > > >>>> >> >    5.
> > > > > >> > > > >> > > >>>> >> >
> > > > > >> > > > >> > > >>>> >> >    API limits on the
> > GetReplicaLogInfoRequest
> > > > and
> > > > > >> > > > >> > > >>>> DescribeTopicRequest.
> > > > > >> > > > >> > > >>>> >> >    6.
> > > > > >> > > > >> > > >>>> >> >
> > > > > >> > > > >> > > >>>> >> >    Two metrics added
> > > > > >> > > > >> > > >>>> >> >    1.
> > > > > >> > > > >> > > >>>> >> >
> > > > > >> > > > >> > > >>>> >> >
> > > > > >> > > >  Kafka.controller.global_under_min_isr_partition_count
> > > > > >> > > > >> > > >>>> >> >       2.
> > > > > >> > > > >> > > >>>> >> >
> > > > > >> > > > >> > > >>>> >> >
> > > > > >>  kafka.controller.unclean_recovery_finished_count
> > > > > >> > > > >> > > >>>> >> >
> > > > > >> > > > >> > > >>>> >> >
> > > > > >> > > > >> > > >>>> >> > On Wed, Sep 13, 2023 at 10:46 AM Colin
> > > McCabe <
> > > > > >> > > > >> > > cmccabe@apache.org>
> > > > > >> > > > >> > > >>>> >> wrote:
> > > > > >> > > > >> > > >>>> >> >
> > > > > >> > > > >> > > >>>> >> >> On Tue, Sep 12, 2023, at 17:21, Calvin
> Liu
> > > > > wrote:
> > > > > >> > > > >> > > >>>> >> >> > Hi Colin
> > > > > >> > > > >> > > >>>> >> >> > Thanks for the comments!
> > > > > >> > > > >> > > >>>> >> >> >
> > > > > >> > > > >> > > >>>> >> >>
> > > > > >> > > > >> > > >>>> >> >> Hi Calvin,
> > > > > >> > > > >> > > >>>> >> >>
> > > > > >> > > > >> > > >>>> >> >> Thanks again for the KIP.
> > > > > >> > > > >> > > >>>> >> >>
> > > > > >> > > > >> > > >>>> >> >> One meta-comment: it's usually better to
> > > just
> > > > > do a
> > > > > >> > diff
> > > > > >> > > > on a
> > > > > >> > > > >> > > >>>> message
> > > > > >> > > > >> > > >>>> >> spec
> > > > > >> > > > >> > > >>>> >> >> file or java file if you're including
> > > changes
> > > > to
> > > > > >> it
> > > > > >> > in
> > > > > >> > > > the
> > > > > >> > > > >> > KIP.
> > > > > >> > > > >> > > >>>> This is
> > > > > >> > > > >> > > >>>> >> >> easier to read than looking for "new
> > fields
> > > > > begin"
> > > > > >> > etc.
> > > > > >> > > > in
> > > > > >> > > > >> the
> > > > > >> > > > >> > > >>>> text, and
> > > > > >> > > > >> > > >>>> >> >> gracefully handles the case where
> existing
> > > > > fields
> > > > > >> > were
> > > > > >> > > > >> > changed.
> > > > > >> > > > >> > > >>>> >> >>
> > > > > >> > > > >> > > >>>> >> >> > Rewrite the Additional High Watermark
> > > > > >> advancement
> > > > > >> > > > >> > requirement
> > > > > >> > > > >> > > >>>> >> >> > There was feedback on this section
> that
> > > some
> > > > > >> > readers
> > > > > >> > > > may
> > > > > >> > > > >> not
> > > > > >> > > > >> > > be
> > > > > >> > > > >> > > >>>> >> familiar
> > > > > >> > > > >> > > >>>> >> >> > with HWM and Ack=0,1,all requests.
> This
> > > can
> > > > > help
> > > > > >> > them
> > > > > >> > > > >> > > understand
> > > > > >> > > > >> > > >>>> the
> > > > > >> > > > >> > > >>>> >> >> > proposal. I will rewrite this part for
> > > more
> > > > > >> > > > readability.
> > > > > >> > > > >> > > >>>> >> >> >
> > > > > >> > > > >> > > >>>> >> >>
> > > > > >> > > > >> > > >>>> >> >> To be clear, I wasn't suggesting
> dropping
> > > > either
> > > > > >> > > > section. I
> > > > > >> > > > >> > > agree
> > > > > >> > > > >> > > >>>> that
> > > > > >> > > > >> > > >>>> >> >> they add useful background. I was just
> > > > > suggesting
> > > > > >> > that
> > > > > >> > > we
> > > > > >> > > > >> > should
> > > > > >> > > > >> > > >>>> discuss
> > > > > >> > > > >> > > >>>> >> >> the "acks" setting AFTER discussing the
> > new
> > > > high
> > > > > >> > > > watermark
> > > > > >> > > > >> > > >>>> advancement
> > > > > >> > > > >> > > >>>> >> >> conditions. We also should discuss
> acks=0.
> > > > While
> > > > > >> it
> > > > > >> > > isn't
> > > > > >> > > > >> > > >>>> conceptually
> > > > > >> > > > >> > > >>>> >> much
> > > > > >> > > > >> > > >>>> >> >> different than acks=1 here, its omission
> > > from
> > > > > this
> > > > > >> > > > section
> > > > > >> > > > >> is
> > > > > >> > > > >> > > >>>> confusing.
> > > > > >> > > > >> > > >>>> >> >>
> > > > > >> > > > >> > > >>>> >> >> > Unclean recovery
> > > > > >> > > > >> > > >>>> >> >> >
> > > > > >> > > > >> > > >>>> >> >> > The plan is to replace the
> > > > > >> > > > unclean.leader.election.enable
> > > > > >> > > > >> > with
> > > > > >> > > > >> > > >>>> >> >> > unclean.recovery.strategy. If the
> > Unclean
> > > > > >> Recovery
> > > > > >> > is
> > > > > >> > > > >> > enabled
> > > > > >> > > > >> > > >>>> then it
> > > > > >> > > > >> > > >>>> >> >> deals
> > > > > >> > > > >> > > >>>> >> >> > with the three options in the
> > > > > >> > > > unclean.recovery.strategy.
> > > > > >> > > > >> > > >>>> >> >> >
> > > > > >> > > > >> > > >>>> >> >> >
> > > > > >> > > > >> > > >>>> >> >> > Let’s refine the Unclean Recovery. We
> > have
> > > > > >> already
> > > > > >> > > > taken a
> > > > > >> > > > >> > > lot of
> > > > > >> > > > >> > > >>>> >> >> > suggestions and I hope to enhance the
> > > > > >> durability of
> > > > > >> > > > Kafka
> > > > > >> > > > >> to
> > > > > >> > > > >> > > the
> > > > > >> > > > >> > > >>>> next
> > > > > >> > > > >> > > >>>> >> >> level
> > > > > >> > > > >> > > >>>> >> >> > with this KIP.
> > > > > >> > > > >> > > >>>> >> >>
> > > > > >> > > > >> > > >>>> >> >> I am OK with doing the unclean leader
> > > recovery
> > > > > >> > > > improvements
> > > > > >> > > > >> in
> > > > > >> > > > >> > > >>>> this KIP.
> > > > > >> > > > >> > > >>>> >> >> However, I think we need to really work
> on
> > > the
> > > > > >> > > > configuration
> > > > > >> > > > >> > > >>>> settings.
> > > > > >> > > > >> > > >>>> >> >>
> > > > > >> > > > >> > > >>>> >> >> Configuration overrides are often quite
> > > messy.
> > > > > For
> > > > > >> > > > example,
> > > > > >> > > > >> > the
> > > > > >> > > > >> > > >>>> cases
> > > > > >> > > > >> > > >>>> >> >> where we have log.roll.hours and
> > > > > >> log.roll.segment.ms
> > > > > >> > ,
> > > > > >> > > > the
> > > > > >> > > > >> > user
> > > > > >> > > > >> > > >>>> has to
> > > > > >> > > > >> > > >>>> >> >> remember which one takes precedence, and
> > it
> > > is
> > > > > not
> > > > > >> > > > obvious.
> > > > > >> > > > >> > So,
> > > > > >> > > > >> > > >>>> rather
> > > > > >> > > > >> > > >>>> >> than
> > > > > >> > > > >> > > >>>> >> >> creating a new configuration, why not
> add
> > > > > >> additional
> > > > > >> > > > values
> > > > > >> > > > >> to
> > > > > >> > > > >> > > >>>> >> >> "unclean.leader.election.enable"? I
> think
> > > this
> > > > > >> will
> > > > > >> > be
> > > > > >> > > > >> simpler
> > > > > >> > > > >> > > for
> > > > > >> > > > >> > > >>>> >> people
> > > > > >> > > > >> > > >>>> >> >> to understand, and simpler in the code
> as
> > > > well.
> > > > > >> > > > >> > > >>>> >> >>
> > > > > >> > > > >> > > >>>> >> >> What if we continued to use
> > > > > >> > > > "unclean.leader.election.enable"
> > > > > >> > > > >> > but
> > > > > >> > > > >> > > >>>> >> extended
> > > > > >> > > > >> > > >>>> >> >> it so that it took a string? Then the
> > string
> > > > > could
> > > > > >> > have
> > > > > >> > > > >> these
> > > > > >> > > > >> > > >>>> values:
> > > > > >> > > > >> > > >>>> >> >>
> > > > > >> > > > >> > > >>>> >> >> never
> > > > > >> > > > >> > > >>>> >> >>     never automatically do an unclean
> > leader
> > > > > >> election
> > > > > >> > > > under
> > > > > >> > > > >> > any
> > > > > >> > > > >> > > >>>> >> conditions
> > > > > >> > > > >> > > >>>> >> >>
> > > > > >> > > > >> > > >>>> >> >> false / default
> > > > > >> > > > >> > > >>>> >> >>     only do an unclean leader election
> if
> > > > there
> > > > > >> may
> > > > > >> > be
> > > > > >> > > > >> > possible
> > > > > >> > > > >> > > >>>> data
> > > > > >> > > > >> > > >>>> >> loss
> > > > > >> > > > >> > > >>>> >> >>
> > > > > >> > > > >> > > >>>> >> >> true / always
> > > > > >> > > > >> > > >>>> >> >>     always do an unclean leader election
> > if
> > > we
> > > > > >> can't
> > > > > >> > > > >> > immediately
> > > > > >> > > > >> > > >>>> elect a
> > > > > >> > > > >> > > >>>> >> >> leader
> > > > > >> > > > >> > > >>>> >> >>
> > > > > >> > > > >> > > >>>> >> >> It's a bit awkward that false maps to
> > > default
> > > > > >> rather
> > > > > >> > > > than to
> > > > > >> > > > >> > > >>>> never. But
> > > > > >> > > > >> > > >>>> >> >> this awkwardness exists if we use two
> > > > different
> > > > > >> > > > >> configuration
> > > > > >> > > > >> > > keys
> > > > > >> > > > >> > > >>>> as
> > > > > >> > > > >> > > >>>> >> well.
> > > > > >> > > > >> > > >>>> >> >> The reason for the awkwardness is that
> we
> > > > simply
> > > > > >> > don't
> > > > > >> > > > want
> > > > > >> > > > >> > most
> > > > > >> > > > >> > > >>>> of the
> > > > > >> > > > >> > > >>>> >> >> people currently setting
> > > > > >> > > > >> unclean.leader.election.enable=false
> > > > > >> > > > >> > to
> > > > > >> > > > >> > > >>>> get the
> > > > > >> > > > >> > > >>>> >> >> "never" behavior. We have to bite that
> > > bullet.
> > > > > >> Better
> > > > > >> > > to
> > > > > >> > > > be
> > > > > >> > > > >> > > clear
> > > > > >> > > > >> > > >>>> and
> > > > > >> > > > >> > > >>>> >> >> explicit than hide it.
> > > > > >> > > > >> > > >>>> >> >>
> > > > > >> > > > >> > > >>>> >> >> Another thing that's a bit awkward is
> > having
> > > > two
> > > > > >> > > > different
> > > > > >> > > > >> > ways
> > > > > >> > > > >> > > to
> > > > > >> > > > >> > > >>>> do
> > > > > >> > > > >> > > >>>> >> >> unclean leader election specified in the
> > > KIP.
> > > > > You
> > > > > >> > > > descirbe
> > > > > >> > > > >> two
> > > > > >> > > > >> > > >>>> methods:
> > > > > >> > > > >> > > >>>> >> the
> > > > > >> > > > >> > > >>>> >> >> simple "choose the last leader" method,
> > and
> > > > the
> > > > > >> > > "unclean
> > > > > >> > > > >> > > recovery
> > > > > >> > > > >> > > >>>> >> manager"
> > > > > >> > > > >> > > >>>> >> >> method. I understand why you did it this
> > way
> > > > --
> > > > > >> > "choose
> > > > > >> > > > the
> > > > > >> > > > >> > last
> > > > > >> > > > >> > > >>>> >> leader" is
> > > > > >> > > > >> > > >>>> >> >> simple, and will help us deliver an
> > > > > implementation
> > > > > >> > > > quickly,
> > > > > >> > > > >> > > while
> > > > > >> > > > >> > > >>>> the
> > > > > >> > > > >> > > >>>> >> URM
> > > > > >> > > > >> > > >>>> >> >> is preferable in the long term. My
> > > suggestion
> > > > > >> here is
> > > > > >> > > to
> > > > > >> > > > >> > > separate
> > > > > >> > > > >> > > >>>> the
> > > > > >> > > > >> > > >>>> >> >> decision of HOW to do unclean leader
> > > election
> > > > > from
> > > > > >> > the
> > > > > >> > > > >> > decision
> > > > > >> > > > >> > > of
> > > > > >> > > > >> > > >>>> WHEN
> > > > > >> > > > >> > > >>>> >> to
> > > > > >> > > > >> > > >>>> >> >> do it.
> > > > > >> > > > >> > > >>>> >> >>
> > > > > >> > > > >> > > >>>> >> >> So in other words, have
> > > > > >> > > "unclean.leader.election.enable"
> > > > > >> > > > >> > specify
> > > > > >> > > > >> > > >>>> when we
> > > > > >> > > > >> > > >>>> >> >> do unclean leader election, and have a
> new
> > > > > >> > > configuration
> > > > > >> > > > >> like
> > > > > >> > > > >> > > >>>> >> >> "unclean.recovery.manager.enable" to
> > > determine
> > > > > if
> > > > > >> we
> > > > > >> > > use
> > > > > >> > > > the
> > > > > >> > > > >> > > URM.
> > > > > >> > > > >> > > >>>> >> >> Presumably the URM will take some time
> to
> > > get
> > > > > >> fully
> > > > > >> > > > stable,
> > > > > >> > > > >> so
> > > > > >> > > > >> > > >>>> this can
> > > > > >> > > > >> > > >>>> >> >> default to false for a while, and we can
> > > flip
> > > > > the
> > > > > >> > > > default to
> > > > > >> > > > >> > > true
> > > > > >> > > > >> > > >>>> when
> > > > > >> > > > >> > > >>>> >> we
> > > > > >> > > > >> > > >>>> >> >> feel ready.
> > > > > >> > > > >> > > >>>> >> >>
> > > > > >> > > > >> > > >>>> >> >> The URM is somewhat under-described
> here.
> > I
> > > > > think
> > > > > >> we
> > > > > >> > > > need a
> > > > > >> > > > >> > few
> > > > > >> > > > >> > > >>>> >> >> configurations here for it. For example,
> > we
> > > > > need a
> > > > > >> > > > >> > > configuration to
> > > > > >> > > > >> > > >>>> >> specify
> > > > > >> > > > >> > > >>>> >> >> how long it should wait for a broker to
> > > > respond
> > > > > to
> > > > > >> > its
> > > > > >> > > > RPCs
> > > > > >> > > > >> > > before
> > > > > >> > > > >> > > >>>> >> moving
> > > > > >> > > > >> > > >>>> >> >> on. We also need to understand how the
> URM
> > > > > >> interacts
> > > > > >> > > with
> > > > > >> > > > >> > > >>>> >> >> unclean.leader.election.enable=always. I
> > > > assume
> > > > > >> that
> > > > > >> > > with
> > > > > >> > > > >> > > "always"
> > > > > >> > > > >> > > >>>> we
> > > > > >> > > > >> > > >>>> >> will
> > > > > >> > > > >> > > >>>> >> >> just unconditionally use the URM rather
> > than
> > > > > >> choosing
> > > > > >> > > > >> > randomly.
> > > > > >> > > > >> > > >>>> But this
> > > > > >> > > > >> > > >>>> >> >> should be spelled out in the KIP.
> > > > > >> > > > >> > > >>>> >> >>
> > > > > >> > > > >> > > >>>> >> >> >
> > > > > >> > > > >> > > >>>> >> >> > DescribeTopicRequest
> > > > > >> > > > >> > > >>>> >> >> >
> > > > > >> > > > >> > > >>>> >> >> >    1.
> > > > > >> > > > >> > > >>>> >> >> >    Yes, the plan is to replace the
> > > > > >> MetadataRequest
> > > > > >> > > with
> > > > > >> > > > >> the
> > > > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest for the admin
> > > > clients.
> > > > > >> Will
> > > > > >> > > > check
> > > > > >> > > > >> > the
> > > > > >> > > > >> > > >>>> details.
> > > > > >> > > > >> > > >>>> >> >>
> > > > > >> > > > >> > > >>>> >> >> Sounds good. But as I said, you need to
> > > > specify
> > > > > >> how
> > > > > >> > > > >> > AdminClient
> > > > > >> > > > >> > > >>>> >> interacts
> > > > > >> > > > >> > > >>>> >> >> with the new request. This will involve
> > > adding
> > > > > >> some
> > > > > >> > > > fields
> > > > > >> > > > >> to
> > > > > >> > > > >> > > >>>> >> >> TopicDescription.java. And you need to
> > > specify
> > > > > the
> > > > > >> > > > changes
> > > > > >> > > > >> to
> > > > > >> > > > >> > > the
> > > > > >> > > > >> > > >>>> >> >> kafka-topics.sh command line tool.
> > Otherwise
> > > > we
> > > > > >> > cannot
> > > > > >> > > > use
> > > > > >> > > > >> the
> > > > > >> > > > >> > > >>>> tool to
> > > > > >> > > > >> > > >>>> >> see
> > > > > >> > > > >> > > >>>> >> >> the new information.
> > > > > >> > > > >> > > >>>> >> >>
> > > > > >> > > > >> > > >>>> >> >> The new requests, DescribeTopicRequest
> and
> > > > > >> > > > >> > > >>>> GetReplicaLogInfoRequest,
> > > > > >> > > > >> > > >>>> >> need
> > > > > >> > > > >> > > >>>> >> >> to have limits placed on them so that
> > their
> > > > size
> > > > > >> > can't
> > > > > >> > > be
> > > > > >> > > > >> > > >>>> infinite. We
> > > > > >> > > > >> > > >>>> >> >> don't want to propagate the current
> > problems
> > > > of
> > > > > >> > > > >> > MetadataRequest,
> > > > > >> > > > >> > > >>>> where
> > > > > >> > > > >> > > >>>> >> >> clients can request massive responses
> that
> > > can
> > > > > >> mess
> > > > > >> > up
> > > > > >> > > > the
> > > > > >> > > > >> JVM
> > > > > >> > > > >> > > when
> > > > > >> > > > >> > > >>>> >> handled.
> > > > > >> > > > >> > > >>>> >> >>
> > > > > >> > > > >> > > >>>> >> >> Adding limits is simple for
> > > > > >> GetReplicaLogInfoRequest
> > > > > >> > --
> > > > > >> > > > we
> > > > > >> > > > >> can
> > > > > >> > > > >> > > >>>> just say
> > > > > >> > > > >> > > >>>> >> >> that only 2000 partitions at a time can
> be
> > > > > >> requested.
> > > > > >> > > For
> > > > > >> > > > >> > > >>>> >> >> DescribeTopicRequest we can probably
> just
> > > > limit
> > > > > >> to 20
> > > > > >> > > > topics
> > > > > >> > > > >> > or
> > > > > >> > > > >> > > >>>> >> something
> > > > > >> > > > >> > > >>>> >> >> like that, to avoid the complexity of
> > doing
> > > > > >> > pagination
> > > > > >> > > in
> > > > > >> > > > >> this
> > > > > >> > > > >> > > KIP.
> > > > > >> > > > >> > > >>>> >> >>
> > > > > >> > > > >> > > >>>> >> >> >    2.
> > > > > >> > > > >> > > >>>> >> >> >    I can let the broker load the ELR
> > info
> > > so
> > > > > >> that
> > > > > >> > > they
> > > > > >> > > > can
> > > > > >> > > > >> > > serve
> > > > > >> > > > >> > > >>>> the
> > > > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest as well.
> > > > > >> > > > >> > > >>>> >> >>
> > > > > >> > > > >> > > >>>> >> >> Yes, it's fine to add to MetadataCache.
> In
> > > > fact,
> > > > > >> > you'll
> > > > > >> > > > be
> > > > > >> > > > >> > > loading
> > > > > >> > > > >> > > >>>> it
> > > > > >> > > > >> > > >>>> >> >> anyway once it's added to
> PartitionImage.
> > > > > >> > > > >> > > >>>> >> >>
> > > > > >> > > > >> > > >>>> >> >> >    3.
> > > > > >> > > > >> > > >>>> >> >> >    Yeah, it does not make sense to
> have
> > > the
> > > > > >> topic
> > > > > >> > id
> > > > > >> > > if
> > > > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest is only used
> by
> > > the
> > > > > >> admin
> > > > > >> > > > client.
> > > > > >> > > > >> > > >>>> >> >>
> > > > > >> > > > >> > > >>>> >> >> OK. That makes things simpler. We can
> > always
> > > > > >> create a
> > > > > >> > > new
> > > > > >> > > > >> API
> > > > > >> > > > >> > > later
> > > > > >> > > > >> > > >>>> >> >> (hopefully not in this KIP!) to query by
> > > topic
> > > > > ID.
> > > > > >> > > > >> > > >>>> >> >>
> > > > > >> > > > >> > > >>>> >> >> >
> > > > > >> > > > >> > > >>>> >> >> >
> > > > > >> > > > >> > > >>>> >> >> > Metrics
> > > > > >> > > > >> > > >>>> >> >> >
> > > > > >> > > > >> > > >>>> >> >> > As for overall cluster health
> metrics, I
> > > > think
> > > > > >> > > > >> under-min-ISR
> > > > > >> > > > >> > > is
> > > > > >> > > > >> > > >>>> still
> > > > > >> > > > >> > > >>>> >> a
> > > > > >> > > > >> > > >>>> >> >> > useful one. ELR is more like a safety
> > > belt.
> > > > > When
> > > > > >> > the
> > > > > >> > > > ELR
> > > > > >> > > > >> is
> > > > > >> > > > >> > > >>>> used, the
> > > > > >> > > > >> > > >>>> >> >> > cluster availability has already been
> > > > > impacted.
> > > > > >> > > > >> > > >>>> >> >> >
> > > > > >> > > > >> > > >>>> >> >> > Maybe we can have a metric to count
> the
> > > > > >> partitions
> > > > > >> > > that
> > > > > >> > > > >> > > sum(ISR,
> > > > > >> > > > >> > > >>>> ELR)
> > > > > >> > > > >> > > >>>> >> <
> > > > > >> > > > >> > > >>>> >> >> min
> > > > > >> > > > >> > > >>>> >> >> > ISR. What do you think?
> > > > > >> > > > >> > > >>>> >> >>
> > > > > >> > > > >> > > >>>> >> >> How about:
> > > > > >> > > > >> > > >>>> >> >>
> > > > > >> > > > >> > > >>>> >> >> A.  a metric for the totoal number of
> > > > > >> under-min-isr
> > > > > >> > > > >> > partitions?
> > > > > >> > > > >> > > We
> > > > > >> > > > >> > > >>>> don't
> > > > > >> > > > >> > > >>>> >> >> have that in Apache Kafka at the moment.
> > > > > >> > > > >> > > >>>> >> >>
> > > > > >> > > > >> > > >>>> >> >> B. a metric for the number of unclean
> > leader
> > > > > >> > elections
> > > > > >> > > we
> > > > > >> > > > >> did
> > > > > >> > > > >> > > (for
> > > > > >> > > > >> > > >>>> >> >> simplicity, it can reset to 0 on
> > controller
> > > > > >> restart:
> > > > > >> > we
> > > > > >> > > > >> expect
> > > > > >> > > > >> > > >>>> people to
> > > > > >> > > > >> > > >>>> >> >> monitor the change over time anyway)
> > > > > >> > > > >> > > >>>> >> >>
> > > > > >> > > > >> > > >>>> >> >> best,
> > > > > >> > > > >> > > >>>> >> >> Colin
> > > > > >> > > > >> > > >>>> >> >>
> > > > > >> > > > >> > > >>>> >> >>
> > > > > >> > > > >> > > >>>> >> >> >
> > > > > >> > > > >> > > >>>> >> >> > Yeah, for the ongoing unclean
> > recoveries,
> > > > the
> > > > > >> > > > controller
> > > > > >> > > > >> can
> > > > > >> > > > >> > > >>>> keep an
> > > > > >> > > > >> > > >>>> >> >> > accurate count through failover
> because
> > > > > >> partition
> > > > > >> > > > >> > registration
> > > > > >> > > > >> > > >>>> can
> > > > > >> > > > >> > > >>>> >> >> indicate
> > > > > >> > > > >> > > >>>> >> >> > whether a recovery is needed. However,
> > for
> > > > the
> > > > > >> > > happened
> > > > > >> > > > >> > ones,
> > > > > >> > > > >> > > >>>> unless
> > > > > >> > > > >> > > >>>> >> we
> > > > > >> > > > >> > > >>>> >> >> > want to persist the number somewhere,
> we
> > > can
> > > > > >> only
> > > > > >> > > > figure
> > > > > >> > > > >> it
> > > > > >> > > > >> > > out
> > > > > >> > > > >> > > >>>> from
> > > > > >> > > > >> > > >>>> >> the
> > > > > >> > > > >> > > >>>> >> >> > log.
> > > > > >> > > > >> > > >>>> >> >> >
> > > > > >> > > > >> > > >>>> >> >> > On Tue, Sep 12, 2023 at 3:16 PM Colin
> > > > McCabe <
> > > > > >> > > > >> > > cmccabe@apache.org
> > > > > >> > > > >> > > >>>> >
> > > > > >> > > > >> > > >>>> >> wrote:
> > > > > >> > > > >> > > >>>> >> >> >
> > > > > >> > > > >> > > >>>> >> >> >> Also, we should have metrics that
> show
> > > what
> > > > > is
> > > > > >> > going
> > > > > >> > > > on
> > > > > >> > > > >> > with
> > > > > >> > > > >> > > >>>> regard
> > > > > >> > > > >> > > >>>> >> to
> > > > > >> > > > >> > > >>>> >> >> the
> > > > > >> > > > >> > > >>>> >> >> >> eligible replica set. I'm not sure
> > > exactly
> > > > > >> what to
> > > > > >> > > > >> suggest,
> > > > > >> > > > >> > > but
> > > > > >> > > > >> > > >>>> >> >> something
> > > > > >> > > > >> > > >>>> >> >> >> that could identify when things are
> > going
> > > > > >> wrong in
> > > > > >> > > the
> > > > > >> > > > >> > > clsuter.
> > > > > >> > > > >> > > >>>> >> >> >>
> > > > > >> > > > >> > > >>>> >> >> >> For example, maybe a metric for
> > > partitions
> > > > > >> > > containing
> > > > > >> > > > >> > > replicas
> > > > > >> > > > >> > > >>>> that
> > > > > >> > > > >> > > >>>> >> are
> > > > > >> > > > >> > > >>>> >> >> >> ineligible to be leader? That would
> > show
> > > a
> > > > > >> spike
> > > > > >> > > when
> > > > > >> > > > a
> > > > > >> > > > >> > > broker
> > > > > >> > > > >> > > >>>> had an
> > > > > >> > > > >> > > >>>> >> >> >> unclean restart.
> > > > > >> > > > >> > > >>>> >> >> >>
> > > > > >> > > > >> > > >>>> >> >> >> Ideally, we'd also have a metric that
> > > > > indicates
> > > > > >> > when
> > > > > >> > > > an
> > > > > >> > > > >> > > unclear
> > > > > >> > > > >> > > >>>> >> leader
> > > > > >> > > > >> > > >>>> >> >> >> election or a recovery happened.
> It's a
> > > bit
> > > > > >> tricky
> > > > > >> > > > >> because
> > > > > >> > > > >> > > the
> > > > > >> > > > >> > > >>>> simple
> > > > > >> > > > >> > > >>>> >> >> >> thing, of tracking it per controller,
> > may
> > > > be
> > > > > a
> > > > > >> bit
> > > > > >> > > > >> > confusing
> > > > > >> > > > >> > > >>>> during
> > > > > >> > > > >> > > >>>> >> >> >> failovers.
> > > > > >> > > > >> > > >>>> >> >> >>
> > > > > >> > > > >> > > >>>> >> >> >> best,
> > > > > >> > > > >> > > >>>> >> >> >> Colin
> > > > > >> > > > >> > > >>>> >> >> >>
> > > > > >> > > > >> > > >>>> >> >> >>
> > > > > >> > > > >> > > >>>> >> >> >> On Tue, Sep 12, 2023, at 14:25, Colin
> > > > McCabe
> > > > > >> > wrote:
> > > > > >> > > > >> > > >>>> >> >> >> > Hi Calvin,
> > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > >> > > > >> > > >>>> >> >> >> > Thanks for the KIP. I think this
> is a
> > > > great
> > > > > >> > > > >> improvement.
> > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > >> > > > >> > > >>>> >> >> >> >> Additional High Watermark advance
> > > > > >> requirement
> > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > >> > > > >> > > >>>> >> >> >> > Typo: change "advance" to
> > "advancement"
> > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > >> > > > >> > > >>>> >> >> >> >> A bit recap of some key concepts.
> > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > >> > > > >> > > >>>> >> >> >> > Typo: change "bit" to "quick"
> > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > >> > > > >> > > >>>> >> >> >> >> Ack=1/all produce request. It
> > defines
> > > > when
> > > > > >> the
> > > > > >> > > > Kafka
> > > > > >> > > > >> > > server
> > > > > >> > > > >> > > >>>> should
> > > > > >> > > > >> > > >>>> >> >> >> respond to the produce request
> > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > >> > > > >> > > >>>> >> >> >> > I think this section would be
> clearer
> > > if
> > > > we
> > > > > >> > talked
> > > > > >> > > > >> about
> > > > > >> > > > >> > > the
> > > > > >> > > > >> > > >>>> new
> > > > > >> > > > >> > > >>>> >> high
> > > > > >> > > > >> > > >>>> >> >> >> > watermark advancement requirement
> > > first,
> > > > > and
> > > > > >> > THEN
> > > > > >> > > > >> talked
> > > > > >> > > > >> > > >>>> about its
> > > > > >> > > > >> > > >>>> >> >> >> > impact on acks=0, acks=1, and
> > > >  acks=all.
> > > > > >> > > > acks=all
> > > > > >> > > > >> is
> > > > > >> > > > >> > of
> > > > > >> > > > >> > > >>>> course
> > > > > >> > > > >> > > >>>> >> the
> > > > > >> > > > >> > > >>>> >> >> >> > main case we care about here, so it
> > > would
> > > > > be
> > > > > >> > good
> > > > > >> > > to
> > > > > >> > > > >> lead
> > > > > >> > > > >> > > with
> > > > > >> > > > >> > > >>>> >> that,
> > > > > >> > > > >> > > >>>> >> >> >> > rather than delving into the
> > > > technicalities
> > > > > >> of
> > > > > >> > > > acks=0/1
> > > > > >> > > > >> > > first.
> > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > >> > > > >> > > >>>> >> >> >> >> Unclean recovery
> > > > > >> > > > >> > > >>>> >> >> >> >
> > > > > >> > > > >> > > >>>> >> >> >> > So, here you are introducing a new
> > > > > >> > configuration,
> > > > > >> > > > >> > > >>>> >> >> >> > unclean.recovery.strategy. The
> > > difficult
> > > > > >> thing
> > > > > >> > > here
> > > > > >> > > > is
> > > > > >> > > > >> > that
> > > > > >> > > > >> > > >>>> there
> > > > > >> > > > >> > > >>>> >> is a
> > > > > >> > > > >> > > >>>> >> >> >> > lot of overlap with
> > > > > >> > > unclean.leader.election.enable.
> > > > > >> > > > So
> > > > > >> > > > >> we
> > > > > >> > > > >> > > >>>> have 3
> > > > > >> > > > >> > > >>>> >> >> >> > different settings for
> > > > > >> > unclean.recovery.strategy,
> > > > > >> > > > plus
> > > > > >> > > > >> 2
> > > > > >> > > > >> > > >>>> different
> > > > > >> > > > >> > > >>>> >> >> >> > settings for
> > > > > unclean.leader.election.enable,
> > > > > >> > > giving
> > > > > >> > > > a
> > > > > >> > > > >> > cross
> > > > > >> > > > >> > > >>>> >> product of
> > > > > >> > > > >> > > >>>> >> >> >> > 6 different options. The following
> > > > "unclean
> > > > > >> > > recovery
> > > > > >> > > > >> > > manager"
> > > > > >> > > > >> > > >>>> >> section
> > > > > >> > > > >> > > >>>> >> >> >> > on
> >
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Calvin Liu <ca...@confluent.io.INVALID>.
Hi David,
Thanks for the comments.
----
I thought that a new snapshot with the downgraded MV is created in this
case. Isn’t it the case?
Yes, you are right, a metadata delta will be generated after the MV
downgrade. Then the user can start the software downgrade.
-----
Could you also elaborate a bit more on the reasoning behind adding the
limits to the admin RPCs? This is a new pattern in Kafka so it would be
good to clear on the motivation.
Thanks to Colin for bringing it up. The current MetadataRequest does not
have a limit on the number of topics to query in a single request. Massive
requests can mess up the JVM. We want to have some sort of throttle on the
new APIs.
-----
Could you also explain how the client is supposed to handle the
topics/partitions above the limit? I suppose that it will have to retry
those, correct?
Corrent. For the official admin clients, it will split the large request
into proper pieces and query one after another.
-----
My understanding is that the topics/partitions above the limit will be
failed with an invalid exception error. I wonder if this choice is
judicious because the invalide request exception is usually fatal. It may
be better to use an new and explicit error for this case.

Thanks for bringing this up. How about "REQUEST_LIMIT_REACHED"?
--------
It seems that we still need to specify the changes to the admin api to
accommodate the new or updated apis. Do you plan to add them?
Try to cover the following
1. The admin client will use the new DescribeTopicRequest to query the
topics
2. Mention the API limit and the new retriable error.
3. Output changes for the admin client when describing a topic (new fields
of ELR...)
4. Changes to data structures like TopicPartitionInfo to include the ELR.
Anything else I missed?

Thanks!





On Wed, Oct 4, 2023 at 12:27 PM David Jacot <da...@gmail.com> wrote:

> Hi Calvin,
>
> I thought that a new snapshot with the downgraded MV is created in this
> case. Isn’t it the case?
>
> Could you also elaborate a bit more on the reasoning behind adding the
> limits to the admin RPCs? This is a new pattern in Kafka so it would be
> good to clear on the motivation.
>
> Could you also explain how the client is supposed to handle the
> topics/partitions above the limit? I suppose that it will have to retry
> those, correct?
>
> My understanding is that the topics/partitions above the limit will be
> failed with an invalid exception error. I wonder if this choice is
> judicious because the invalide request exception is usually fatal. It may
> be better to use an new and explicit error for this case.
>
> It seems that we still need to specify the changes to the admin api to
> accommodate the new or updated apis. Do you plan to add them?
>
> Best,
> David
>
> Le mer. 4 oct. 2023 à 20:39, Calvin Liu <ca...@confluent.io.invalid> a
> écrit :
>
> > Hi Jun,
> > After the MV downgrade, the controller will write in the old version of
> the
> > PartitionRecord/PartitionChangeRecord. If I understand correctly, it is
> > possible to downgrade the software version if the controller only has to
> > handle old version records.
> > However, the controller will not automatically rewrite the
> PartitionRecord
> > with the old version unless there is a partition update. Then, the user
> may
> > have to wait an unknown amount of time before the software downgrades
> > unless they do a roll to force update every partition. If it makes
> sense, I
> > can mention these steps to do a software downgrade.
> > Thanks
> >
> > On Wed, Oct 4, 2023 at 11:20 AM Jun Rao <ju...@confluent.io.invalid>
> wrote:
> >
> > > Hi, Calvin and Justine,
> > >
> > > Historically, when we change the record format in the log, we don't
> > support
> > > software version downgrading.
> > >
> > > For the record format change in the metadata log, have we thought about
> > > forcing the write of the latest metadata records with the old version
> > > during MV downgrading? This will in theory allow the old version of the
> > > software to obtain the latest metadata.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Wed, Oct 4, 2023 at 9:53 AM Justine Olshan
> > <jolshan@confluent.io.invalid
> > > >
> > > wrote:
> > >
> > > > Sorry -- not MV but software version.
> > > >
> > > > On Wed, Oct 4, 2023 at 9:51 AM Justine Olshan <jo...@confluent.io>
> > > > wrote:
> > > >
> > > > > Catching up with this discussion.
> > > > >
> > > > > I was just curious -- have we had other instances where downgrading
> > MV
> > > is
> > > > > not supported? I think Kafka typically tries to support downgrades,
> > > and I
> > > > > couldn't think of other examples.
> > > > >
> > > > > Thanks,
> > > > > Justine
> > > > >
> > > > > On Wed, Oct 4, 2023 at 9:40 AM Calvin Liu
> <caliu@confluent.io.invalid
> > >
> > > > > wrote:
> > > > >
> > > > >> Hi Jun,
> > > > >> 54. Marked the software downgrading is not supported. As the old
> > > > >> controller
> > > > >> will not understand the new PartitionRecord and
> > PartitionChangeRecord.
> > > > >> Thanks!
> > > > >>
> > > > >> On Wed, Oct 4, 2023 at 9:12 AM Jun Rao <ju...@confluent.io.invalid>
> > > > wrote:
> > > > >>
> > > > >> > Hi, Calvin,
> > > > >> >
> > > > >> > Thanks for the reply. Just one more comment.
> > > > >> >
> > > > >> > 54. It seems that downgrading MV is supported. Is downgrading
> the
> > > > >> software
> > > > >> > version supported? It would be useful to document that.
> > > > >> >
> > > > >> > Thanks,
> > > > >> >
> > > > >> > Jun
> > > > >> >
> > > > >> > On Tue, Oct 3, 2023 at 4:55 PM Artem Livshits
> > > > >> > <al...@confluent.io.invalid> wrote:
> > > > >> >
> > > > >> > > Hi Colin,
> > > > >> > >
> > > > >> > > I think in your example "do_unclean_recovery" would need to do
> > > > >> different
> > > > >> > > things depending on the strategy.
> > > > >> > >
> > > > >> > > do_unclean_recovery() {
> > > > >> > >    if (unclean.recovery.manager.enabled) {
> > > > >> > >     if (strategy == Aggressive)
> > > > >> > >       use UncleanRecoveryManager(waitLastKnownERL=false)  //
> > just
> > > > >> inspect
> > > > >> > > logs from whoever is available
> > > > >> > >     else
> > > > >> > >       use  UncleanRecoveryManager(waitLastKnownERL=true)  //
> > must
> > > > wait
> > > > >> > for
> > > > >> > > at least last known ELR
> > > > >> > >   } else {
> > > > >> > >     if (strategy == Aggressive)
> > > > >> > >       choose the last known leader if that is available, or a
> > > random
> > > > >> > leader
> > > > >> > > if not)
> > > > >> > >     else
> > > > >> > >       wait for last known leader to get back
> > > > >> > >   }
> > > > >> > > }
> > > > >> > >
> > > > >> > > The idea is that the Aggressive strategy would kick in as soon
> > as
> > > we
> > > > >> lost
> > > > >> > > the leader and would pick a leader from whoever is available;
> > but
> > > > the
> > > > >> > > Balanced will only kick in when ELR is empty and will wait for
> > the
> > > > >> > brokers
> > > > >> > > that likely have most data to be available.
> > > > >> > >
> > > > >> > > On Tue, Oct 3, 2023 at 3:04 PM Colin McCabe <
> cmccabe@apache.org
> > >
> > > > >> wrote:
> > > > >> > >
> > > > >> > > > On Tue, Oct 3, 2023, at 10:49, Jun Rao wrote:
> > > > >> > > > > Hi, Calvin,
> > > > >> > > > >
> > > > >> > > > > Thanks for the update KIP. A few more comments.
> > > > >> > > > >
> > > > >> > > > > 41. Why would a user choose the option to select a random
> > > > replica
> > > > >> as
> > > > >> > > the
> > > > >> > > > > leader instead of using
> unclean.recovery.strateg=Aggressive?
> > > It
> > > > >> seems
> > > > >> > > > that
> > > > >> > > > > the latter is strictly better? If that's not the case,
> could
> > > we
> > > > >> fold
> > > > >> > > this
> > > > >> > > > > option under unclean.recovery.strategy instead of
> > introducing
> > > a
> > > > >> > > separate
> > > > >> > > > > config?
> > > > >> > > >
> > > > >> > > > Hi Jun,
> > > > >> > > >
> > > > >> > > > I thought the flow of control was:
> > > > >> > > >
> > > > >> > > > If there is no leader for the partition {
> > > > >> > > >   If (there are unfenced ELR members) {
> > > > >> > > >     choose_an_unfenced_ELR_member
> > > > >> > > >   } else if (there are fenced ELR members AND
> > > > strategy=Aggressive) {
> > > > >> > > >     do_unclean_recovery
> > > > >> > > >   } else if (there are no ELR members AND strategy != None)
> {
> > > > >> > > >     do_unclean_recovery
> > > > >> > > >   } else {
> > > > >> > > >     do nothing about the missing leader
> > > > >> > > >   }
> > > > >> > > > }
> > > > >> > > >
> > > > >> > > > do_unclean_recovery() {
> > > > >> > > >    if (unclean.recovery.manager.enabled) {
> > > > >> > > >     use UncleanRecoveryManager
> > > > >> > > >   } else {
> > > > >> > > >     choose the last known leader if that is available, or a
> > > random
> > > > >> > leader
> > > > >> > > > if not)
> > > > >> > > >   }
> > > > >> > > > }
> > > > >> > > >
> > > > >> > > > However, I think this could be clarified, especially the
> > > behavior
> > > > >> when
> > > > >> > > > unclean.recovery.manager.enabled=false. Inuitively the goal
> > for
> > > > >> > > > unclean.recovery.manager.enabled=false is to be "the same as
> > > now,
> > > > >> > mostly"
> > > > >> > > > but it's very underspecified in the KIP, I agree.
> > > > >> > > >
> > > > >> > > > >
> > > > >> > > > > 50. ElectLeadersRequest: "If more than 20 topics are
> > included,
> > > > >> only
> > > > >> > the
> > > > >> > > > > first 20 will be served. Others will be returned with
> > > > >> > DesiredLeaders."
> > > > >> > > > Hmm,
> > > > >> > > > > not sure that I understand this. ElectLeadersResponse
> > doesn't
> > > > >> have a
> > > > >> > > > > DesiredLeaders field.
> > > > >> > > > >
> > > > >> > > > > 51. GetReplicaLogInfo: "If more than 2000 partitions are
> > > > included,
> > > > >> > only
> > > > >> > > > the
> > > > >> > > > > first 2000 will be served" Do we return an error for the
> > > > remaining
> > > > >> > > > > partitions? Actually, should we include an errorCode field
> > at
> > > > the
> > > > >> > > > partition
> > > > >> > > > > level in GetReplicaLogInfoResponse to cover non-existing
> > > > >> partitions
> > > > >> > and
> > > > >> > > > no
> > > > >> > > > > authorization, etc?
> > > > >> > > > >
> > > > >> > > > > 52. The entry should matches => The entry should match
> > > > >> > > > >
> > > > >> > > > > 53. ElectLeadersRequest.DesiredLeaders: Should it be
> > nullable
> > > > >> since a
> > > > >> > > > user
> > > > >> > > > > may not specify DesiredLeaders?
> > > > >> > > > >
> > > > >> > > > > 54. Downgrade: Is that indeed possible? I thought earlier
> > you
> > > > said
> > > > >> > that
> > > > >> > > > > once the new version of the records are in the metadata
> log,
> > > one
> > > > >> > can't
> > > > >> > > > > downgrade since the old broker doesn't know how to parse
> the
> > > new
> > > > >> > > version
> > > > >> > > > of
> > > > >> > > > > the metadata records?
> > > > >> > > > >
> > > > >> > > >
> > > > >> > > > MetadataVersion downgrade is currently broken but we have
> > fixing
> > > > it
> > > > >> on
> > > > >> > > our
> > > > >> > > > plate for Kafka 3.7.
> > > > >> > > >
> > > > >> > > > The way downgrade works is that "new features" are dropped,
> > > > leaving
> > > > >> > only
> > > > >> > > > the old ones.
> > > > >> > > >
> > > > >> > > > > 55. CleanShutdownFile: Should we add a version field for
> > > future
> > > > >> > > > extension?
> > > > >> > > > >
> > > > >> > > > > 56. Config changes are public facing. Could we have a
> > separate
> > > > >> > section
> > > > >> > > to
> > > > >> > > > > document all the config changes?
> > > > >> > > >
> > > > >> > > > +1. A separate section for this would be good.
> > > > >> > > >
> > > > >> > > > best,
> > > > >> > > > Colin
> > > > >> > > >
> > > > >> > > > >
> > > > >> > > > > Thanks,
> > > > >> > > > >
> > > > >> > > > > Jun
> > > > >> > > > >
> > > > >> > > > > On Mon, Sep 25, 2023 at 4:29 PM Calvin Liu
> > > > >> > <caliu@confluent.io.invalid
> > > > >> > > >
> > > > >> > > > > wrote:
> > > > >> > > > >
> > > > >> > > > >> Hi Jun
> > > > >> > > > >> Thanks for the comments.
> > > > >> > > > >>
> > > > >> > > > >> 40. If we change to None, it is not guaranteed for no
> data
> > > > loss.
> > > > >> For
> > > > >> > > > users
> > > > >> > > > >> who are not able to validate the data with external
> > > resources,
> > > > >> > manual
> > > > >> > > > >> intervention does not give a better result but a loss of
> > > > >> > availability.
> > > > >> > > > So
> > > > >> > > > >> practically speaking, the Balance mode would be a better
> > > > default
> > > > >> > > value.
> > > > >> > > > >>
> > > > >> > > > >> 41. No, it represents how we want to do the unclean
> leader
> > > > >> election.
> > > > >> > > If
> > > > >> > > > it
> > > > >> > > > >> is false, the unclean leader election will be the old
> > random
> > > > way.
> > > > >> > > > >> Otherwise, the unclean recovery will be used.
> > > > >> > > > >>
> > > > >> > > > >> 42. Good catch. Updated.
> > > > >> > > > >>
> > > > >> > > > >> 43. Only the first 20 topics will be served. Others will
> be
> > > > >> returned
> > > > >> > > > with
> > > > >> > > > >> InvalidRequestError
> > > > >> > > > >>
> > > > >> > > > >> 44. The order matters. The desired leader entries match
> > with
> > > > the
> > > > >> > topic
> > > > >> > > > >> partition list by the index.
> > > > >> > > > >>
> > > > >> > > > >> 45. Thanks! Updated.
> > > > >> > > > >>
> > > > >> > > > >> 46. Good advice! Updated.
> > > > >> > > > >>
> > > > >> > > > >> 47.1, updated the comment. Basically it will elect the
> > > replica
> > > > in
> > > > >> > the
> > > > >> > > > >> desiredLeader field to be the leader
> > > > >> > > > >>
> > > > >> > > > >> 47.2 We can let the admin client do the conversion. Using
> > the
> > > > >> > > > desiredLeader
> > > > >> > > > >> field in the json format seems easier for users.
> > > > >> > > > >>
> > > > >> > > > >> 48. Once the MV version is downgraded, all the ELR
> related
> > > > fields
> > > > >> > will
> > > > >> > > > be
> > > > >> > > > >> removed on the next partition change. The controller will
> > > also
> > > > >> > ignore
> > > > >> > > > the
> > > > >> > > > >> ELR fields. Updated the KIP.
> > > > >> > > > >>
> > > > >> > > > >> 49. Yes, it would be deprecated/removed.
> > > > >> > > > >>
> > > > >> > > > >>
> > > > >> > > > >> On Mon, Sep 25, 2023 at 3:49 PM Jun Rao
> > > > <jun@confluent.io.invalid
> > > > >> >
> > > > >> > > > wrote:
> > > > >> > > > >>
> > > > >> > > > >> > Hi, Calvin,
> > > > >> > > > >> >
> > > > >> > > > >> > Thanks for the updated KIP. Made another pass. A few
> more
> > > > >> comments
> > > > >> > > > below.
> > > > >> > > > >> >
> > > > >> > > > >> > 40. unclean.leader.election.enable.false ->
> > > > >> > > > >> > unclean.recovery.strategy.Balanced: The Balanced mode
> > could
> > > > >> still
> > > > >> > > > lead to
> > > > >> > > > >> > data loss. So, I am wondering if
> > > > >> > > unclean.leader.election.enable.false
> > > > >> > > > >> > should map to None?
> > > > >> > > > >> >
> > > > >> > > > >> > 41. unclean.recovery.manager.enabled: I am not sure why
> > we
> > > > >> > introduce
> > > > >> > > > this
> > > > >> > > > >> > additional config. Is it the same as
> > > > >> > unclean.recovery.strategy=None?
> > > > >> > > > >> >
> > > > >> > > > >> > 42. DescribeTopicResponse.TopicAuthorizedOperations:
> > Should
> > > > >> this
> > > > >> > be
> > > > >> > > at
> > > > >> > > > >> the
> > > > >> > > > >> > topic level?
> > > > >> > > > >> >
> > > > >> > > > >> > 43. "Limit: 20 topics max per request": Could we
> describe
> > > > what
> > > > >> > > > happens if
> > > > >> > > > >> > the request includes more than 20 topics?
> > > > >> > > > >> >
> > > > >> > > > >> > 44. ElectLeadersRequest.DesiredLeaders: Could we
> describe
> > > > >> whether
> > > > >> > > the
> > > > >> > > > >> > ordering matters?
> > > > >> > > > >> >
> > > > >> > > > >> > 45. GetReplicaLogInfo.TopicPartitions: "about": "The
> > topic
> > > > >> > > partitions
> > > > >> > > > to
> > > > >> > > > >> > elect leaders.": The description in "about" is
> incorrect.
> > > > >> > > > >> >
> > > > >> > > > >> > 46. GetReplicaLogInfoResponse: Should we nest
> partitions
> > > > under
> > > > >> > > > topicId to
> > > > >> > > > >> > be consistent with other types of responses?
> > > > >> > > > >> >
> > > > >> > > > >> > 47. kafka-leader-election.sh:
> > > > >> > > > >> > 47.1 Could we explain DESIGNATION?
> > > > >> > > > >> > 47.2 desiredLeader: Should it be a list to match the
> > field
> > > in
> > > > >> > > > >> > ElectLeadersRequest?
> > > > >> > > > >> >
> > > > >> > > > >> > 48. We could add a section on downgrade?
> > > > >> > > > >> >
> > > > >> > > > >> > 49. LastKnownLeader: This seems only needed in the
> first
> > > > phase
> > > > >> of
> > > > >> > > > >> > delivering ELR. Will it be removed when the complete
> KIP
> > is
> > > > >> > > delivered?
> > > > >> > > > >> >
> > > > >> > > > >> > Thanks,
> > > > >> > > > >> >
> > > > >> > > > >> > Jun
> > > > >> > > > >> >
> > > > >> > > > >> > On Tue, Sep 19, 2023 at 1:30 PM Colin McCabe <
> > > > >> cmccabe@apache.org>
> > > > >> > > > wrote:
> > > > >> > > > >> >
> > > > >> > > > >> > > Hi Calvin,
> > > > >> > > > >> > >
> > > > >> > > > >> > > Thanks for the explanations. I like the idea of using
> > > none,
> > > > >> > > > balanced,
> > > > >> > > > >> > > aggressive. We also had an offline discussion about
> why
> > > it
> > > > is
> > > > >> > good
> > > > >> > > > to
> > > > >> > > > >> > use a
> > > > >> > > > >> > > new config key (basically, so that we can deprecate
> the
> > > old
> > > > >> one
> > > > >> > > > which
> > > > >> > > > >> had
> > > > >> > > > >> > > only false/true values in 4.0) With these changes, I
> am
> > > +1.
> > > > >> > > > >> > >
> > > > >> > > > >> > > best,
> > > > >> > > > >> > > Colin
> > > > >> > > > >> > >
> > > > >> > > > >> > > On Mon, Sep 18, 2023, at 15:54, Calvin Liu wrote:
> > > > >> > > > >> > > > Hi Colin,
> > > > >> > > > >> > > > Also, can we deprecate
> unclean.leader.election.enable
> > > in
> > > > >> 4.0?
> > > > >> > > > Before
> > > > >> > > > >> > > that,
> > > > >> > > > >> > > > we can have both the config
> unclean.recovery.strategy
> > > and
> > > > >> > > > >> > > > unclean.leader.election.enable
> > > > >> > > > >> > > > and using the unclean.recovery.Enabled to determine
> > > which
> > > > >> > config
> > > > >> > > > to
> > > > >> > > > >> use
> > > > >> > > > >> > > > during the unclean leader election.
> > > > >> > > > >> > > >
> > > > >> > > > >> > > > On Mon, Sep 18, 2023 at 3:51 PM Calvin Liu <
> > > > >> > caliu@confluent.io>
> > > > >> > > > >> wrote:
> > > > >> > > > >> > > >
> > > > >> > > > >> > > >> Hi Colin,
> > > > >> > > > >> > > >> For the unclean.recovery.strategy config name, how
> > > about
> > > > >> we
> > > > >> > use
> > > > >> > > > the
> > > > >> > > > >> > > >> following
> > > > >> > > > >> > > >> None. It basically means no unclean recovery will
> be
> > > > >> > performed.
> > > > >> > > > >> > > >> Aggressive. It means availability goes first.
> > Whenever
> > > > the
> > > > >> > > > partition
> > > > >> > > > >> > > can't
> > > > >> > > > >> > > >> elect a durable replica, the controller will try
> the
> > > > >> unclean
> > > > >> > > > >> recovery.
> > > > >> > > > >> > > >> Balanced. It is the balance point of the
> > availability
> > > > >> > > > >> > first(Aggressive)
> > > > >> > > > >> > > >> and least availability(None). The controller
> > performs
> > > > >> unclean
> > > > >> > > > >> recovery
> > > > >> > > > >> > > when
> > > > >> > > > >> > > >> both ISR and ELR are empty.
> > > > >> > > > >> > > >>
> > > > >> > > > >> > > >>
> > > > >> > > > >> > > >> On Fri, Sep 15, 2023 at 11:42 AM Calvin Liu <
> > > > >> > > caliu@confluent.io>
> > > > >> > > > >> > wrote:
> > > > >> > > > >> > > >>
> > > > >> > > > >> > > >>> Hi Colin,
> > > > >> > > > >> > > >>>
> > > > >> > > > >> > > >>> > So, the proposal is that if someone sets
> > > > >> > > > >> > > "unclean.leader.election.enable
> > > > >> > > > >> > > >>> = true"...
> > > > >> > > > >> > > >>>
> > > > >> > > > >> > > >>>
> > > > >> > > > >> > > >>> The idea is to use one of the
> > > > >> unclean.leader.election.enable
> > > > >> > > and
> > > > >> > > > >> > > >>> unclean.recovery.strategy based on the
> > > > >> > > > unclean.recovery.Enabled. A
> > > > >> > > > >> > > possible
> > > > >> > > > >> > > >>> version can be
> > > > >> > > > >> > > >>>
> > > > >> > > > >> > > >>> If unclean.recovery.Enabled:
> > > > >> > > > >> > > >>>
> > > > >> > > > >> > > >>> {
> > > > >> > > > >> > > >>>
> > > > >> > > > >> > > >>> Check unclean.recovery.strategy. If set, use it.
> > > > >> Otherwise,
> > > > >> > > > check
> > > > >> > > > >> > > >>> unclean.leader.election.enable and translate it
> to
> > > > >> > > > >> > > >>> unclean.recovery.strategy.
> > > > >> > > > >> > > >>>
> > > > >> > > > >> > > >>> } else {
> > > > >> > > > >> > > >>>
> > > > >> > > > >> > > >>> Use unclean.leader.election.enable
> > > > >> > > > >> > > >>>
> > > > >> > > > >> > > >>> }
> > > > >> > > > >> > > >>>
> > > > >> > > > >> > > >>>
> > > > >> > > > >> > > >>> —--------
> > > > >> > > > >> > > >>>
> > > > >> > > > >> > > >>> >The configuration key should be
> > > > >> > > > >> "unclean.recovery.manager.enabled",
> > > > >> > > > >> > > >>> right?
> > > > >> > > > >> > > >>>
> > > > >> > > > >> > > >>>
> > > > >> > > > >> > > >>> I think we have two ways of choosing a leader
> > > > uncleanly,
> > > > >> > > unclean
> > > > >> > > > >> > leader
> > > > >> > > > >> > > >>> election and unclean recovery(log inspection) and
> > we
> > > > try
> > > > >> to
> > > > >> > > > switch
> > > > >> > > > >> > > between
> > > > >> > > > >> > > >>> them.
> > > > >> > > > >> > > >>>
> > > > >> > > > >> > > >>> Do you mean we want to develop two ways of
> > performing
> > > > the
> > > > >> > > > unclean
> > > > >> > > > >> > > >>> recovery and one of them is using “unclean
> recovery
> > > > >> > manager”?
> > > > >> > > I
> > > > >> > > > >> guess
> > > > >> > > > >> > > we
> > > > >> > > > >> > > >>> haven’t discussed the second way.
> > > > >> > > > >> > > >>>
> > > > >> > > > >> > > >>>
> > > > >> > > > >> > > >>> —-------
> > > > >> > > > >> > > >>>
> > > > >> > > > >> > > >>> >How do these 4 levels of overrides interact with
> > > your
> > > > >> new
> > > > >> > > > >> > > >>> configurations?
> > > > >> > > > >> > > >>>
> > > > >> > > > >> > > >>>
> > > > >> > > > >> > > >>> I do notice in the Kraft controller code, the
> > method
> > > to
> > > > >> > check
> > > > >> > > > >> whether
> > > > >> > > > >> > > >>> perform unclean leader election is hard coded to
> > > false
> > > > >> since
> > > > >> > > > >> > > >>> 2021(uncleanLeaderElectionEnabledForTopic). Isn’t
> > it
> > > a
> > > > >> good
> > > > >> > > > chance
> > > > >> > > > >> to
> > > > >> > > > >> > > >>> completely deprecate the
> > > > unclean.leader.election.enable?
> > > > >> We
> > > > >> > > > don’t
> > > > >> > > > >> > even
> > > > >> > > > >> > > have
> > > > >> > > > >> > > >>> to worry about the config conversion.
> > > > >> > > > >> > > >>>
> > > > >> > > > >> > > >>> On the other hand, whatever the override is, as
> > long
> > > as
> > > > >> the
> > > > >> > > > >> > controller
> > > > >> > > > >> > > >>> can have the final effective
> > > > >> unclean.leader.election.enable,
> > > > >> > > the
> > > > >> > > > >> > topic
> > > > >> > > > >> > > >>> level config unclean.recovery.strategy, the
> cluster
> > > > level
> > > > >> > > config
> > > > >> > > > >> > > >>> unclean.recovery.Enabled, the controller can
> > > calculate
> > > > >> the
> > > > >> > > > correct
> > > > >> > > > >> > > methods
> > > > >> > > > >> > > >>> to use right?
> > > > >> > > > >> > > >>>
> > > > >> > > > >> > > >>>
> > > > >> > > > >> > > >>> On Fri, Sep 15, 2023 at 10:02 AM Colin McCabe <
> > > > >> > > > cmccabe@apache.org>
> > > > >> > > > >> > > wrote:
> > > > >> > > > >> > > >>>
> > > > >> > > > >> > > >>>> On Thu, Sep 14, 2023, at 22:23, Calvin Liu
> wrote:
> > > > >> > > > >> > > >>>> > Hi Colin
> > > > >> > > > >> > > >>>> > 1. I think using the new config name is more
> > > clear.
> > > > >> > > > >> > > >>>> >        a. The unclean leader election is
> > actually
> > > > >> removed
> > > > >> > > if
> > > > >> > > > >> > unclean
> > > > >> > > > >> > > >>>> > recovery is in use.
> > > > >> > > > >> > > >>>> >        b. Using multiple values in
> > > > >> > > > >> unclean.leader.election.enable
> > > > >> > > > >> > is
> > > > >> > > > >> > > >>>> > confusing and it will be more confusing after
> > > people
> > > > >> > forget
> > > > >> > > > >> about
> > > > >> > > > >> > > this
> > > > >> > > > >> > > >>>> > discussion.
> > > > >> > > > >> > > >>>>
> > > > >> > > > >> > > >>>> Hi Calvin,
> > > > >> > > > >> > > >>>>
> > > > >> > > > >> > > >>>> So, the proposal is that if someone sets
> > > > >> > > > >> > > "unclean.leader.election.enable
> > > > >> > > > >> > > >>>> = true" but then sets one of your new
> > > configurations,
> > > > >> the
> > > > >> > > > value of
> > > > >> > > > >> > > >>>> unclean.leader.election.enable is ignored? That
> > > seems
> > > > >> less
> > > > >> > > > clear
> > > > >> > > > >> to
> > > > >> > > > >> > > me, not
> > > > >> > > > >> > > >>>> more. Just in general, having multiple
> > configuration
> > > > >> keys
> > > > >> > to
> > > > >> > > > >> control
> > > > >> > > > >> > > the
> > > > >> > > > >> > > >>>> same thing confuses users. Basically, they are
> > > sitting
> > > > >> at a
> > > > >> > > > giant
> > > > >> > > > >> > > control
> > > > >> > > > >> > > >>>> panel, and some of the levers do nothing.
> > > > >> > > > >> > > >>>>
> > > > >> > > > >> > > >>>> > 2. Sorry I forgot to mention in the response
> > that
> > > I
> > > > >> did
> > > > >> > add
> > > > >> > > > the
> > > > >> > > > >> > > >>>> > unclean.recovery.Enabled flag.
> > > > >> > > > >> > > >>>>
> > > > >> > > > >> > > >>>> The configuration key should be
> > > > >> > > > >> "unclean.recovery.manager.enabled",
> > > > >> > > > >> > > >>>> right? Becuase we can do "unclean recovery"
> > without
> > > > the
> > > > >> > > > manager.
> > > > >> > > > >> > > Disabling
> > > > >> > > > >> > > >>>> the manager just means we use a different
> > mechanism
> > > > for
> > > > >> > > > recovery.
> > > > >> > > > >> > > >>>>
> > > > >> > > > >> > > >>>> >        c. Maybe I underestimated the challenge
> > of
> > > > >> > replacing
> > > > >> > > > the
> > > > >> > > > >> > > >>>> config. Any
> > > > >> > > > >> > > >>>> > implementation problems ahead?
> > > > >> > > > >> > > >>>>
> > > > >> > > > >> > > >>>> There are four levels of overrides for
> > > > >> > > > >> > unclean.leader.election.enable.
> > > > >> > > > >> > > >>>>
> > > > >> > > > >> > > >>>> 1. static configuration for node.
> > > > >> > > > >> > > >>>>     This goes in the configuration file,
> typically
> > > > named
> > > > >> > > > >> > > >>>> server.properties
> > > > >> > > > >> > > >>>>
> > > > >> > > > >> > > >>>> 2. dynamic configuration for node default
> > > > >> > > > >> > > >>>>   ConfigResource(type=BROKER, name="")
> > > > >> > > > >> > > >>>>
> > > > >> > > > >> > > >>>> 3. dynamic configuration for node
> > > > >> > > > >> > > >>>>   ConfigResource(type=BROKER, name=<controller
> > id>)
> > > > >> > > > >> > > >>>>
> > > > >> > > > >> > > >>>> 4. dynamic configuration for topic
> > > > >> > > > >> > > >>>>   ConfigResource(type=TOPIC, name=<topic-name>)
> > > > >> > > > >> > > >>>>
> > > > >> > > > >> > > >>>> How do these 4 levels of overrides interact with
> > > your
> > > > >> new
> > > > >> > > > >> > > >>>> configurations? If the new configurations
> dominate
> > > > over
> > > > >> the
> > > > >> > > old
> > > > >> > > > >> > ones,
> > > > >> > > > >> > > it
> > > > >> > > > >> > > >>>> seems like this will get a lot more confusing to
> > > > >> implement
> > > > >> > > (and
> > > > >> > > > >> also
> > > > >> > > > >> > > to
> > > > >> > > > >> > > >>>> use.)
> > > > >> > > > >> > > >>>>
> > > > >> > > > >> > > >>>> Again, I'd recommend just adding some new values
> > to
> > > > >> > > > >> > > >>>> unclean.leader.election.enable. It's simple and
> > will
> > > > >> > prevent
> > > > >> > > > user
> > > > >> > > > >> > > confusion
> > > > >> > > > >> > > >>>> (as well as developer confusion.)
> > > > >> > > > >> > > >>>>
> > > > >> > > > >> > > >>>> best,
> > > > >> > > > >> > > >>>> Colin
> > > > >> > > > >> > > >>>>
> > > > >> > > > >> > > >>>>
> > > > >> > > > >> > > >>>> > 3. About the admin client, I mentioned 3
> changes
> > > in
> > > > >> the
> > > > >> > > > client.
> > > > >> > > > >> > > >>>> Anything
> > > > >> > > > >> > > >>>> > else I missed in the KIP?
> > > > >> > > > >> > > >>>> >       a. The client will switch to using the
> new
> > > RPC
> > > > >> > > instead
> > > > >> > > > of
> > > > >> > > > >> > > >>>> > MetadataRequest for the topics.
> > > > >> > > > >> > > >>>> >       b. The TopicPartitionInfo used in
> > > > >> TopicDescription
> > > > >> > > > needs
> > > > >> > > > >> to
> > > > >> > > > >> > > add
> > > > >> > > > >> > > >>>> new
> > > > >> > > > >> > > >>>> > fields related to the ELR.
> > > > >> > > > >> > > >>>> >       c. The outputs will add the ELR related
> > > > fields.
> > > > >> > > > >> > > >>>> >
> > > > >> > > > >> > > >>>> > On Thu, Sep 14, 2023 at 9:19 PM Colin McCabe <
> > > > >> > > > >> cmccabe@apache.org>
> > > > >> > > > >> > > >>>> wrote:
> > > > >> > > > >> > > >>>> >
> > > > >> > > > >> > > >>>> >> Hi Calvin,
> > > > >> > > > >> > > >>>> >>
> > > > >> > > > >> > > >>>> >> Thanks for the changes.
> > > > >> > > > >> > > >>>> >>
> > > > >> > > > >> > > >>>> >> 1. Earlier I commented that creating
> > > > >> > > > >> "unclean.recovery.strategy "
> > > > >> > > > >> > > is
> > > > >> > > > >> > > >>>> not
> > > > >> > > > >> > > >>>> >> necessary, and we can just reuse the existing
> > > > >> > > > >> > > >>>> >> "unclean.leader.election.enable"
> configuration
> > > key.
> > > > >> > Let's
> > > > >> > > > >> discuss
> > > > >> > > > >> > > >>>> that.
> > > > >> > > > >> > > >>>> >>
> > > > >> > > > >> > > >>>> >> 2.I also don't understand why you didn't add
> a
> > > > >> > > > configuration to
> > > > >> > > > >> > > >>>> enable or
> > > > >> > > > >> > > >>>> >> disable the Unclean Recovery Manager. This
> > seems
> > > > >> like a
> > > > >> > > very
> > > > >> > > > >> > simple
> > > > >> > > > >> > > >>>> way to
> > > > >> > > > >> > > >>>> >> handle the staging issue which we discussed.
> > The
> > > > URM
> > > > >> can
> > > > >> > > > just
> > > > >> > > > >> be
> > > > >> > > > >> > > >>>> turned off
> > > > >> > > > >> > > >>>> >> until it is production ready. Let's discuss
> > this.
> > > > >> > > > >> > > >>>> >>
> > > > >> > > > >> > > >>>> >> 3. You still need to describe the changes to
> > > > >> AdminClient
> > > > >> > > > that
> > > > >> > > > >> are
> > > > >> > > > >> > > >>>> needed
> > > > >> > > > >> > > >>>> >> to use DescribeTopicRequest.
> > > > >> > > > >> > > >>>> >>
> > > > >> > > > >> > > >>>> >> Keep at it. It's looking better. :)
> > > > >> > > > >> > > >>>> >>
> > > > >> > > > >> > > >>>> >> best,
> > > > >> > > > >> > > >>>> >> Colin
> > > > >> > > > >> > > >>>> >>
> > > > >> > > > >> > > >>>> >>
> > > > >> > > > >> > > >>>> >> On Thu, Sep 14, 2023, at 11:03, Calvin Liu
> > wrote:
> > > > >> > > > >> > > >>>> >> > Hi Colin
> > > > >> > > > >> > > >>>> >> > Thanks for the comments!
> > > > >> > > > >> > > >>>> >> >
> > > > >> > > > >> > > >>>> >> > I did the following changes
> > > > >> > > > >> > > >>>> >> >
> > > > >> > > > >> > > >>>> >> >    1.
> > > > >> > > > >> > > >>>> >> >
> > > > >> > > > >> > > >>>> >> >    Simplified the API spec section to only
> > > > include
> > > > >> the
> > > > >> > > > diff.
> > > > >> > > > >> > > >>>> >> >    2.
> > > > >> > > > >> > > >>>> >> >
> > > > >> > > > >> > > >>>> >> >    Reordered the HWM requirement section.
> > > > >> > > > >> > > >>>> >> >    3.
> > > > >> > > > >> > > >>>> >> >
> > > > >> > > > >> > > >>>> >> >    Removed the URM implementation details
> to
> > > keep
> > > > >> the
> > > > >> > > > >> necessary
> > > > >> > > > >> > > >>>> >> >    characteristics to perform the unclean
> > > > recovery.
> > > > >> > > > >> > > >>>> >> >    1.
> > > > >> > > > >> > > >>>> >> >
> > > > >> > > > >> > > >>>> >> >       When to perform the unclean recovery
> > > > >> > > > >> > > >>>> >> >       2.
> > > > >> > > > >> > > >>>> >> >
> > > > >> > > > >> > > >>>> >> >       Under different config, how the
> unclean
> > > > >> recovery
> > > > >> > > > finds
> > > > >> > > > >> > the
> > > > >> > > > >> > > >>>> leader.
> > > > >> > > > >> > > >>>> >> >       3.
> > > > >> > > > >> > > >>>> >> >
> > > > >> > > > >> > > >>>> >> >       How the config
> > > > unclean.leader.election.enable
> > > > >> > and
> > > > >> > > > >> > > >>>> >> >       unclean.recovery.strategy are
> converted
> > > > when
> > > > >> > users
> > > > >> > > > >> > > >>>> enable/disable
> > > > >> > > > >> > > >>>> >> the
> > > > >> > > > >> > > >>>> >> >       unclean recovery.
> > > > >> > > > >> > > >>>> >> >       4.
> > > > >> > > > >> > > >>>> >> >
> > > > >> > > > >> > > >>>> >> >    More details about how we change admin
> > > client.
> > > > >> > > > >> > > >>>> >> >    5.
> > > > >> > > > >> > > >>>> >> >
> > > > >> > > > >> > > >>>> >> >    API limits on the
> GetReplicaLogInfoRequest
> > > and
> > > > >> > > > >> > > >>>> DescribeTopicRequest.
> > > > >> > > > >> > > >>>> >> >    6.
> > > > >> > > > >> > > >>>> >> >
> > > > >> > > > >> > > >>>> >> >    Two metrics added
> > > > >> > > > >> > > >>>> >> >    1.
> > > > >> > > > >> > > >>>> >> >
> > > > >> > > > >> > > >>>> >> >
> > > > >> > > >  Kafka.controller.global_under_min_isr_partition_count
> > > > >> > > > >> > > >>>> >> >       2.
> > > > >> > > > >> > > >>>> >> >
> > > > >> > > > >> > > >>>> >> >
> > > > >>  kafka.controller.unclean_recovery_finished_count
> > > > >> > > > >> > > >>>> >> >
> > > > >> > > > >> > > >>>> >> >
> > > > >> > > > >> > > >>>> >> > On Wed, Sep 13, 2023 at 10:46 AM Colin
> > McCabe <
> > > > >> > > > >> > > cmccabe@apache.org>
> > > > >> > > > >> > > >>>> >> wrote:
> > > > >> > > > >> > > >>>> >> >
> > > > >> > > > >> > > >>>> >> >> On Tue, Sep 12, 2023, at 17:21, Calvin Liu
> > > > wrote:
> > > > >> > > > >> > > >>>> >> >> > Hi Colin
> > > > >> > > > >> > > >>>> >> >> > Thanks for the comments!
> > > > >> > > > >> > > >>>> >> >> >
> > > > >> > > > >> > > >>>> >> >>
> > > > >> > > > >> > > >>>> >> >> Hi Calvin,
> > > > >> > > > >> > > >>>> >> >>
> > > > >> > > > >> > > >>>> >> >> Thanks again for the KIP.
> > > > >> > > > >> > > >>>> >> >>
> > > > >> > > > >> > > >>>> >> >> One meta-comment: it's usually better to
> > just
> > > > do a
> > > > >> > diff
> > > > >> > > > on a
> > > > >> > > > >> > > >>>> message
> > > > >> > > > >> > > >>>> >> spec
> > > > >> > > > >> > > >>>> >> >> file or java file if you're including
> > changes
> > > to
> > > > >> it
> > > > >> > in
> > > > >> > > > the
> > > > >> > > > >> > KIP.
> > > > >> > > > >> > > >>>> This is
> > > > >> > > > >> > > >>>> >> >> easier to read than looking for "new
> fields
> > > > begin"
> > > > >> > etc.
> > > > >> > > > in
> > > > >> > > > >> the
> > > > >> > > > >> > > >>>> text, and
> > > > >> > > > >> > > >>>> >> >> gracefully handles the case where existing
> > > > fields
> > > > >> > were
> > > > >> > > > >> > changed.
> > > > >> > > > >> > > >>>> >> >>
> > > > >> > > > >> > > >>>> >> >> > Rewrite the Additional High Watermark
> > > > >> advancement
> > > > >> > > > >> > requirement
> > > > >> > > > >> > > >>>> >> >> > There was feedback on this section that
> > some
> > > > >> > readers
> > > > >> > > > may
> > > > >> > > > >> not
> > > > >> > > > >> > > be
> > > > >> > > > >> > > >>>> >> familiar
> > > > >> > > > >> > > >>>> >> >> > with HWM and Ack=0,1,all requests. This
> > can
> > > > help
> > > > >> > them
> > > > >> > > > >> > > understand
> > > > >> > > > >> > > >>>> the
> > > > >> > > > >> > > >>>> >> >> > proposal. I will rewrite this part for
> > more
> > > > >> > > > readability.
> > > > >> > > > >> > > >>>> >> >> >
> > > > >> > > > >> > > >>>> >> >>
> > > > >> > > > >> > > >>>> >> >> To be clear, I wasn't suggesting dropping
> > > either
> > > > >> > > > section. I
> > > > >> > > > >> > > agree
> > > > >> > > > >> > > >>>> that
> > > > >> > > > >> > > >>>> >> >> they add useful background. I was just
> > > > suggesting
> > > > >> > that
> > > > >> > > we
> > > > >> > > > >> > should
> > > > >> > > > >> > > >>>> discuss
> > > > >> > > > >> > > >>>> >> >> the "acks" setting AFTER discussing the
> new
> > > high
> > > > >> > > > watermark
> > > > >> > > > >> > > >>>> advancement
> > > > >> > > > >> > > >>>> >> >> conditions. We also should discuss acks=0.
> > > While
> > > > >> it
> > > > >> > > isn't
> > > > >> > > > >> > > >>>> conceptually
> > > > >> > > > >> > > >>>> >> much
> > > > >> > > > >> > > >>>> >> >> different than acks=1 here, its omission
> > from
> > > > this
> > > > >> > > > section
> > > > >> > > > >> is
> > > > >> > > > >> > > >>>> confusing.
> > > > >> > > > >> > > >>>> >> >>
> > > > >> > > > >> > > >>>> >> >> > Unclean recovery
> > > > >> > > > >> > > >>>> >> >> >
> > > > >> > > > >> > > >>>> >> >> > The plan is to replace the
> > > > >> > > > unclean.leader.election.enable
> > > > >> > > > >> > with
> > > > >> > > > >> > > >>>> >> >> > unclean.recovery.strategy. If the
> Unclean
> > > > >> Recovery
> > > > >> > is
> > > > >> > > > >> > enabled
> > > > >> > > > >> > > >>>> then it
> > > > >> > > > >> > > >>>> >> >> deals
> > > > >> > > > >> > > >>>> >> >> > with the three options in the
> > > > >> > > > unclean.recovery.strategy.
> > > > >> > > > >> > > >>>> >> >> >
> > > > >> > > > >> > > >>>> >> >> >
> > > > >> > > > >> > > >>>> >> >> > Let’s refine the Unclean Recovery. We
> have
> > > > >> already
> > > > >> > > > taken a
> > > > >> > > > >> > > lot of
> > > > >> > > > >> > > >>>> >> >> > suggestions and I hope to enhance the
> > > > >> durability of
> > > > >> > > > Kafka
> > > > >> > > > >> to
> > > > >> > > > >> > > the
> > > > >> > > > >> > > >>>> next
> > > > >> > > > >> > > >>>> >> >> level
> > > > >> > > > >> > > >>>> >> >> > with this KIP.
> > > > >> > > > >> > > >>>> >> >>
> > > > >> > > > >> > > >>>> >> >> I am OK with doing the unclean leader
> > recovery
> > > > >> > > > improvements
> > > > >> > > > >> in
> > > > >> > > > >> > > >>>> this KIP.
> > > > >> > > > >> > > >>>> >> >> However, I think we need to really work on
> > the
> > > > >> > > > configuration
> > > > >> > > > >> > > >>>> settings.
> > > > >> > > > >> > > >>>> >> >>
> > > > >> > > > >> > > >>>> >> >> Configuration overrides are often quite
> > messy.
> > > > For
> > > > >> > > > example,
> > > > >> > > > >> > the
> > > > >> > > > >> > > >>>> cases
> > > > >> > > > >> > > >>>> >> >> where we have log.roll.hours and
> > > > >> log.roll.segment.ms
> > > > >> > ,
> > > > >> > > > the
> > > > >> > > > >> > user
> > > > >> > > > >> > > >>>> has to
> > > > >> > > > >> > > >>>> >> >> remember which one takes precedence, and
> it
> > is
> > > > not
> > > > >> > > > obvious.
> > > > >> > > > >> > So,
> > > > >> > > > >> > > >>>> rather
> > > > >> > > > >> > > >>>> >> than
> > > > >> > > > >> > > >>>> >> >> creating a new configuration, why not add
> > > > >> additional
> > > > >> > > > values
> > > > >> > > > >> to
> > > > >> > > > >> > > >>>> >> >> "unclean.leader.election.enable"? I think
> > this
> > > > >> will
> > > > >> > be
> > > > >> > > > >> simpler
> > > > >> > > > >> > > for
> > > > >> > > > >> > > >>>> >> people
> > > > >> > > > >> > > >>>> >> >> to understand, and simpler in the code as
> > > well.
> > > > >> > > > >> > > >>>> >> >>
> > > > >> > > > >> > > >>>> >> >> What if we continued to use
> > > > >> > > > "unclean.leader.election.enable"
> > > > >> > > > >> > but
> > > > >> > > > >> > > >>>> >> extended
> > > > >> > > > >> > > >>>> >> >> it so that it took a string? Then the
> string
> > > > could
> > > > >> > have
> > > > >> > > > >> these
> > > > >> > > > >> > > >>>> values:
> > > > >> > > > >> > > >>>> >> >>
> > > > >> > > > >> > > >>>> >> >> never
> > > > >> > > > >> > > >>>> >> >>     never automatically do an unclean
> leader
> > > > >> election
> > > > >> > > > under
> > > > >> > > > >> > any
> > > > >> > > > >> > > >>>> >> conditions
> > > > >> > > > >> > > >>>> >> >>
> > > > >> > > > >> > > >>>> >> >> false / default
> > > > >> > > > >> > > >>>> >> >>     only do an unclean leader election if
> > > there
> > > > >> may
> > > > >> > be
> > > > >> > > > >> > possible
> > > > >> > > > >> > > >>>> data
> > > > >> > > > >> > > >>>> >> loss
> > > > >> > > > >> > > >>>> >> >>
> > > > >> > > > >> > > >>>> >> >> true / always
> > > > >> > > > >> > > >>>> >> >>     always do an unclean leader election
> if
> > we
> > > > >> can't
> > > > >> > > > >> > immediately
> > > > >> > > > >> > > >>>> elect a
> > > > >> > > > >> > > >>>> >> >> leader
> > > > >> > > > >> > > >>>> >> >>
> > > > >> > > > >> > > >>>> >> >> It's a bit awkward that false maps to
> > default
> > > > >> rather
> > > > >> > > > than to
> > > > >> > > > >> > > >>>> never. But
> > > > >> > > > >> > > >>>> >> >> this awkwardness exists if we use two
> > > different
> > > > >> > > > >> configuration
> > > > >> > > > >> > > keys
> > > > >> > > > >> > > >>>> as
> > > > >> > > > >> > > >>>> >> well.
> > > > >> > > > >> > > >>>> >> >> The reason for the awkwardness is that we
> > > simply
> > > > >> > don't
> > > > >> > > > want
> > > > >> > > > >> > most
> > > > >> > > > >> > > >>>> of the
> > > > >> > > > >> > > >>>> >> >> people currently setting
> > > > >> > > > >> unclean.leader.election.enable=false
> > > > >> > > > >> > to
> > > > >> > > > >> > > >>>> get the
> > > > >> > > > >> > > >>>> >> >> "never" behavior. We have to bite that
> > bullet.
> > > > >> Better
> > > > >> > > to
> > > > >> > > > be
> > > > >> > > > >> > > clear
> > > > >> > > > >> > > >>>> and
> > > > >> > > > >> > > >>>> >> >> explicit than hide it.
> > > > >> > > > >> > > >>>> >> >>
> > > > >> > > > >> > > >>>> >> >> Another thing that's a bit awkward is
> having
> > > two
> > > > >> > > > different
> > > > >> > > > >> > ways
> > > > >> > > > >> > > to
> > > > >> > > > >> > > >>>> do
> > > > >> > > > >> > > >>>> >> >> unclean leader election specified in the
> > KIP.
> > > > You
> > > > >> > > > descirbe
> > > > >> > > > >> two
> > > > >> > > > >> > > >>>> methods:
> > > > >> > > > >> > > >>>> >> the
> > > > >> > > > >> > > >>>> >> >> simple "choose the last leader" method,
> and
> > > the
> > > > >> > > "unclean
> > > > >> > > > >> > > recovery
> > > > >> > > > >> > > >>>> >> manager"
> > > > >> > > > >> > > >>>> >> >> method. I understand why you did it this
> way
> > > --
> > > > >> > "choose
> > > > >> > > > the
> > > > >> > > > >> > last
> > > > >> > > > >> > > >>>> >> leader" is
> > > > >> > > > >> > > >>>> >> >> simple, and will help us deliver an
> > > > implementation
> > > > >> > > > quickly,
> > > > >> > > > >> > > while
> > > > >> > > > >> > > >>>> the
> > > > >> > > > >> > > >>>> >> URM
> > > > >> > > > >> > > >>>> >> >> is preferable in the long term. My
> > suggestion
> > > > >> here is
> > > > >> > > to
> > > > >> > > > >> > > separate
> > > > >> > > > >> > > >>>> the
> > > > >> > > > >> > > >>>> >> >> decision of HOW to do unclean leader
> > election
> > > > from
> > > > >> > the
> > > > >> > > > >> > decision
> > > > >> > > > >> > > of
> > > > >> > > > >> > > >>>> WHEN
> > > > >> > > > >> > > >>>> >> to
> > > > >> > > > >> > > >>>> >> >> do it.
> > > > >> > > > >> > > >>>> >> >>
> > > > >> > > > >> > > >>>> >> >> So in other words, have
> > > > >> > > "unclean.leader.election.enable"
> > > > >> > > > >> > specify
> > > > >> > > > >> > > >>>> when we
> > > > >> > > > >> > > >>>> >> >> do unclean leader election, and have a new
> > > > >> > > configuration
> > > > >> > > > >> like
> > > > >> > > > >> > > >>>> >> >> "unclean.recovery.manager.enable" to
> > determine
> > > > if
> > > > >> we
> > > > >> > > use
> > > > >> > > > the
> > > > >> > > > >> > > URM.
> > > > >> > > > >> > > >>>> >> >> Presumably the URM will take some time to
> > get
> > > > >> fully
> > > > >> > > > stable,
> > > > >> > > > >> so
> > > > >> > > > >> > > >>>> this can
> > > > >> > > > >> > > >>>> >> >> default to false for a while, and we can
> > flip
> > > > the
> > > > >> > > > default to
> > > > >> > > > >> > > true
> > > > >> > > > >> > > >>>> when
> > > > >> > > > >> > > >>>> >> we
> > > > >> > > > >> > > >>>> >> >> feel ready.
> > > > >> > > > >> > > >>>> >> >>
> > > > >> > > > >> > > >>>> >> >> The URM is somewhat under-described here.
> I
> > > > think
> > > > >> we
> > > > >> > > > need a
> > > > >> > > > >> > few
> > > > >> > > > >> > > >>>> >> >> configurations here for it. For example,
> we
> > > > need a
> > > > >> > > > >> > > configuration to
> > > > >> > > > >> > > >>>> >> specify
> > > > >> > > > >> > > >>>> >> >> how long it should wait for a broker to
> > > respond
> > > > to
> > > > >> > its
> > > > >> > > > RPCs
> > > > >> > > > >> > > before
> > > > >> > > > >> > > >>>> >> moving
> > > > >> > > > >> > > >>>> >> >> on. We also need to understand how the URM
> > > > >> interacts
> > > > >> > > with
> > > > >> > > > >> > > >>>> >> >> unclean.leader.election.enable=always. I
> > > assume
> > > > >> that
> > > > >> > > with
> > > > >> > > > >> > > "always"
> > > > >> > > > >> > > >>>> we
> > > > >> > > > >> > > >>>> >> will
> > > > >> > > > >> > > >>>> >> >> just unconditionally use the URM rather
> than
> > > > >> choosing
> > > > >> > > > >> > randomly.
> > > > >> > > > >> > > >>>> But this
> > > > >> > > > >> > > >>>> >> >> should be spelled out in the KIP.
> > > > >> > > > >> > > >>>> >> >>
> > > > >> > > > >> > > >>>> >> >> >
> > > > >> > > > >> > > >>>> >> >> > DescribeTopicRequest
> > > > >> > > > >> > > >>>> >> >> >
> > > > >> > > > >> > > >>>> >> >> >    1.
> > > > >> > > > >> > > >>>> >> >> >    Yes, the plan is to replace the
> > > > >> MetadataRequest
> > > > >> > > with
> > > > >> > > > >> the
> > > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest for the admin
> > > clients.
> > > > >> Will
> > > > >> > > > check
> > > > >> > > > >> > the
> > > > >> > > > >> > > >>>> details.
> > > > >> > > > >> > > >>>> >> >>
> > > > >> > > > >> > > >>>> >> >> Sounds good. But as I said, you need to
> > > specify
> > > > >> how
> > > > >> > > > >> > AdminClient
> > > > >> > > > >> > > >>>> >> interacts
> > > > >> > > > >> > > >>>> >> >> with the new request. This will involve
> > adding
> > > > >> some
> > > > >> > > > fields
> > > > >> > > > >> to
> > > > >> > > > >> > > >>>> >> >> TopicDescription.java. And you need to
> > specify
> > > > the
> > > > >> > > > changes
> > > > >> > > > >> to
> > > > >> > > > >> > > the
> > > > >> > > > >> > > >>>> >> >> kafka-topics.sh command line tool.
> Otherwise
> > > we
> > > > >> > cannot
> > > > >> > > > use
> > > > >> > > > >> the
> > > > >> > > > >> > > >>>> tool to
> > > > >> > > > >> > > >>>> >> see
> > > > >> > > > >> > > >>>> >> >> the new information.
> > > > >> > > > >> > > >>>> >> >>
> > > > >> > > > >> > > >>>> >> >> The new requests, DescribeTopicRequest and
> > > > >> > > > >> > > >>>> GetReplicaLogInfoRequest,
> > > > >> > > > >> > > >>>> >> need
> > > > >> > > > >> > > >>>> >> >> to have limits placed on them so that
> their
> > > size
> > > > >> > can't
> > > > >> > > be
> > > > >> > > > >> > > >>>> infinite. We
> > > > >> > > > >> > > >>>> >> >> don't want to propagate the current
> problems
> > > of
> > > > >> > > > >> > MetadataRequest,
> > > > >> > > > >> > > >>>> where
> > > > >> > > > >> > > >>>> >> >> clients can request massive responses that
> > can
> > > > >> mess
> > > > >> > up
> > > > >> > > > the
> > > > >> > > > >> JVM
> > > > >> > > > >> > > when
> > > > >> > > > >> > > >>>> >> handled.
> > > > >> > > > >> > > >>>> >> >>
> > > > >> > > > >> > > >>>> >> >> Adding limits is simple for
> > > > >> GetReplicaLogInfoRequest
> > > > >> > --
> > > > >> > > > we
> > > > >> > > > >> can
> > > > >> > > > >> > > >>>> just say
> > > > >> > > > >> > > >>>> >> >> that only 2000 partitions at a time can be
> > > > >> requested.
> > > > >> > > For
> > > > >> > > > >> > > >>>> >> >> DescribeTopicRequest we can probably just
> > > limit
> > > > >> to 20
> > > > >> > > > topics
> > > > >> > > > >> > or
> > > > >> > > > >> > > >>>> >> something
> > > > >> > > > >> > > >>>> >> >> like that, to avoid the complexity of
> doing
> > > > >> > pagination
> > > > >> > > in
> > > > >> > > > >> this
> > > > >> > > > >> > > KIP.
> > > > >> > > > >> > > >>>> >> >>
> > > > >> > > > >> > > >>>> >> >> >    2.
> > > > >> > > > >> > > >>>> >> >> >    I can let the broker load the ELR
> info
> > so
> > > > >> that
> > > > >> > > they
> > > > >> > > > can
> > > > >> > > > >> > > serve
> > > > >> > > > >> > > >>>> the
> > > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest as well.
> > > > >> > > > >> > > >>>> >> >>
> > > > >> > > > >> > > >>>> >> >> Yes, it's fine to add to MetadataCache. In
> > > fact,
> > > > >> > you'll
> > > > >> > > > be
> > > > >> > > > >> > > loading
> > > > >> > > > >> > > >>>> it
> > > > >> > > > >> > > >>>> >> >> anyway once it's added to PartitionImage.
> > > > >> > > > >> > > >>>> >> >>
> > > > >> > > > >> > > >>>> >> >> >    3.
> > > > >> > > > >> > > >>>> >> >> >    Yeah, it does not make sense to have
> > the
> > > > >> topic
> > > > >> > id
> > > > >> > > if
> > > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest is only used by
> > the
> > > > >> admin
> > > > >> > > > client.
> > > > >> > > > >> > > >>>> >> >>
> > > > >> > > > >> > > >>>> >> >> OK. That makes things simpler. We can
> always
> > > > >> create a
> > > > >> > > new
> > > > >> > > > >> API
> > > > >> > > > >> > > later
> > > > >> > > > >> > > >>>> >> >> (hopefully not in this KIP!) to query by
> > topic
> > > > ID.
> > > > >> > > > >> > > >>>> >> >>
> > > > >> > > > >> > > >>>> >> >> >
> > > > >> > > > >> > > >>>> >> >> >
> > > > >> > > > >> > > >>>> >> >> > Metrics
> > > > >> > > > >> > > >>>> >> >> >
> > > > >> > > > >> > > >>>> >> >> > As for overall cluster health metrics, I
> > > think
> > > > >> > > > >> under-min-ISR
> > > > >> > > > >> > > is
> > > > >> > > > >> > > >>>> still
> > > > >> > > > >> > > >>>> >> a
> > > > >> > > > >> > > >>>> >> >> > useful one. ELR is more like a safety
> > belt.
> > > > When
> > > > >> > the
> > > > >> > > > ELR
> > > > >> > > > >> is
> > > > >> > > > >> > > >>>> used, the
> > > > >> > > > >> > > >>>> >> >> > cluster availability has already been
> > > > impacted.
> > > > >> > > > >> > > >>>> >> >> >
> > > > >> > > > >> > > >>>> >> >> > Maybe we can have a metric to count the
> > > > >> partitions
> > > > >> > > that
> > > > >> > > > >> > > sum(ISR,
> > > > >> > > > >> > > >>>> ELR)
> > > > >> > > > >> > > >>>> >> <
> > > > >> > > > >> > > >>>> >> >> min
> > > > >> > > > >> > > >>>> >> >> > ISR. What do you think?
> > > > >> > > > >> > > >>>> >> >>
> > > > >> > > > >> > > >>>> >> >> How about:
> > > > >> > > > >> > > >>>> >> >>
> > > > >> > > > >> > > >>>> >> >> A.  a metric for the totoal number of
> > > > >> under-min-isr
> > > > >> > > > >> > partitions?
> > > > >> > > > >> > > We
> > > > >> > > > >> > > >>>> don't
> > > > >> > > > >> > > >>>> >> >> have that in Apache Kafka at the moment.
> > > > >> > > > >> > > >>>> >> >>
> > > > >> > > > >> > > >>>> >> >> B. a metric for the number of unclean
> leader
> > > > >> > elections
> > > > >> > > we
> > > > >> > > > >> did
> > > > >> > > > >> > > (for
> > > > >> > > > >> > > >>>> >> >> simplicity, it can reset to 0 on
> controller
> > > > >> restart:
> > > > >> > we
> > > > >> > > > >> expect
> > > > >> > > > >> > > >>>> people to
> > > > >> > > > >> > > >>>> >> >> monitor the change over time anyway)
> > > > >> > > > >> > > >>>> >> >>
> > > > >> > > > >> > > >>>> >> >> best,
> > > > >> > > > >> > > >>>> >> >> Colin
> > > > >> > > > >> > > >>>> >> >>
> > > > >> > > > >> > > >>>> >> >>
> > > > >> > > > >> > > >>>> >> >> >
> > > > >> > > > >> > > >>>> >> >> > Yeah, for the ongoing unclean
> recoveries,
> > > the
> > > > >> > > > controller
> > > > >> > > > >> can
> > > > >> > > > >> > > >>>> keep an
> > > > >> > > > >> > > >>>> >> >> > accurate count through failover because
> > > > >> partition
> > > > >> > > > >> > registration
> > > > >> > > > >> > > >>>> can
> > > > >> > > > >> > > >>>> >> >> indicate
> > > > >> > > > >> > > >>>> >> >> > whether a recovery is needed. However,
> for
> > > the
> > > > >> > > happened
> > > > >> > > > >> > ones,
> > > > >> > > > >> > > >>>> unless
> > > > >> > > > >> > > >>>> >> we
> > > > >> > > > >> > > >>>> >> >> > want to persist the number somewhere, we
> > can
> > > > >> only
> > > > >> > > > figure
> > > > >> > > > >> it
> > > > >> > > > >> > > out
> > > > >> > > > >> > > >>>> from
> > > > >> > > > >> > > >>>> >> the
> > > > >> > > > >> > > >>>> >> >> > log.
> > > > >> > > > >> > > >>>> >> >> >
> > > > >> > > > >> > > >>>> >> >> > On Tue, Sep 12, 2023 at 3:16 PM Colin
> > > McCabe <
> > > > >> > > > >> > > cmccabe@apache.org
> > > > >> > > > >> > > >>>> >
> > > > >> > > > >> > > >>>> >> wrote:
> > > > >> > > > >> > > >>>> >> >> >
> > > > >> > > > >> > > >>>> >> >> >> Also, we should have metrics that show
> > what
> > > > is
> > > > >> > going
> > > > >> > > > on
> > > > >> > > > >> > with
> > > > >> > > > >> > > >>>> regard
> > > > >> > > > >> > > >>>> >> to
> > > > >> > > > >> > > >>>> >> >> the
> > > > >> > > > >> > > >>>> >> >> >> eligible replica set. I'm not sure
> > exactly
> > > > >> what to
> > > > >> > > > >> suggest,
> > > > >> > > > >> > > but
> > > > >> > > > >> > > >>>> >> >> something
> > > > >> > > > >> > > >>>> >> >> >> that could identify when things are
> going
> > > > >> wrong in
> > > > >> > > the
> > > > >> > > > >> > > clsuter.
> > > > >> > > > >> > > >>>> >> >> >>
> > > > >> > > > >> > > >>>> >> >> >> For example, maybe a metric for
> > partitions
> > > > >> > > containing
> > > > >> > > > >> > > replicas
> > > > >> > > > >> > > >>>> that
> > > > >> > > > >> > > >>>> >> are
> > > > >> > > > >> > > >>>> >> >> >> ineligible to be leader? That would
> show
> > a
> > > > >> spike
> > > > >> > > when
> > > > >> > > > a
> > > > >> > > > >> > > broker
> > > > >> > > > >> > > >>>> had an
> > > > >> > > > >> > > >>>> >> >> >> unclean restart.
> > > > >> > > > >> > > >>>> >> >> >>
> > > > >> > > > >> > > >>>> >> >> >> Ideally, we'd also have a metric that
> > > > indicates
> > > > >> > when
> > > > >> > > > an
> > > > >> > > > >> > > unclear
> > > > >> > > > >> > > >>>> >> leader
> > > > >> > > > >> > > >>>> >> >> >> election or a recovery happened. It's a
> > bit
> > > > >> tricky
> > > > >> > > > >> because
> > > > >> > > > >> > > the
> > > > >> > > > >> > > >>>> simple
> > > > >> > > > >> > > >>>> >> >> >> thing, of tracking it per controller,
> may
> > > be
> > > > a
> > > > >> bit
> > > > >> > > > >> > confusing
> > > > >> > > > >> > > >>>> during
> > > > >> > > > >> > > >>>> >> >> >> failovers.
> > > > >> > > > >> > > >>>> >> >> >>
> > > > >> > > > >> > > >>>> >> >> >> best,
> > > > >> > > > >> > > >>>> >> >> >> Colin
> > > > >> > > > >> > > >>>> >> >> >>
> > > > >> > > > >> > > >>>> >> >> >>
> > > > >> > > > >> > > >>>> >> >> >> On Tue, Sep 12, 2023, at 14:25, Colin
> > > McCabe
> > > > >> > wrote:
> > > > >> > > > >> > > >>>> >> >> >> > Hi Calvin,
> > > > >> > > > >> > > >>>> >> >> >> >
> > > > >> > > > >> > > >>>> >> >> >> > Thanks for the KIP. I think this is a
> > > great
> > > > >> > > > >> improvement.
> > > > >> > > > >> > > >>>> >> >> >> >
> > > > >> > > > >> > > >>>> >> >> >> >> Additional High Watermark advance
> > > > >> requirement
> > > > >> > > > >> > > >>>> >> >> >> >
> > > > >> > > > >> > > >>>> >> >> >> > Typo: change "advance" to
> "advancement"
> > > > >> > > > >> > > >>>> >> >> >> >
> > > > >> > > > >> > > >>>> >> >> >> >> A bit recap of some key concepts.
> > > > >> > > > >> > > >>>> >> >> >> >
> > > > >> > > > >> > > >>>> >> >> >> > Typo: change "bit" to "quick"
> > > > >> > > > >> > > >>>> >> >> >> >
> > > > >> > > > >> > > >>>> >> >> >> >> Ack=1/all produce request. It
> defines
> > > when
> > > > >> the
> > > > >> > > > Kafka
> > > > >> > > > >> > > server
> > > > >> > > > >> > > >>>> should
> > > > >> > > > >> > > >>>> >> >> >> respond to the produce request
> > > > >> > > > >> > > >>>> >> >> >> >
> > > > >> > > > >> > > >>>> >> >> >> > I think this section would be clearer
> > if
> > > we
> > > > >> > talked
> > > > >> > > > >> about
> > > > >> > > > >> > > the
> > > > >> > > > >> > > >>>> new
> > > > >> > > > >> > > >>>> >> high
> > > > >> > > > >> > > >>>> >> >> >> > watermark advancement requirement
> > first,
> > > > and
> > > > >> > THEN
> > > > >> > > > >> talked
> > > > >> > > > >> > > >>>> about its
> > > > >> > > > >> > > >>>> >> >> >> > impact on acks=0, acks=1, and
> > >  acks=all.
> > > > >> > > > acks=all
> > > > >> > > > >> is
> > > > >> > > > >> > of
> > > > >> > > > >> > > >>>> course
> > > > >> > > > >> > > >>>> >> the
> > > > >> > > > >> > > >>>> >> >> >> > main case we care about here, so it
> > would
> > > > be
> > > > >> > good
> > > > >> > > to
> > > > >> > > > >> lead
> > > > >> > > > >> > > with
> > > > >> > > > >> > > >>>> >> that,
> > > > >> > > > >> > > >>>> >> >> >> > rather than delving into the
> > > technicalities
> > > > >> of
> > > > >> > > > acks=0/1
> > > > >> > > > >> > > first.
> > > > >> > > > >> > > >>>> >> >> >> >
> > > > >> > > > >> > > >>>> >> >> >> >> Unclean recovery
> > > > >> > > > >> > > >>>> >> >> >> >
> > > > >> > > > >> > > >>>> >> >> >> > So, here you are introducing a new
> > > > >> > configuration,
> > > > >> > > > >> > > >>>> >> >> >> > unclean.recovery.strategy. The
> > difficult
> > > > >> thing
> > > > >> > > here
> > > > >> > > > is
> > > > >> > > > >> > that
> > > > >> > > > >> > > >>>> there
> > > > >> > > > >> > > >>>> >> is a
> > > > >> > > > >> > > >>>> >> >> >> > lot of overlap with
> > > > >> > > unclean.leader.election.enable.
> > > > >> > > > So
> > > > >> > > > >> we
> > > > >> > > > >> > > >>>> have 3
> > > > >> > > > >> > > >>>> >> >> >> > different settings for
> > > > >> > unclean.recovery.strategy,
> > > > >> > > > plus
> > > > >> > > > >> 2
> > > > >> > > > >> > > >>>> different
> > > > >> > > > >> > > >>>> >> >> >> > settings for
> > > > unclean.leader.election.enable,
> > > > >> > > giving
> > > > >> > > > a
> > > > >> > > > >> > cross
> > > > >> > > > >> > > >>>> >> product of
> > > > >> > > > >> > > >>>> >> >> >> > 6 different options. The following
> > > "unclean
> > > > >> > > recovery
> > > > >> > > > >> > > manager"
> > > > >> > > > >> > > >>>> >> section
> > > > >> > > > >> > > >>>> >> >> >> > on
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by David Jacot <da...@gmail.com>.
Hi Calvin,

I thought that a new snapshot with the downgraded MV is created in this
case. Isn’t it the case?

Could you also elaborate a bit more on the reasoning behind adding the
limits to the admin RPCs? This is a new pattern in Kafka so it would be
good to clear on the motivation.

Could you also explain how the client is supposed to handle the
topics/partitions above the limit? I suppose that it will have to retry
those, correct?

My understanding is that the topics/partitions above the limit will be
failed with an invalid exception error. I wonder if this choice is
judicious because the invalide request exception is usually fatal. It may
be better to use an new and explicit error for this case.

It seems that we still need to specify the changes to the admin api to
accommodate the new or updated apis. Do you plan to add them?

Best,
David

Le mer. 4 oct. 2023 à 20:39, Calvin Liu <ca...@confluent.io.invalid> a
écrit :

> Hi Jun,
> After the MV downgrade, the controller will write in the old version of the
> PartitionRecord/PartitionChangeRecord. If I understand correctly, it is
> possible to downgrade the software version if the controller only has to
> handle old version records.
> However, the controller will not automatically rewrite the PartitionRecord
> with the old version unless there is a partition update. Then, the user may
> have to wait an unknown amount of time before the software downgrades
> unless they do a roll to force update every partition. If it makes sense, I
> can mention these steps to do a software downgrade.
> Thanks
>
> On Wed, Oct 4, 2023 at 11:20 AM Jun Rao <ju...@confluent.io.invalid> wrote:
>
> > Hi, Calvin and Justine,
> >
> > Historically, when we change the record format in the log, we don't
> support
> > software version downgrading.
> >
> > For the record format change in the metadata log, have we thought about
> > forcing the write of the latest metadata records with the old version
> > during MV downgrading? This will in theory allow the old version of the
> > software to obtain the latest metadata.
> >
> > Thanks,
> >
> > Jun
> >
> > On Wed, Oct 4, 2023 at 9:53 AM Justine Olshan
> <jolshan@confluent.io.invalid
> > >
> > wrote:
> >
> > > Sorry -- not MV but software version.
> > >
> > > On Wed, Oct 4, 2023 at 9:51 AM Justine Olshan <jo...@confluent.io>
> > > wrote:
> > >
> > > > Catching up with this discussion.
> > > >
> > > > I was just curious -- have we had other instances where downgrading
> MV
> > is
> > > > not supported? I think Kafka typically tries to support downgrades,
> > and I
> > > > couldn't think of other examples.
> > > >
> > > > Thanks,
> > > > Justine
> > > >
> > > > On Wed, Oct 4, 2023 at 9:40 AM Calvin Liu <caliu@confluent.io.invalid
> >
> > > > wrote:
> > > >
> > > >> Hi Jun,
> > > >> 54. Marked the software downgrading is not supported. As the old
> > > >> controller
> > > >> will not understand the new PartitionRecord and
> PartitionChangeRecord.
> > > >> Thanks!
> > > >>
> > > >> On Wed, Oct 4, 2023 at 9:12 AM Jun Rao <ju...@confluent.io.invalid>
> > > wrote:
> > > >>
> > > >> > Hi, Calvin,
> > > >> >
> > > >> > Thanks for the reply. Just one more comment.
> > > >> >
> > > >> > 54. It seems that downgrading MV is supported. Is downgrading the
> > > >> software
> > > >> > version supported? It would be useful to document that.
> > > >> >
> > > >> > Thanks,
> > > >> >
> > > >> > Jun
> > > >> >
> > > >> > On Tue, Oct 3, 2023 at 4:55 PM Artem Livshits
> > > >> > <al...@confluent.io.invalid> wrote:
> > > >> >
> > > >> > > Hi Colin,
> > > >> > >
> > > >> > > I think in your example "do_unclean_recovery" would need to do
> > > >> different
> > > >> > > things depending on the strategy.
> > > >> > >
> > > >> > > do_unclean_recovery() {
> > > >> > >    if (unclean.recovery.manager.enabled) {
> > > >> > >     if (strategy == Aggressive)
> > > >> > >       use UncleanRecoveryManager(waitLastKnownERL=false)  //
> just
> > > >> inspect
> > > >> > > logs from whoever is available
> > > >> > >     else
> > > >> > >       use  UncleanRecoveryManager(waitLastKnownERL=true)  //
> must
> > > wait
> > > >> > for
> > > >> > > at least last known ELR
> > > >> > >   } else {
> > > >> > >     if (strategy == Aggressive)
> > > >> > >       choose the last known leader if that is available, or a
> > random
> > > >> > leader
> > > >> > > if not)
> > > >> > >     else
> > > >> > >       wait for last known leader to get back
> > > >> > >   }
> > > >> > > }
> > > >> > >
> > > >> > > The idea is that the Aggressive strategy would kick in as soon
> as
> > we
> > > >> lost
> > > >> > > the leader and would pick a leader from whoever is available;
> but
> > > the
> > > >> > > Balanced will only kick in when ELR is empty and will wait for
> the
> > > >> > brokers
> > > >> > > that likely have most data to be available.
> > > >> > >
> > > >> > > On Tue, Oct 3, 2023 at 3:04 PM Colin McCabe <cmccabe@apache.org
> >
> > > >> wrote:
> > > >> > >
> > > >> > > > On Tue, Oct 3, 2023, at 10:49, Jun Rao wrote:
> > > >> > > > > Hi, Calvin,
> > > >> > > > >
> > > >> > > > > Thanks for the update KIP. A few more comments.
> > > >> > > > >
> > > >> > > > > 41. Why would a user choose the option to select a random
> > > replica
> > > >> as
> > > >> > > the
> > > >> > > > > leader instead of using unclean.recovery.strateg=Aggressive?
> > It
> > > >> seems
> > > >> > > > that
> > > >> > > > > the latter is strictly better? If that's not the case, could
> > we
> > > >> fold
> > > >> > > this
> > > >> > > > > option under unclean.recovery.strategy instead of
> introducing
> > a
> > > >> > > separate
> > > >> > > > > config?
> > > >> > > >
> > > >> > > > Hi Jun,
> > > >> > > >
> > > >> > > > I thought the flow of control was:
> > > >> > > >
> > > >> > > > If there is no leader for the partition {
> > > >> > > >   If (there are unfenced ELR members) {
> > > >> > > >     choose_an_unfenced_ELR_member
> > > >> > > >   } else if (there are fenced ELR members AND
> > > strategy=Aggressive) {
> > > >> > > >     do_unclean_recovery
> > > >> > > >   } else if (there are no ELR members AND strategy != None) {
> > > >> > > >     do_unclean_recovery
> > > >> > > >   } else {
> > > >> > > >     do nothing about the missing leader
> > > >> > > >   }
> > > >> > > > }
> > > >> > > >
> > > >> > > > do_unclean_recovery() {
> > > >> > > >    if (unclean.recovery.manager.enabled) {
> > > >> > > >     use UncleanRecoveryManager
> > > >> > > >   } else {
> > > >> > > >     choose the last known leader if that is available, or a
> > random
> > > >> > leader
> > > >> > > > if not)
> > > >> > > >   }
> > > >> > > > }
> > > >> > > >
> > > >> > > > However, I think this could be clarified, especially the
> > behavior
> > > >> when
> > > >> > > > unclean.recovery.manager.enabled=false. Inuitively the goal
> for
> > > >> > > > unclean.recovery.manager.enabled=false is to be "the same as
> > now,
> > > >> > mostly"
> > > >> > > > but it's very underspecified in the KIP, I agree.
> > > >> > > >
> > > >> > > > >
> > > >> > > > > 50. ElectLeadersRequest: "If more than 20 topics are
> included,
> > > >> only
> > > >> > the
> > > >> > > > > first 20 will be served. Others will be returned with
> > > >> > DesiredLeaders."
> > > >> > > > Hmm,
> > > >> > > > > not sure that I understand this. ElectLeadersResponse
> doesn't
> > > >> have a
> > > >> > > > > DesiredLeaders field.
> > > >> > > > >
> > > >> > > > > 51. GetReplicaLogInfo: "If more than 2000 partitions are
> > > included,
> > > >> > only
> > > >> > > > the
> > > >> > > > > first 2000 will be served" Do we return an error for the
> > > remaining
> > > >> > > > > partitions? Actually, should we include an errorCode field
> at
> > > the
> > > >> > > > partition
> > > >> > > > > level in GetReplicaLogInfoResponse to cover non-existing
> > > >> partitions
> > > >> > and
> > > >> > > > no
> > > >> > > > > authorization, etc?
> > > >> > > > >
> > > >> > > > > 52. The entry should matches => The entry should match
> > > >> > > > >
> > > >> > > > > 53. ElectLeadersRequest.DesiredLeaders: Should it be
> nullable
> > > >> since a
> > > >> > > > user
> > > >> > > > > may not specify DesiredLeaders?
> > > >> > > > >
> > > >> > > > > 54. Downgrade: Is that indeed possible? I thought earlier
> you
> > > said
> > > >> > that
> > > >> > > > > once the new version of the records are in the metadata log,
> > one
> > > >> > can't
> > > >> > > > > downgrade since the old broker doesn't know how to parse the
> > new
> > > >> > > version
> > > >> > > > of
> > > >> > > > > the metadata records?
> > > >> > > > >
> > > >> > > >
> > > >> > > > MetadataVersion downgrade is currently broken but we have
> fixing
> > > it
> > > >> on
> > > >> > > our
> > > >> > > > plate for Kafka 3.7.
> > > >> > > >
> > > >> > > > The way downgrade works is that "new features" are dropped,
> > > leaving
> > > >> > only
> > > >> > > > the old ones.
> > > >> > > >
> > > >> > > > > 55. CleanShutdownFile: Should we add a version field for
> > future
> > > >> > > > extension?
> > > >> > > > >
> > > >> > > > > 56. Config changes are public facing. Could we have a
> separate
> > > >> > section
> > > >> > > to
> > > >> > > > > document all the config changes?
> > > >> > > >
> > > >> > > > +1. A separate section for this would be good.
> > > >> > > >
> > > >> > > > best,
> > > >> > > > Colin
> > > >> > > >
> > > >> > > > >
> > > >> > > > > Thanks,
> > > >> > > > >
> > > >> > > > > Jun
> > > >> > > > >
> > > >> > > > > On Mon, Sep 25, 2023 at 4:29 PM Calvin Liu
> > > >> > <caliu@confluent.io.invalid
> > > >> > > >
> > > >> > > > > wrote:
> > > >> > > > >
> > > >> > > > >> Hi Jun
> > > >> > > > >> Thanks for the comments.
> > > >> > > > >>
> > > >> > > > >> 40. If we change to None, it is not guaranteed for no data
> > > loss.
> > > >> For
> > > >> > > > users
> > > >> > > > >> who are not able to validate the data with external
> > resources,
> > > >> > manual
> > > >> > > > >> intervention does not give a better result but a loss of
> > > >> > availability.
> > > >> > > > So
> > > >> > > > >> practically speaking, the Balance mode would be a better
> > > default
> > > >> > > value.
> > > >> > > > >>
> > > >> > > > >> 41. No, it represents how we want to do the unclean leader
> > > >> election.
> > > >> > > If
> > > >> > > > it
> > > >> > > > >> is false, the unclean leader election will be the old
> random
> > > way.
> > > >> > > > >> Otherwise, the unclean recovery will be used.
> > > >> > > > >>
> > > >> > > > >> 42. Good catch. Updated.
> > > >> > > > >>
> > > >> > > > >> 43. Only the first 20 topics will be served. Others will be
> > > >> returned
> > > >> > > > with
> > > >> > > > >> InvalidRequestError
> > > >> > > > >>
> > > >> > > > >> 44. The order matters. The desired leader entries match
> with
> > > the
> > > >> > topic
> > > >> > > > >> partition list by the index.
> > > >> > > > >>
> > > >> > > > >> 45. Thanks! Updated.
> > > >> > > > >>
> > > >> > > > >> 46. Good advice! Updated.
> > > >> > > > >>
> > > >> > > > >> 47.1, updated the comment. Basically it will elect the
> > replica
> > > in
> > > >> > the
> > > >> > > > >> desiredLeader field to be the leader
> > > >> > > > >>
> > > >> > > > >> 47.2 We can let the admin client do the conversion. Using
> the
> > > >> > > > desiredLeader
> > > >> > > > >> field in the json format seems easier for users.
> > > >> > > > >>
> > > >> > > > >> 48. Once the MV version is downgraded, all the ELR related
> > > fields
> > > >> > will
> > > >> > > > be
> > > >> > > > >> removed on the next partition change. The controller will
> > also
> > > >> > ignore
> > > >> > > > the
> > > >> > > > >> ELR fields. Updated the KIP.
> > > >> > > > >>
> > > >> > > > >> 49. Yes, it would be deprecated/removed.
> > > >> > > > >>
> > > >> > > > >>
> > > >> > > > >> On Mon, Sep 25, 2023 at 3:49 PM Jun Rao
> > > <jun@confluent.io.invalid
> > > >> >
> > > >> > > > wrote:
> > > >> > > > >>
> > > >> > > > >> > Hi, Calvin,
> > > >> > > > >> >
> > > >> > > > >> > Thanks for the updated KIP. Made another pass. A few more
> > > >> comments
> > > >> > > > below.
> > > >> > > > >> >
> > > >> > > > >> > 40. unclean.leader.election.enable.false ->
> > > >> > > > >> > unclean.recovery.strategy.Balanced: The Balanced mode
> could
> > > >> still
> > > >> > > > lead to
> > > >> > > > >> > data loss. So, I am wondering if
> > > >> > > unclean.leader.election.enable.false
> > > >> > > > >> > should map to None?
> > > >> > > > >> >
> > > >> > > > >> > 41. unclean.recovery.manager.enabled: I am not sure why
> we
> > > >> > introduce
> > > >> > > > this
> > > >> > > > >> > additional config. Is it the same as
> > > >> > unclean.recovery.strategy=None?
> > > >> > > > >> >
> > > >> > > > >> > 42. DescribeTopicResponse.TopicAuthorizedOperations:
> Should
> > > >> this
> > > >> > be
> > > >> > > at
> > > >> > > > >> the
> > > >> > > > >> > topic level?
> > > >> > > > >> >
> > > >> > > > >> > 43. "Limit: 20 topics max per request": Could we describe
> > > what
> > > >> > > > happens if
> > > >> > > > >> > the request includes more than 20 topics?
> > > >> > > > >> >
> > > >> > > > >> > 44. ElectLeadersRequest.DesiredLeaders: Could we describe
> > > >> whether
> > > >> > > the
> > > >> > > > >> > ordering matters?
> > > >> > > > >> >
> > > >> > > > >> > 45. GetReplicaLogInfo.TopicPartitions: "about": "The
> topic
> > > >> > > partitions
> > > >> > > > to
> > > >> > > > >> > elect leaders.": The description in "about" is incorrect.
> > > >> > > > >> >
> > > >> > > > >> > 46. GetReplicaLogInfoResponse: Should we nest partitions
> > > under
> > > >> > > > topicId to
> > > >> > > > >> > be consistent with other types of responses?
> > > >> > > > >> >
> > > >> > > > >> > 47. kafka-leader-election.sh:
> > > >> > > > >> > 47.1 Could we explain DESIGNATION?
> > > >> > > > >> > 47.2 desiredLeader: Should it be a list to match the
> field
> > in
> > > >> > > > >> > ElectLeadersRequest?
> > > >> > > > >> >
> > > >> > > > >> > 48. We could add a section on downgrade?
> > > >> > > > >> >
> > > >> > > > >> > 49. LastKnownLeader: This seems only needed in the first
> > > phase
> > > >> of
> > > >> > > > >> > delivering ELR. Will it be removed when the complete KIP
> is
> > > >> > > delivered?
> > > >> > > > >> >
> > > >> > > > >> > Thanks,
> > > >> > > > >> >
> > > >> > > > >> > Jun
> > > >> > > > >> >
> > > >> > > > >> > On Tue, Sep 19, 2023 at 1:30 PM Colin McCabe <
> > > >> cmccabe@apache.org>
> > > >> > > > wrote:
> > > >> > > > >> >
> > > >> > > > >> > > Hi Calvin,
> > > >> > > > >> > >
> > > >> > > > >> > > Thanks for the explanations. I like the idea of using
> > none,
> > > >> > > > balanced,
> > > >> > > > >> > > aggressive. We also had an offline discussion about why
> > it
> > > is
> > > >> > good
> > > >> > > > to
> > > >> > > > >> > use a
> > > >> > > > >> > > new config key (basically, so that we can deprecate the
> > old
> > > >> one
> > > >> > > > which
> > > >> > > > >> had
> > > >> > > > >> > > only false/true values in 4.0) With these changes, I am
> > +1.
> > > >> > > > >> > >
> > > >> > > > >> > > best,
> > > >> > > > >> > > Colin
> > > >> > > > >> > >
> > > >> > > > >> > > On Mon, Sep 18, 2023, at 15:54, Calvin Liu wrote:
> > > >> > > > >> > > > Hi Colin,
> > > >> > > > >> > > > Also, can we deprecate unclean.leader.election.enable
> > in
> > > >> 4.0?
> > > >> > > > Before
> > > >> > > > >> > > that,
> > > >> > > > >> > > > we can have both the config unclean.recovery.strategy
> > and
> > > >> > > > >> > > > unclean.leader.election.enable
> > > >> > > > >> > > > and using the unclean.recovery.Enabled to determine
> > which
> > > >> > config
> > > >> > > > to
> > > >> > > > >> use
> > > >> > > > >> > > > during the unclean leader election.
> > > >> > > > >> > > >
> > > >> > > > >> > > > On Mon, Sep 18, 2023 at 3:51 PM Calvin Liu <
> > > >> > caliu@confluent.io>
> > > >> > > > >> wrote:
> > > >> > > > >> > > >
> > > >> > > > >> > > >> Hi Colin,
> > > >> > > > >> > > >> For the unclean.recovery.strategy config name, how
> > about
> > > >> we
> > > >> > use
> > > >> > > > the
> > > >> > > > >> > > >> following
> > > >> > > > >> > > >> None. It basically means no unclean recovery will be
> > > >> > performed.
> > > >> > > > >> > > >> Aggressive. It means availability goes first.
> Whenever
> > > the
> > > >> > > > partition
> > > >> > > > >> > > can't
> > > >> > > > >> > > >> elect a durable replica, the controller will try the
> > > >> unclean
> > > >> > > > >> recovery.
> > > >> > > > >> > > >> Balanced. It is the balance point of the
> availability
> > > >> > > > >> > first(Aggressive)
> > > >> > > > >> > > >> and least availability(None). The controller
> performs
> > > >> unclean
> > > >> > > > >> recovery
> > > >> > > > >> > > when
> > > >> > > > >> > > >> both ISR and ELR are empty.
> > > >> > > > >> > > >>
> > > >> > > > >> > > >>
> > > >> > > > >> > > >> On Fri, Sep 15, 2023 at 11:42 AM Calvin Liu <
> > > >> > > caliu@confluent.io>
> > > >> > > > >> > wrote:
> > > >> > > > >> > > >>
> > > >> > > > >> > > >>> Hi Colin,
> > > >> > > > >> > > >>>
> > > >> > > > >> > > >>> > So, the proposal is that if someone sets
> > > >> > > > >> > > "unclean.leader.election.enable
> > > >> > > > >> > > >>> = true"...
> > > >> > > > >> > > >>>
> > > >> > > > >> > > >>>
> > > >> > > > >> > > >>> The idea is to use one of the
> > > >> unclean.leader.election.enable
> > > >> > > and
> > > >> > > > >> > > >>> unclean.recovery.strategy based on the
> > > >> > > > unclean.recovery.Enabled. A
> > > >> > > > >> > > possible
> > > >> > > > >> > > >>> version can be
> > > >> > > > >> > > >>>
> > > >> > > > >> > > >>> If unclean.recovery.Enabled:
> > > >> > > > >> > > >>>
> > > >> > > > >> > > >>> {
> > > >> > > > >> > > >>>
> > > >> > > > >> > > >>> Check unclean.recovery.strategy. If set, use it.
> > > >> Otherwise,
> > > >> > > > check
> > > >> > > > >> > > >>> unclean.leader.election.enable and translate it to
> > > >> > > > >> > > >>> unclean.recovery.strategy.
> > > >> > > > >> > > >>>
> > > >> > > > >> > > >>> } else {
> > > >> > > > >> > > >>>
> > > >> > > > >> > > >>> Use unclean.leader.election.enable
> > > >> > > > >> > > >>>
> > > >> > > > >> > > >>> }
> > > >> > > > >> > > >>>
> > > >> > > > >> > > >>>
> > > >> > > > >> > > >>> —--------
> > > >> > > > >> > > >>>
> > > >> > > > >> > > >>> >The configuration key should be
> > > >> > > > >> "unclean.recovery.manager.enabled",
> > > >> > > > >> > > >>> right?
> > > >> > > > >> > > >>>
> > > >> > > > >> > > >>>
> > > >> > > > >> > > >>> I think we have two ways of choosing a leader
> > > uncleanly,
> > > >> > > unclean
> > > >> > > > >> > leader
> > > >> > > > >> > > >>> election and unclean recovery(log inspection) and
> we
> > > try
> > > >> to
> > > >> > > > switch
> > > >> > > > >> > > between
> > > >> > > > >> > > >>> them.
> > > >> > > > >> > > >>>
> > > >> > > > >> > > >>> Do you mean we want to develop two ways of
> performing
> > > the
> > > >> > > > unclean
> > > >> > > > >> > > >>> recovery and one of them is using “unclean recovery
> > > >> > manager”?
> > > >> > > I
> > > >> > > > >> guess
> > > >> > > > >> > > we
> > > >> > > > >> > > >>> haven’t discussed the second way.
> > > >> > > > >> > > >>>
> > > >> > > > >> > > >>>
> > > >> > > > >> > > >>> —-------
> > > >> > > > >> > > >>>
> > > >> > > > >> > > >>> >How do these 4 levels of overrides interact with
> > your
> > > >> new
> > > >> > > > >> > > >>> configurations?
> > > >> > > > >> > > >>>
> > > >> > > > >> > > >>>
> > > >> > > > >> > > >>> I do notice in the Kraft controller code, the
> method
> > to
> > > >> > check
> > > >> > > > >> whether
> > > >> > > > >> > > >>> perform unclean leader election is hard coded to
> > false
> > > >> since
> > > >> > > > >> > > >>> 2021(uncleanLeaderElectionEnabledForTopic). Isn’t
> it
> > a
> > > >> good
> > > >> > > > chance
> > > >> > > > >> to
> > > >> > > > >> > > >>> completely deprecate the
> > > unclean.leader.election.enable?
> > > >> We
> > > >> > > > don’t
> > > >> > > > >> > even
> > > >> > > > >> > > have
> > > >> > > > >> > > >>> to worry about the config conversion.
> > > >> > > > >> > > >>>
> > > >> > > > >> > > >>> On the other hand, whatever the override is, as
> long
> > as
> > > >> the
> > > >> > > > >> > controller
> > > >> > > > >> > > >>> can have the final effective
> > > >> unclean.leader.election.enable,
> > > >> > > the
> > > >> > > > >> > topic
> > > >> > > > >> > > >>> level config unclean.recovery.strategy, the cluster
> > > level
> > > >> > > config
> > > >> > > > >> > > >>> unclean.recovery.Enabled, the controller can
> > calculate
> > > >> the
> > > >> > > > correct
> > > >> > > > >> > > methods
> > > >> > > > >> > > >>> to use right?
> > > >> > > > >> > > >>>
> > > >> > > > >> > > >>>
> > > >> > > > >> > > >>> On Fri, Sep 15, 2023 at 10:02 AM Colin McCabe <
> > > >> > > > cmccabe@apache.org>
> > > >> > > > >> > > wrote:
> > > >> > > > >> > > >>>
> > > >> > > > >> > > >>>> On Thu, Sep 14, 2023, at 22:23, Calvin Liu wrote:
> > > >> > > > >> > > >>>> > Hi Colin
> > > >> > > > >> > > >>>> > 1. I think using the new config name is more
> > clear.
> > > >> > > > >> > > >>>> >        a. The unclean leader election is
> actually
> > > >> removed
> > > >> > > if
> > > >> > > > >> > unclean
> > > >> > > > >> > > >>>> > recovery is in use.
> > > >> > > > >> > > >>>> >        b. Using multiple values in
> > > >> > > > >> unclean.leader.election.enable
> > > >> > > > >> > is
> > > >> > > > >> > > >>>> > confusing and it will be more confusing after
> > people
> > > >> > forget
> > > >> > > > >> about
> > > >> > > > >> > > this
> > > >> > > > >> > > >>>> > discussion.
> > > >> > > > >> > > >>>>
> > > >> > > > >> > > >>>> Hi Calvin,
> > > >> > > > >> > > >>>>
> > > >> > > > >> > > >>>> So, the proposal is that if someone sets
> > > >> > > > >> > > "unclean.leader.election.enable
> > > >> > > > >> > > >>>> = true" but then sets one of your new
> > configurations,
> > > >> the
> > > >> > > > value of
> > > >> > > > >> > > >>>> unclean.leader.election.enable is ignored? That
> > seems
> > > >> less
> > > >> > > > clear
> > > >> > > > >> to
> > > >> > > > >> > > me, not
> > > >> > > > >> > > >>>> more. Just in general, having multiple
> configuration
> > > >> keys
> > > >> > to
> > > >> > > > >> control
> > > >> > > > >> > > the
> > > >> > > > >> > > >>>> same thing confuses users. Basically, they are
> > sitting
> > > >> at a
> > > >> > > > giant
> > > >> > > > >> > > control
> > > >> > > > >> > > >>>> panel, and some of the levers do nothing.
> > > >> > > > >> > > >>>>
> > > >> > > > >> > > >>>> > 2. Sorry I forgot to mention in the response
> that
> > I
> > > >> did
> > > >> > add
> > > >> > > > the
> > > >> > > > >> > > >>>> > unclean.recovery.Enabled flag.
> > > >> > > > >> > > >>>>
> > > >> > > > >> > > >>>> The configuration key should be
> > > >> > > > >> "unclean.recovery.manager.enabled",
> > > >> > > > >> > > >>>> right? Becuase we can do "unclean recovery"
> without
> > > the
> > > >> > > > manager.
> > > >> > > > >> > > Disabling
> > > >> > > > >> > > >>>> the manager just means we use a different
> mechanism
> > > for
> > > >> > > > recovery.
> > > >> > > > >> > > >>>>
> > > >> > > > >> > > >>>> >        c. Maybe I underestimated the challenge
> of
> > > >> > replacing
> > > >> > > > the
> > > >> > > > >> > > >>>> config. Any
> > > >> > > > >> > > >>>> > implementation problems ahead?
> > > >> > > > >> > > >>>>
> > > >> > > > >> > > >>>> There are four levels of overrides for
> > > >> > > > >> > unclean.leader.election.enable.
> > > >> > > > >> > > >>>>
> > > >> > > > >> > > >>>> 1. static configuration for node.
> > > >> > > > >> > > >>>>     This goes in the configuration file, typically
> > > named
> > > >> > > > >> > > >>>> server.properties
> > > >> > > > >> > > >>>>
> > > >> > > > >> > > >>>> 2. dynamic configuration for node default
> > > >> > > > >> > > >>>>   ConfigResource(type=BROKER, name="")
> > > >> > > > >> > > >>>>
> > > >> > > > >> > > >>>> 3. dynamic configuration for node
> > > >> > > > >> > > >>>>   ConfigResource(type=BROKER, name=<controller
> id>)
> > > >> > > > >> > > >>>>
> > > >> > > > >> > > >>>> 4. dynamic configuration for topic
> > > >> > > > >> > > >>>>   ConfigResource(type=TOPIC, name=<topic-name>)
> > > >> > > > >> > > >>>>
> > > >> > > > >> > > >>>> How do these 4 levels of overrides interact with
> > your
> > > >> new
> > > >> > > > >> > > >>>> configurations? If the new configurations dominate
> > > over
> > > >> the
> > > >> > > old
> > > >> > > > >> > ones,
> > > >> > > > >> > > it
> > > >> > > > >> > > >>>> seems like this will get a lot more confusing to
> > > >> implement
> > > >> > > (and
> > > >> > > > >> also
> > > >> > > > >> > > to
> > > >> > > > >> > > >>>> use.)
> > > >> > > > >> > > >>>>
> > > >> > > > >> > > >>>> Again, I'd recommend just adding some new values
> to
> > > >> > > > >> > > >>>> unclean.leader.election.enable. It's simple and
> will
> > > >> > prevent
> > > >> > > > user
> > > >> > > > >> > > confusion
> > > >> > > > >> > > >>>> (as well as developer confusion.)
> > > >> > > > >> > > >>>>
> > > >> > > > >> > > >>>> best,
> > > >> > > > >> > > >>>> Colin
> > > >> > > > >> > > >>>>
> > > >> > > > >> > > >>>>
> > > >> > > > >> > > >>>> > 3. About the admin client, I mentioned 3 changes
> > in
> > > >> the
> > > >> > > > client.
> > > >> > > > >> > > >>>> Anything
> > > >> > > > >> > > >>>> > else I missed in the KIP?
> > > >> > > > >> > > >>>> >       a. The client will switch to using the new
> > RPC
> > > >> > > instead
> > > >> > > > of
> > > >> > > > >> > > >>>> > MetadataRequest for the topics.
> > > >> > > > >> > > >>>> >       b. The TopicPartitionInfo used in
> > > >> TopicDescription
> > > >> > > > needs
> > > >> > > > >> to
> > > >> > > > >> > > add
> > > >> > > > >> > > >>>> new
> > > >> > > > >> > > >>>> > fields related to the ELR.
> > > >> > > > >> > > >>>> >       c. The outputs will add the ELR related
> > > fields.
> > > >> > > > >> > > >>>> >
> > > >> > > > >> > > >>>> > On Thu, Sep 14, 2023 at 9:19 PM Colin McCabe <
> > > >> > > > >> cmccabe@apache.org>
> > > >> > > > >> > > >>>> wrote:
> > > >> > > > >> > > >>>> >
> > > >> > > > >> > > >>>> >> Hi Calvin,
> > > >> > > > >> > > >>>> >>
> > > >> > > > >> > > >>>> >> Thanks for the changes.
> > > >> > > > >> > > >>>> >>
> > > >> > > > >> > > >>>> >> 1. Earlier I commented that creating
> > > >> > > > >> "unclean.recovery.strategy "
> > > >> > > > >> > > is
> > > >> > > > >> > > >>>> not
> > > >> > > > >> > > >>>> >> necessary, and we can just reuse the existing
> > > >> > > > >> > > >>>> >> "unclean.leader.election.enable" configuration
> > key.
> > > >> > Let's
> > > >> > > > >> discuss
> > > >> > > > >> > > >>>> that.
> > > >> > > > >> > > >>>> >>
> > > >> > > > >> > > >>>> >> 2.I also don't understand why you didn't add a
> > > >> > > > configuration to
> > > >> > > > >> > > >>>> enable or
> > > >> > > > >> > > >>>> >> disable the Unclean Recovery Manager. This
> seems
> > > >> like a
> > > >> > > very
> > > >> > > > >> > simple
> > > >> > > > >> > > >>>> way to
> > > >> > > > >> > > >>>> >> handle the staging issue which we discussed.
> The
> > > URM
> > > >> can
> > > >> > > > just
> > > >> > > > >> be
> > > >> > > > >> > > >>>> turned off
> > > >> > > > >> > > >>>> >> until it is production ready. Let's discuss
> this.
> > > >> > > > >> > > >>>> >>
> > > >> > > > >> > > >>>> >> 3. You still need to describe the changes to
> > > >> AdminClient
> > > >> > > > that
> > > >> > > > >> are
> > > >> > > > >> > > >>>> needed
> > > >> > > > >> > > >>>> >> to use DescribeTopicRequest.
> > > >> > > > >> > > >>>> >>
> > > >> > > > >> > > >>>> >> Keep at it. It's looking better. :)
> > > >> > > > >> > > >>>> >>
> > > >> > > > >> > > >>>> >> best,
> > > >> > > > >> > > >>>> >> Colin
> > > >> > > > >> > > >>>> >>
> > > >> > > > >> > > >>>> >>
> > > >> > > > >> > > >>>> >> On Thu, Sep 14, 2023, at 11:03, Calvin Liu
> wrote:
> > > >> > > > >> > > >>>> >> > Hi Colin
> > > >> > > > >> > > >>>> >> > Thanks for the comments!
> > > >> > > > >> > > >>>> >> >
> > > >> > > > >> > > >>>> >> > I did the following changes
> > > >> > > > >> > > >>>> >> >
> > > >> > > > >> > > >>>> >> >    1.
> > > >> > > > >> > > >>>> >> >
> > > >> > > > >> > > >>>> >> >    Simplified the API spec section to only
> > > include
> > > >> the
> > > >> > > > diff.
> > > >> > > > >> > > >>>> >> >    2.
> > > >> > > > >> > > >>>> >> >
> > > >> > > > >> > > >>>> >> >    Reordered the HWM requirement section.
> > > >> > > > >> > > >>>> >> >    3.
> > > >> > > > >> > > >>>> >> >
> > > >> > > > >> > > >>>> >> >    Removed the URM implementation details to
> > keep
> > > >> the
> > > >> > > > >> necessary
> > > >> > > > >> > > >>>> >> >    characteristics to perform the unclean
> > > recovery.
> > > >> > > > >> > > >>>> >> >    1.
> > > >> > > > >> > > >>>> >> >
> > > >> > > > >> > > >>>> >> >       When to perform the unclean recovery
> > > >> > > > >> > > >>>> >> >       2.
> > > >> > > > >> > > >>>> >> >
> > > >> > > > >> > > >>>> >> >       Under different config, how the unclean
> > > >> recovery
> > > >> > > > finds
> > > >> > > > >> > the
> > > >> > > > >> > > >>>> leader.
> > > >> > > > >> > > >>>> >> >       3.
> > > >> > > > >> > > >>>> >> >
> > > >> > > > >> > > >>>> >> >       How the config
> > > unclean.leader.election.enable
> > > >> > and
> > > >> > > > >> > > >>>> >> >       unclean.recovery.strategy are converted
> > > when
> > > >> > users
> > > >> > > > >> > > >>>> enable/disable
> > > >> > > > >> > > >>>> >> the
> > > >> > > > >> > > >>>> >> >       unclean recovery.
> > > >> > > > >> > > >>>> >> >       4.
> > > >> > > > >> > > >>>> >> >
> > > >> > > > >> > > >>>> >> >    More details about how we change admin
> > client.
> > > >> > > > >> > > >>>> >> >    5.
> > > >> > > > >> > > >>>> >> >
> > > >> > > > >> > > >>>> >> >    API limits on the GetReplicaLogInfoRequest
> > and
> > > >> > > > >> > > >>>> DescribeTopicRequest.
> > > >> > > > >> > > >>>> >> >    6.
> > > >> > > > >> > > >>>> >> >
> > > >> > > > >> > > >>>> >> >    Two metrics added
> > > >> > > > >> > > >>>> >> >    1.
> > > >> > > > >> > > >>>> >> >
> > > >> > > > >> > > >>>> >> >
> > > >> > > >  Kafka.controller.global_under_min_isr_partition_count
> > > >> > > > >> > > >>>> >> >       2.
> > > >> > > > >> > > >>>> >> >
> > > >> > > > >> > > >>>> >> >
> > > >>  kafka.controller.unclean_recovery_finished_count
> > > >> > > > >> > > >>>> >> >
> > > >> > > > >> > > >>>> >> >
> > > >> > > > >> > > >>>> >> > On Wed, Sep 13, 2023 at 10:46 AM Colin
> McCabe <
> > > >> > > > >> > > cmccabe@apache.org>
> > > >> > > > >> > > >>>> >> wrote:
> > > >> > > > >> > > >>>> >> >
> > > >> > > > >> > > >>>> >> >> On Tue, Sep 12, 2023, at 17:21, Calvin Liu
> > > wrote:
> > > >> > > > >> > > >>>> >> >> > Hi Colin
> > > >> > > > >> > > >>>> >> >> > Thanks for the comments!
> > > >> > > > >> > > >>>> >> >> >
> > > >> > > > >> > > >>>> >> >>
> > > >> > > > >> > > >>>> >> >> Hi Calvin,
> > > >> > > > >> > > >>>> >> >>
> > > >> > > > >> > > >>>> >> >> Thanks again for the KIP.
> > > >> > > > >> > > >>>> >> >>
> > > >> > > > >> > > >>>> >> >> One meta-comment: it's usually better to
> just
> > > do a
> > > >> > diff
> > > >> > > > on a
> > > >> > > > >> > > >>>> message
> > > >> > > > >> > > >>>> >> spec
> > > >> > > > >> > > >>>> >> >> file or java file if you're including
> changes
> > to
> > > >> it
> > > >> > in
> > > >> > > > the
> > > >> > > > >> > KIP.
> > > >> > > > >> > > >>>> This is
> > > >> > > > >> > > >>>> >> >> easier to read than looking for "new fields
> > > begin"
> > > >> > etc.
> > > >> > > > in
> > > >> > > > >> the
> > > >> > > > >> > > >>>> text, and
> > > >> > > > >> > > >>>> >> >> gracefully handles the case where existing
> > > fields
> > > >> > were
> > > >> > > > >> > changed.
> > > >> > > > >> > > >>>> >> >>
> > > >> > > > >> > > >>>> >> >> > Rewrite the Additional High Watermark
> > > >> advancement
> > > >> > > > >> > requirement
> > > >> > > > >> > > >>>> >> >> > There was feedback on this section that
> some
> > > >> > readers
> > > >> > > > may
> > > >> > > > >> not
> > > >> > > > >> > > be
> > > >> > > > >> > > >>>> >> familiar
> > > >> > > > >> > > >>>> >> >> > with HWM and Ack=0,1,all requests. This
> can
> > > help
> > > >> > them
> > > >> > > > >> > > understand
> > > >> > > > >> > > >>>> the
> > > >> > > > >> > > >>>> >> >> > proposal. I will rewrite this part for
> more
> > > >> > > > readability.
> > > >> > > > >> > > >>>> >> >> >
> > > >> > > > >> > > >>>> >> >>
> > > >> > > > >> > > >>>> >> >> To be clear, I wasn't suggesting dropping
> > either
> > > >> > > > section. I
> > > >> > > > >> > > agree
> > > >> > > > >> > > >>>> that
> > > >> > > > >> > > >>>> >> >> they add useful background. I was just
> > > suggesting
> > > >> > that
> > > >> > > we
> > > >> > > > >> > should
> > > >> > > > >> > > >>>> discuss
> > > >> > > > >> > > >>>> >> >> the "acks" setting AFTER discussing the new
> > high
> > > >> > > > watermark
> > > >> > > > >> > > >>>> advancement
> > > >> > > > >> > > >>>> >> >> conditions. We also should discuss acks=0.
> > While
> > > >> it
> > > >> > > isn't
> > > >> > > > >> > > >>>> conceptually
> > > >> > > > >> > > >>>> >> much
> > > >> > > > >> > > >>>> >> >> different than acks=1 here, its omission
> from
> > > this
> > > >> > > > section
> > > >> > > > >> is
> > > >> > > > >> > > >>>> confusing.
> > > >> > > > >> > > >>>> >> >>
> > > >> > > > >> > > >>>> >> >> > Unclean recovery
> > > >> > > > >> > > >>>> >> >> >
> > > >> > > > >> > > >>>> >> >> > The plan is to replace the
> > > >> > > > unclean.leader.election.enable
> > > >> > > > >> > with
> > > >> > > > >> > > >>>> >> >> > unclean.recovery.strategy. If the Unclean
> > > >> Recovery
> > > >> > is
> > > >> > > > >> > enabled
> > > >> > > > >> > > >>>> then it
> > > >> > > > >> > > >>>> >> >> deals
> > > >> > > > >> > > >>>> >> >> > with the three options in the
> > > >> > > > unclean.recovery.strategy.
> > > >> > > > >> > > >>>> >> >> >
> > > >> > > > >> > > >>>> >> >> >
> > > >> > > > >> > > >>>> >> >> > Let’s refine the Unclean Recovery. We have
> > > >> already
> > > >> > > > taken a
> > > >> > > > >> > > lot of
> > > >> > > > >> > > >>>> >> >> > suggestions and I hope to enhance the
> > > >> durability of
> > > >> > > > Kafka
> > > >> > > > >> to
> > > >> > > > >> > > the
> > > >> > > > >> > > >>>> next
> > > >> > > > >> > > >>>> >> >> level
> > > >> > > > >> > > >>>> >> >> > with this KIP.
> > > >> > > > >> > > >>>> >> >>
> > > >> > > > >> > > >>>> >> >> I am OK with doing the unclean leader
> recovery
> > > >> > > > improvements
> > > >> > > > >> in
> > > >> > > > >> > > >>>> this KIP.
> > > >> > > > >> > > >>>> >> >> However, I think we need to really work on
> the
> > > >> > > > configuration
> > > >> > > > >> > > >>>> settings.
> > > >> > > > >> > > >>>> >> >>
> > > >> > > > >> > > >>>> >> >> Configuration overrides are often quite
> messy.
> > > For
> > > >> > > > example,
> > > >> > > > >> > the
> > > >> > > > >> > > >>>> cases
> > > >> > > > >> > > >>>> >> >> where we have log.roll.hours and
> > > >> log.roll.segment.ms
> > > >> > ,
> > > >> > > > the
> > > >> > > > >> > user
> > > >> > > > >> > > >>>> has to
> > > >> > > > >> > > >>>> >> >> remember which one takes precedence, and it
> is
> > > not
> > > >> > > > obvious.
> > > >> > > > >> > So,
> > > >> > > > >> > > >>>> rather
> > > >> > > > >> > > >>>> >> than
> > > >> > > > >> > > >>>> >> >> creating a new configuration, why not add
> > > >> additional
> > > >> > > > values
> > > >> > > > >> to
> > > >> > > > >> > > >>>> >> >> "unclean.leader.election.enable"? I think
> this
> > > >> will
> > > >> > be
> > > >> > > > >> simpler
> > > >> > > > >> > > for
> > > >> > > > >> > > >>>> >> people
> > > >> > > > >> > > >>>> >> >> to understand, and simpler in the code as
> > well.
> > > >> > > > >> > > >>>> >> >>
> > > >> > > > >> > > >>>> >> >> What if we continued to use
> > > >> > > > "unclean.leader.election.enable"
> > > >> > > > >> > but
> > > >> > > > >> > > >>>> >> extended
> > > >> > > > >> > > >>>> >> >> it so that it took a string? Then the string
> > > could
> > > >> > have
> > > >> > > > >> these
> > > >> > > > >> > > >>>> values:
> > > >> > > > >> > > >>>> >> >>
> > > >> > > > >> > > >>>> >> >> never
> > > >> > > > >> > > >>>> >> >>     never automatically do an unclean leader
> > > >> election
> > > >> > > > under
> > > >> > > > >> > any
> > > >> > > > >> > > >>>> >> conditions
> > > >> > > > >> > > >>>> >> >>
> > > >> > > > >> > > >>>> >> >> false / default
> > > >> > > > >> > > >>>> >> >>     only do an unclean leader election if
> > there
> > > >> may
> > > >> > be
> > > >> > > > >> > possible
> > > >> > > > >> > > >>>> data
> > > >> > > > >> > > >>>> >> loss
> > > >> > > > >> > > >>>> >> >>
> > > >> > > > >> > > >>>> >> >> true / always
> > > >> > > > >> > > >>>> >> >>     always do an unclean leader election if
> we
> > > >> can't
> > > >> > > > >> > immediately
> > > >> > > > >> > > >>>> elect a
> > > >> > > > >> > > >>>> >> >> leader
> > > >> > > > >> > > >>>> >> >>
> > > >> > > > >> > > >>>> >> >> It's a bit awkward that false maps to
> default
> > > >> rather
> > > >> > > > than to
> > > >> > > > >> > > >>>> never. But
> > > >> > > > >> > > >>>> >> >> this awkwardness exists if we use two
> > different
> > > >> > > > >> configuration
> > > >> > > > >> > > keys
> > > >> > > > >> > > >>>> as
> > > >> > > > >> > > >>>> >> well.
> > > >> > > > >> > > >>>> >> >> The reason for the awkwardness is that we
> > simply
> > > >> > don't
> > > >> > > > want
> > > >> > > > >> > most
> > > >> > > > >> > > >>>> of the
> > > >> > > > >> > > >>>> >> >> people currently setting
> > > >> > > > >> unclean.leader.election.enable=false
> > > >> > > > >> > to
> > > >> > > > >> > > >>>> get the
> > > >> > > > >> > > >>>> >> >> "never" behavior. We have to bite that
> bullet.
> > > >> Better
> > > >> > > to
> > > >> > > > be
> > > >> > > > >> > > clear
> > > >> > > > >> > > >>>> and
> > > >> > > > >> > > >>>> >> >> explicit than hide it.
> > > >> > > > >> > > >>>> >> >>
> > > >> > > > >> > > >>>> >> >> Another thing that's a bit awkward is having
> > two
> > > >> > > > different
> > > >> > > > >> > ways
> > > >> > > > >> > > to
> > > >> > > > >> > > >>>> do
> > > >> > > > >> > > >>>> >> >> unclean leader election specified in the
> KIP.
> > > You
> > > >> > > > descirbe
> > > >> > > > >> two
> > > >> > > > >> > > >>>> methods:
> > > >> > > > >> > > >>>> >> the
> > > >> > > > >> > > >>>> >> >> simple "choose the last leader" method, and
> > the
> > > >> > > "unclean
> > > >> > > > >> > > recovery
> > > >> > > > >> > > >>>> >> manager"
> > > >> > > > >> > > >>>> >> >> method. I understand why you did it this way
> > --
> > > >> > "choose
> > > >> > > > the
> > > >> > > > >> > last
> > > >> > > > >> > > >>>> >> leader" is
> > > >> > > > >> > > >>>> >> >> simple, and will help us deliver an
> > > implementation
> > > >> > > > quickly,
> > > >> > > > >> > > while
> > > >> > > > >> > > >>>> the
> > > >> > > > >> > > >>>> >> URM
> > > >> > > > >> > > >>>> >> >> is preferable in the long term. My
> suggestion
> > > >> here is
> > > >> > > to
> > > >> > > > >> > > separate
> > > >> > > > >> > > >>>> the
> > > >> > > > >> > > >>>> >> >> decision of HOW to do unclean leader
> election
> > > from
> > > >> > the
> > > >> > > > >> > decision
> > > >> > > > >> > > of
> > > >> > > > >> > > >>>> WHEN
> > > >> > > > >> > > >>>> >> to
> > > >> > > > >> > > >>>> >> >> do it.
> > > >> > > > >> > > >>>> >> >>
> > > >> > > > >> > > >>>> >> >> So in other words, have
> > > >> > > "unclean.leader.election.enable"
> > > >> > > > >> > specify
> > > >> > > > >> > > >>>> when we
> > > >> > > > >> > > >>>> >> >> do unclean leader election, and have a new
> > > >> > > configuration
> > > >> > > > >> like
> > > >> > > > >> > > >>>> >> >> "unclean.recovery.manager.enable" to
> determine
> > > if
> > > >> we
> > > >> > > use
> > > >> > > > the
> > > >> > > > >> > > URM.
> > > >> > > > >> > > >>>> >> >> Presumably the URM will take some time to
> get
> > > >> fully
> > > >> > > > stable,
> > > >> > > > >> so
> > > >> > > > >> > > >>>> this can
> > > >> > > > >> > > >>>> >> >> default to false for a while, and we can
> flip
> > > the
> > > >> > > > default to
> > > >> > > > >> > > true
> > > >> > > > >> > > >>>> when
> > > >> > > > >> > > >>>> >> we
> > > >> > > > >> > > >>>> >> >> feel ready.
> > > >> > > > >> > > >>>> >> >>
> > > >> > > > >> > > >>>> >> >> The URM is somewhat under-described here. I
> > > think
> > > >> we
> > > >> > > > need a
> > > >> > > > >> > few
> > > >> > > > >> > > >>>> >> >> configurations here for it. For example, we
> > > need a
> > > >> > > > >> > > configuration to
> > > >> > > > >> > > >>>> >> specify
> > > >> > > > >> > > >>>> >> >> how long it should wait for a broker to
> > respond
> > > to
> > > >> > its
> > > >> > > > RPCs
> > > >> > > > >> > > before
> > > >> > > > >> > > >>>> >> moving
> > > >> > > > >> > > >>>> >> >> on. We also need to understand how the URM
> > > >> interacts
> > > >> > > with
> > > >> > > > >> > > >>>> >> >> unclean.leader.election.enable=always. I
> > assume
> > > >> that
> > > >> > > with
> > > >> > > > >> > > "always"
> > > >> > > > >> > > >>>> we
> > > >> > > > >> > > >>>> >> will
> > > >> > > > >> > > >>>> >> >> just unconditionally use the URM rather than
> > > >> choosing
> > > >> > > > >> > randomly.
> > > >> > > > >> > > >>>> But this
> > > >> > > > >> > > >>>> >> >> should be spelled out in the KIP.
> > > >> > > > >> > > >>>> >> >>
> > > >> > > > >> > > >>>> >> >> >
> > > >> > > > >> > > >>>> >> >> > DescribeTopicRequest
> > > >> > > > >> > > >>>> >> >> >
> > > >> > > > >> > > >>>> >> >> >    1.
> > > >> > > > >> > > >>>> >> >> >    Yes, the plan is to replace the
> > > >> MetadataRequest
> > > >> > > with
> > > >> > > > >> the
> > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest for the admin
> > clients.
> > > >> Will
> > > >> > > > check
> > > >> > > > >> > the
> > > >> > > > >> > > >>>> details.
> > > >> > > > >> > > >>>> >> >>
> > > >> > > > >> > > >>>> >> >> Sounds good. But as I said, you need to
> > specify
> > > >> how
> > > >> > > > >> > AdminClient
> > > >> > > > >> > > >>>> >> interacts
> > > >> > > > >> > > >>>> >> >> with the new request. This will involve
> adding
> > > >> some
> > > >> > > > fields
> > > >> > > > >> to
> > > >> > > > >> > > >>>> >> >> TopicDescription.java. And you need to
> specify
> > > the
> > > >> > > > changes
> > > >> > > > >> to
> > > >> > > > >> > > the
> > > >> > > > >> > > >>>> >> >> kafka-topics.sh command line tool. Otherwise
> > we
> > > >> > cannot
> > > >> > > > use
> > > >> > > > >> the
> > > >> > > > >> > > >>>> tool to
> > > >> > > > >> > > >>>> >> see
> > > >> > > > >> > > >>>> >> >> the new information.
> > > >> > > > >> > > >>>> >> >>
> > > >> > > > >> > > >>>> >> >> The new requests, DescribeTopicRequest and
> > > >> > > > >> > > >>>> GetReplicaLogInfoRequest,
> > > >> > > > >> > > >>>> >> need
> > > >> > > > >> > > >>>> >> >> to have limits placed on them so that their
> > size
> > > >> > can't
> > > >> > > be
> > > >> > > > >> > > >>>> infinite. We
> > > >> > > > >> > > >>>> >> >> don't want to propagate the current problems
> > of
> > > >> > > > >> > MetadataRequest,
> > > >> > > > >> > > >>>> where
> > > >> > > > >> > > >>>> >> >> clients can request massive responses that
> can
> > > >> mess
> > > >> > up
> > > >> > > > the
> > > >> > > > >> JVM
> > > >> > > > >> > > when
> > > >> > > > >> > > >>>> >> handled.
> > > >> > > > >> > > >>>> >> >>
> > > >> > > > >> > > >>>> >> >> Adding limits is simple for
> > > >> GetReplicaLogInfoRequest
> > > >> > --
> > > >> > > > we
> > > >> > > > >> can
> > > >> > > > >> > > >>>> just say
> > > >> > > > >> > > >>>> >> >> that only 2000 partitions at a time can be
> > > >> requested.
> > > >> > > For
> > > >> > > > >> > > >>>> >> >> DescribeTopicRequest we can probably just
> > limit
> > > >> to 20
> > > >> > > > topics
> > > >> > > > >> > or
> > > >> > > > >> > > >>>> >> something
> > > >> > > > >> > > >>>> >> >> like that, to avoid the complexity of doing
> > > >> > pagination
> > > >> > > in
> > > >> > > > >> this
> > > >> > > > >> > > KIP.
> > > >> > > > >> > > >>>> >> >>
> > > >> > > > >> > > >>>> >> >> >    2.
> > > >> > > > >> > > >>>> >> >> >    I can let the broker load the ELR info
> so
> > > >> that
> > > >> > > they
> > > >> > > > can
> > > >> > > > >> > > serve
> > > >> > > > >> > > >>>> the
> > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest as well.
> > > >> > > > >> > > >>>> >> >>
> > > >> > > > >> > > >>>> >> >> Yes, it's fine to add to MetadataCache. In
> > fact,
> > > >> > you'll
> > > >> > > > be
> > > >> > > > >> > > loading
> > > >> > > > >> > > >>>> it
> > > >> > > > >> > > >>>> >> >> anyway once it's added to PartitionImage.
> > > >> > > > >> > > >>>> >> >>
> > > >> > > > >> > > >>>> >> >> >    3.
> > > >> > > > >> > > >>>> >> >> >    Yeah, it does not make sense to have
> the
> > > >> topic
> > > >> > id
> > > >> > > if
> > > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest is only used by
> the
> > > >> admin
> > > >> > > > client.
> > > >> > > > >> > > >>>> >> >>
> > > >> > > > >> > > >>>> >> >> OK. That makes things simpler. We can always
> > > >> create a
> > > >> > > new
> > > >> > > > >> API
> > > >> > > > >> > > later
> > > >> > > > >> > > >>>> >> >> (hopefully not in this KIP!) to query by
> topic
> > > ID.
> > > >> > > > >> > > >>>> >> >>
> > > >> > > > >> > > >>>> >> >> >
> > > >> > > > >> > > >>>> >> >> >
> > > >> > > > >> > > >>>> >> >> > Metrics
> > > >> > > > >> > > >>>> >> >> >
> > > >> > > > >> > > >>>> >> >> > As for overall cluster health metrics, I
> > think
> > > >> > > > >> under-min-ISR
> > > >> > > > >> > > is
> > > >> > > > >> > > >>>> still
> > > >> > > > >> > > >>>> >> a
> > > >> > > > >> > > >>>> >> >> > useful one. ELR is more like a safety
> belt.
> > > When
> > > >> > the
> > > >> > > > ELR
> > > >> > > > >> is
> > > >> > > > >> > > >>>> used, the
> > > >> > > > >> > > >>>> >> >> > cluster availability has already been
> > > impacted.
> > > >> > > > >> > > >>>> >> >> >
> > > >> > > > >> > > >>>> >> >> > Maybe we can have a metric to count the
> > > >> partitions
> > > >> > > that
> > > >> > > > >> > > sum(ISR,
> > > >> > > > >> > > >>>> ELR)
> > > >> > > > >> > > >>>> >> <
> > > >> > > > >> > > >>>> >> >> min
> > > >> > > > >> > > >>>> >> >> > ISR. What do you think?
> > > >> > > > >> > > >>>> >> >>
> > > >> > > > >> > > >>>> >> >> How about:
> > > >> > > > >> > > >>>> >> >>
> > > >> > > > >> > > >>>> >> >> A.  a metric for the totoal number of
> > > >> under-min-isr
> > > >> > > > >> > partitions?
> > > >> > > > >> > > We
> > > >> > > > >> > > >>>> don't
> > > >> > > > >> > > >>>> >> >> have that in Apache Kafka at the moment.
> > > >> > > > >> > > >>>> >> >>
> > > >> > > > >> > > >>>> >> >> B. a metric for the number of unclean leader
> > > >> > elections
> > > >> > > we
> > > >> > > > >> did
> > > >> > > > >> > > (for
> > > >> > > > >> > > >>>> >> >> simplicity, it can reset to 0 on controller
> > > >> restart:
> > > >> > we
> > > >> > > > >> expect
> > > >> > > > >> > > >>>> people to
> > > >> > > > >> > > >>>> >> >> monitor the change over time anyway)
> > > >> > > > >> > > >>>> >> >>
> > > >> > > > >> > > >>>> >> >> best,
> > > >> > > > >> > > >>>> >> >> Colin
> > > >> > > > >> > > >>>> >> >>
> > > >> > > > >> > > >>>> >> >>
> > > >> > > > >> > > >>>> >> >> >
> > > >> > > > >> > > >>>> >> >> > Yeah, for the ongoing unclean recoveries,
> > the
> > > >> > > > controller
> > > >> > > > >> can
> > > >> > > > >> > > >>>> keep an
> > > >> > > > >> > > >>>> >> >> > accurate count through failover because
> > > >> partition
> > > >> > > > >> > registration
> > > >> > > > >> > > >>>> can
> > > >> > > > >> > > >>>> >> >> indicate
> > > >> > > > >> > > >>>> >> >> > whether a recovery is needed. However, for
> > the
> > > >> > > happened
> > > >> > > > >> > ones,
> > > >> > > > >> > > >>>> unless
> > > >> > > > >> > > >>>> >> we
> > > >> > > > >> > > >>>> >> >> > want to persist the number somewhere, we
> can
> > > >> only
> > > >> > > > figure
> > > >> > > > >> it
> > > >> > > > >> > > out
> > > >> > > > >> > > >>>> from
> > > >> > > > >> > > >>>> >> the
> > > >> > > > >> > > >>>> >> >> > log.
> > > >> > > > >> > > >>>> >> >> >
> > > >> > > > >> > > >>>> >> >> > On Tue, Sep 12, 2023 at 3:16 PM Colin
> > McCabe <
> > > >> > > > >> > > cmccabe@apache.org
> > > >> > > > >> > > >>>> >
> > > >> > > > >> > > >>>> >> wrote:
> > > >> > > > >> > > >>>> >> >> >
> > > >> > > > >> > > >>>> >> >> >> Also, we should have metrics that show
> what
> > > is
> > > >> > going
> > > >> > > > on
> > > >> > > > >> > with
> > > >> > > > >> > > >>>> regard
> > > >> > > > >> > > >>>> >> to
> > > >> > > > >> > > >>>> >> >> the
> > > >> > > > >> > > >>>> >> >> >> eligible replica set. I'm not sure
> exactly
> > > >> what to
> > > >> > > > >> suggest,
> > > >> > > > >> > > but
> > > >> > > > >> > > >>>> >> >> something
> > > >> > > > >> > > >>>> >> >> >> that could identify when things are going
> > > >> wrong in
> > > >> > > the
> > > >> > > > >> > > clsuter.
> > > >> > > > >> > > >>>> >> >> >>
> > > >> > > > >> > > >>>> >> >> >> For example, maybe a metric for
> partitions
> > > >> > > containing
> > > >> > > > >> > > replicas
> > > >> > > > >> > > >>>> that
> > > >> > > > >> > > >>>> >> are
> > > >> > > > >> > > >>>> >> >> >> ineligible to be leader? That would show
> a
> > > >> spike
> > > >> > > when
> > > >> > > > a
> > > >> > > > >> > > broker
> > > >> > > > >> > > >>>> had an
> > > >> > > > >> > > >>>> >> >> >> unclean restart.
> > > >> > > > >> > > >>>> >> >> >>
> > > >> > > > >> > > >>>> >> >> >> Ideally, we'd also have a metric that
> > > indicates
> > > >> > when
> > > >> > > > an
> > > >> > > > >> > > unclear
> > > >> > > > >> > > >>>> >> leader
> > > >> > > > >> > > >>>> >> >> >> election or a recovery happened. It's a
> bit
> > > >> tricky
> > > >> > > > >> because
> > > >> > > > >> > > the
> > > >> > > > >> > > >>>> simple
> > > >> > > > >> > > >>>> >> >> >> thing, of tracking it per controller, may
> > be
> > > a
> > > >> bit
> > > >> > > > >> > confusing
> > > >> > > > >> > > >>>> during
> > > >> > > > >> > > >>>> >> >> >> failovers.
> > > >> > > > >> > > >>>> >> >> >>
> > > >> > > > >> > > >>>> >> >> >> best,
> > > >> > > > >> > > >>>> >> >> >> Colin
> > > >> > > > >> > > >>>> >> >> >>
> > > >> > > > >> > > >>>> >> >> >>
> > > >> > > > >> > > >>>> >> >> >> On Tue, Sep 12, 2023, at 14:25, Colin
> > McCabe
> > > >> > wrote:
> > > >> > > > >> > > >>>> >> >> >> > Hi Calvin,
> > > >> > > > >> > > >>>> >> >> >> >
> > > >> > > > >> > > >>>> >> >> >> > Thanks for the KIP. I think this is a
> > great
> > > >> > > > >> improvement.
> > > >> > > > >> > > >>>> >> >> >> >
> > > >> > > > >> > > >>>> >> >> >> >> Additional High Watermark advance
> > > >> requirement
> > > >> > > > >> > > >>>> >> >> >> >
> > > >> > > > >> > > >>>> >> >> >> > Typo: change "advance" to "advancement"
> > > >> > > > >> > > >>>> >> >> >> >
> > > >> > > > >> > > >>>> >> >> >> >> A bit recap of some key concepts.
> > > >> > > > >> > > >>>> >> >> >> >
> > > >> > > > >> > > >>>> >> >> >> > Typo: change "bit" to "quick"
> > > >> > > > >> > > >>>> >> >> >> >
> > > >> > > > >> > > >>>> >> >> >> >> Ack=1/all produce request. It defines
> > when
> > > >> the
> > > >> > > > Kafka
> > > >> > > > >> > > server
> > > >> > > > >> > > >>>> should
> > > >> > > > >> > > >>>> >> >> >> respond to the produce request
> > > >> > > > >> > > >>>> >> >> >> >
> > > >> > > > >> > > >>>> >> >> >> > I think this section would be clearer
> if
> > we
> > > >> > talked
> > > >> > > > >> about
> > > >> > > > >> > > the
> > > >> > > > >> > > >>>> new
> > > >> > > > >> > > >>>> >> high
> > > >> > > > >> > > >>>> >> >> >> > watermark advancement requirement
> first,
> > > and
> > > >> > THEN
> > > >> > > > >> talked
> > > >> > > > >> > > >>>> about its
> > > >> > > > >> > > >>>> >> >> >> > impact on acks=0, acks=1, and
> >  acks=all.
> > > >> > > > acks=all
> > > >> > > > >> is
> > > >> > > > >> > of
> > > >> > > > >> > > >>>> course
> > > >> > > > >> > > >>>> >> the
> > > >> > > > >> > > >>>> >> >> >> > main case we care about here, so it
> would
> > > be
> > > >> > good
> > > >> > > to
> > > >> > > > >> lead
> > > >> > > > >> > > with
> > > >> > > > >> > > >>>> >> that,
> > > >> > > > >> > > >>>> >> >> >> > rather than delving into the
> > technicalities
> > > >> of
> > > >> > > > acks=0/1
> > > >> > > > >> > > first.
> > > >> > > > >> > > >>>> >> >> >> >
> > > >> > > > >> > > >>>> >> >> >> >> Unclean recovery
> > > >> > > > >> > > >>>> >> >> >> >
> > > >> > > > >> > > >>>> >> >> >> > So, here you are introducing a new
> > > >> > configuration,
> > > >> > > > >> > > >>>> >> >> >> > unclean.recovery.strategy. The
> difficult
> > > >> thing
> > > >> > > here
> > > >> > > > is
> > > >> > > > >> > that
> > > >> > > > >> > > >>>> there
> > > >> > > > >> > > >>>> >> is a
> > > >> > > > >> > > >>>> >> >> >> > lot of overlap with
> > > >> > > unclean.leader.election.enable.
> > > >> > > > So
> > > >> > > > >> we
> > > >> > > > >> > > >>>> have 3
> > > >> > > > >> > > >>>> >> >> >> > different settings for
> > > >> > unclean.recovery.strategy,
> > > >> > > > plus
> > > >> > > > >> 2
> > > >> > > > >> > > >>>> different
> > > >> > > > >> > > >>>> >> >> >> > settings for
> > > unclean.leader.election.enable,
> > > >> > > giving
> > > >> > > > a
> > > >> > > > >> > cross
> > > >> > > > >> > > >>>> >> product of
> > > >> > > > >> > > >>>> >> >> >> > 6 different options. The following
> > "unclean
> > > >> > > recovery
> > > >> > > > >> > > manager"
> > > >> > > > >> > > >>>> >> section
> > > >> > > > >> > > >>>> >> >> >> > on

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Calvin Liu <ca...@confluent.io.INVALID>.
Hi Jun,
After the MV downgrade, the controller will write in the old version of the
PartitionRecord/PartitionChangeRecord. If I understand correctly, it is
possible to downgrade the software version if the controller only has to
handle old version records.
However, the controller will not automatically rewrite the PartitionRecord
with the old version unless there is a partition update. Then, the user may
have to wait an unknown amount of time before the software downgrades
unless they do a roll to force update every partition. If it makes sense, I
can mention these steps to do a software downgrade.
Thanks

On Wed, Oct 4, 2023 at 11:20 AM Jun Rao <ju...@confluent.io.invalid> wrote:

> Hi, Calvin and Justine,
>
> Historically, when we change the record format in the log, we don't support
> software version downgrading.
>
> For the record format change in the metadata log, have we thought about
> forcing the write of the latest metadata records with the old version
> during MV downgrading? This will in theory allow the old version of the
> software to obtain the latest metadata.
>
> Thanks,
>
> Jun
>
> On Wed, Oct 4, 2023 at 9:53 AM Justine Olshan <jolshan@confluent.io.invalid
> >
> wrote:
>
> > Sorry -- not MV but software version.
> >
> > On Wed, Oct 4, 2023 at 9:51 AM Justine Olshan <jo...@confluent.io>
> > wrote:
> >
> > > Catching up with this discussion.
> > >
> > > I was just curious -- have we had other instances where downgrading MV
> is
> > > not supported? I think Kafka typically tries to support downgrades,
> and I
> > > couldn't think of other examples.
> > >
> > > Thanks,
> > > Justine
> > >
> > > On Wed, Oct 4, 2023 at 9:40 AM Calvin Liu <ca...@confluent.io.invalid>
> > > wrote:
> > >
> > >> Hi Jun,
> > >> 54. Marked the software downgrading is not supported. As the old
> > >> controller
> > >> will not understand the new PartitionRecord and PartitionChangeRecord.
> > >> Thanks!
> > >>
> > >> On Wed, Oct 4, 2023 at 9:12 AM Jun Rao <ju...@confluent.io.invalid>
> > wrote:
> > >>
> > >> > Hi, Calvin,
> > >> >
> > >> > Thanks for the reply. Just one more comment.
> > >> >
> > >> > 54. It seems that downgrading MV is supported. Is downgrading the
> > >> software
> > >> > version supported? It would be useful to document that.
> > >> >
> > >> > Thanks,
> > >> >
> > >> > Jun
> > >> >
> > >> > On Tue, Oct 3, 2023 at 4:55 PM Artem Livshits
> > >> > <al...@confluent.io.invalid> wrote:
> > >> >
> > >> > > Hi Colin,
> > >> > >
> > >> > > I think in your example "do_unclean_recovery" would need to do
> > >> different
> > >> > > things depending on the strategy.
> > >> > >
> > >> > > do_unclean_recovery() {
> > >> > >    if (unclean.recovery.manager.enabled) {
> > >> > >     if (strategy == Aggressive)
> > >> > >       use UncleanRecoveryManager(waitLastKnownERL=false)  // just
> > >> inspect
> > >> > > logs from whoever is available
> > >> > >     else
> > >> > >       use  UncleanRecoveryManager(waitLastKnownERL=true)  // must
> > wait
> > >> > for
> > >> > > at least last known ELR
> > >> > >   } else {
> > >> > >     if (strategy == Aggressive)
> > >> > >       choose the last known leader if that is available, or a
> random
> > >> > leader
> > >> > > if not)
> > >> > >     else
> > >> > >       wait for last known leader to get back
> > >> > >   }
> > >> > > }
> > >> > >
> > >> > > The idea is that the Aggressive strategy would kick in as soon as
> we
> > >> lost
> > >> > > the leader and would pick a leader from whoever is available; but
> > the
> > >> > > Balanced will only kick in when ELR is empty and will wait for the
> > >> > brokers
> > >> > > that likely have most data to be available.
> > >> > >
> > >> > > On Tue, Oct 3, 2023 at 3:04 PM Colin McCabe <cm...@apache.org>
> > >> wrote:
> > >> > >
> > >> > > > On Tue, Oct 3, 2023, at 10:49, Jun Rao wrote:
> > >> > > > > Hi, Calvin,
> > >> > > > >
> > >> > > > > Thanks for the update KIP. A few more comments.
> > >> > > > >
> > >> > > > > 41. Why would a user choose the option to select a random
> > replica
> > >> as
> > >> > > the
> > >> > > > > leader instead of using unclean.recovery.strateg=Aggressive?
> It
> > >> seems
> > >> > > > that
> > >> > > > > the latter is strictly better? If that's not the case, could
> we
> > >> fold
> > >> > > this
> > >> > > > > option under unclean.recovery.strategy instead of introducing
> a
> > >> > > separate
> > >> > > > > config?
> > >> > > >
> > >> > > > Hi Jun,
> > >> > > >
> > >> > > > I thought the flow of control was:
> > >> > > >
> > >> > > > If there is no leader for the partition {
> > >> > > >   If (there are unfenced ELR members) {
> > >> > > >     choose_an_unfenced_ELR_member
> > >> > > >   } else if (there are fenced ELR members AND
> > strategy=Aggressive) {
> > >> > > >     do_unclean_recovery
> > >> > > >   } else if (there are no ELR members AND strategy != None) {
> > >> > > >     do_unclean_recovery
> > >> > > >   } else {
> > >> > > >     do nothing about the missing leader
> > >> > > >   }
> > >> > > > }
> > >> > > >
> > >> > > > do_unclean_recovery() {
> > >> > > >    if (unclean.recovery.manager.enabled) {
> > >> > > >     use UncleanRecoveryManager
> > >> > > >   } else {
> > >> > > >     choose the last known leader if that is available, or a
> random
> > >> > leader
> > >> > > > if not)
> > >> > > >   }
> > >> > > > }
> > >> > > >
> > >> > > > However, I think this could be clarified, especially the
> behavior
> > >> when
> > >> > > > unclean.recovery.manager.enabled=false. Inuitively the goal for
> > >> > > > unclean.recovery.manager.enabled=false is to be "the same as
> now,
> > >> > mostly"
> > >> > > > but it's very underspecified in the KIP, I agree.
> > >> > > >
> > >> > > > >
> > >> > > > > 50. ElectLeadersRequest: "If more than 20 topics are included,
> > >> only
> > >> > the
> > >> > > > > first 20 will be served. Others will be returned with
> > >> > DesiredLeaders."
> > >> > > > Hmm,
> > >> > > > > not sure that I understand this. ElectLeadersResponse doesn't
> > >> have a
> > >> > > > > DesiredLeaders field.
> > >> > > > >
> > >> > > > > 51. GetReplicaLogInfo: "If more than 2000 partitions are
> > included,
> > >> > only
> > >> > > > the
> > >> > > > > first 2000 will be served" Do we return an error for the
> > remaining
> > >> > > > > partitions? Actually, should we include an errorCode field at
> > the
> > >> > > > partition
> > >> > > > > level in GetReplicaLogInfoResponse to cover non-existing
> > >> partitions
> > >> > and
> > >> > > > no
> > >> > > > > authorization, etc?
> > >> > > > >
> > >> > > > > 52. The entry should matches => The entry should match
> > >> > > > >
> > >> > > > > 53. ElectLeadersRequest.DesiredLeaders: Should it be nullable
> > >> since a
> > >> > > > user
> > >> > > > > may not specify DesiredLeaders?
> > >> > > > >
> > >> > > > > 54. Downgrade: Is that indeed possible? I thought earlier you
> > said
> > >> > that
> > >> > > > > once the new version of the records are in the metadata log,
> one
> > >> > can't
> > >> > > > > downgrade since the old broker doesn't know how to parse the
> new
> > >> > > version
> > >> > > > of
> > >> > > > > the metadata records?
> > >> > > > >
> > >> > > >
> > >> > > > MetadataVersion downgrade is currently broken but we have fixing
> > it
> > >> on
> > >> > > our
> > >> > > > plate for Kafka 3.7.
> > >> > > >
> > >> > > > The way downgrade works is that "new features" are dropped,
> > leaving
> > >> > only
> > >> > > > the old ones.
> > >> > > >
> > >> > > > > 55. CleanShutdownFile: Should we add a version field for
> future
> > >> > > > extension?
> > >> > > > >
> > >> > > > > 56. Config changes are public facing. Could we have a separate
> > >> > section
> > >> > > to
> > >> > > > > document all the config changes?
> > >> > > >
> > >> > > > +1. A separate section for this would be good.
> > >> > > >
> > >> > > > best,
> > >> > > > Colin
> > >> > > >
> > >> > > > >
> > >> > > > > Thanks,
> > >> > > > >
> > >> > > > > Jun
> > >> > > > >
> > >> > > > > On Mon, Sep 25, 2023 at 4:29 PM Calvin Liu
> > >> > <caliu@confluent.io.invalid
> > >> > > >
> > >> > > > > wrote:
> > >> > > > >
> > >> > > > >> Hi Jun
> > >> > > > >> Thanks for the comments.
> > >> > > > >>
> > >> > > > >> 40. If we change to None, it is not guaranteed for no data
> > loss.
> > >> For
> > >> > > > users
> > >> > > > >> who are not able to validate the data with external
> resources,
> > >> > manual
> > >> > > > >> intervention does not give a better result but a loss of
> > >> > availability.
> > >> > > > So
> > >> > > > >> practically speaking, the Balance mode would be a better
> > default
> > >> > > value.
> > >> > > > >>
> > >> > > > >> 41. No, it represents how we want to do the unclean leader
> > >> election.
> > >> > > If
> > >> > > > it
> > >> > > > >> is false, the unclean leader election will be the old random
> > way.
> > >> > > > >> Otherwise, the unclean recovery will be used.
> > >> > > > >>
> > >> > > > >> 42. Good catch. Updated.
> > >> > > > >>
> > >> > > > >> 43. Only the first 20 topics will be served. Others will be
> > >> returned
> > >> > > > with
> > >> > > > >> InvalidRequestError
> > >> > > > >>
> > >> > > > >> 44. The order matters. The desired leader entries match with
> > the
> > >> > topic
> > >> > > > >> partition list by the index.
> > >> > > > >>
> > >> > > > >> 45. Thanks! Updated.
> > >> > > > >>
> > >> > > > >> 46. Good advice! Updated.
> > >> > > > >>
> > >> > > > >> 47.1, updated the comment. Basically it will elect the
> replica
> > in
> > >> > the
> > >> > > > >> desiredLeader field to be the leader
> > >> > > > >>
> > >> > > > >> 47.2 We can let the admin client do the conversion. Using the
> > >> > > > desiredLeader
> > >> > > > >> field in the json format seems easier for users.
> > >> > > > >>
> > >> > > > >> 48. Once the MV version is downgraded, all the ELR related
> > fields
> > >> > will
> > >> > > > be
> > >> > > > >> removed on the next partition change. The controller will
> also
> > >> > ignore
> > >> > > > the
> > >> > > > >> ELR fields. Updated the KIP.
> > >> > > > >>
> > >> > > > >> 49. Yes, it would be deprecated/removed.
> > >> > > > >>
> > >> > > > >>
> > >> > > > >> On Mon, Sep 25, 2023 at 3:49 PM Jun Rao
> > <jun@confluent.io.invalid
> > >> >
> > >> > > > wrote:
> > >> > > > >>
> > >> > > > >> > Hi, Calvin,
> > >> > > > >> >
> > >> > > > >> > Thanks for the updated KIP. Made another pass. A few more
> > >> comments
> > >> > > > below.
> > >> > > > >> >
> > >> > > > >> > 40. unclean.leader.election.enable.false ->
> > >> > > > >> > unclean.recovery.strategy.Balanced: The Balanced mode could
> > >> still
> > >> > > > lead to
> > >> > > > >> > data loss. So, I am wondering if
> > >> > > unclean.leader.election.enable.false
> > >> > > > >> > should map to None?
> > >> > > > >> >
> > >> > > > >> > 41. unclean.recovery.manager.enabled: I am not sure why we
> > >> > introduce
> > >> > > > this
> > >> > > > >> > additional config. Is it the same as
> > >> > unclean.recovery.strategy=None?
> > >> > > > >> >
> > >> > > > >> > 42. DescribeTopicResponse.TopicAuthorizedOperations: Should
> > >> this
> > >> > be
> > >> > > at
> > >> > > > >> the
> > >> > > > >> > topic level?
> > >> > > > >> >
> > >> > > > >> > 43. "Limit: 20 topics max per request": Could we describe
> > what
> > >> > > > happens if
> > >> > > > >> > the request includes more than 20 topics?
> > >> > > > >> >
> > >> > > > >> > 44. ElectLeadersRequest.DesiredLeaders: Could we describe
> > >> whether
> > >> > > the
> > >> > > > >> > ordering matters?
> > >> > > > >> >
> > >> > > > >> > 45. GetReplicaLogInfo.TopicPartitions: "about": "The topic
> > >> > > partitions
> > >> > > > to
> > >> > > > >> > elect leaders.": The description in "about" is incorrect.
> > >> > > > >> >
> > >> > > > >> > 46. GetReplicaLogInfoResponse: Should we nest partitions
> > under
> > >> > > > topicId to
> > >> > > > >> > be consistent with other types of responses?
> > >> > > > >> >
> > >> > > > >> > 47. kafka-leader-election.sh:
> > >> > > > >> > 47.1 Could we explain DESIGNATION?
> > >> > > > >> > 47.2 desiredLeader: Should it be a list to match the field
> in
> > >> > > > >> > ElectLeadersRequest?
> > >> > > > >> >
> > >> > > > >> > 48. We could add a section on downgrade?
> > >> > > > >> >
> > >> > > > >> > 49. LastKnownLeader: This seems only needed in the first
> > phase
> > >> of
> > >> > > > >> > delivering ELR. Will it be removed when the complete KIP is
> > >> > > delivered?
> > >> > > > >> >
> > >> > > > >> > Thanks,
> > >> > > > >> >
> > >> > > > >> > Jun
> > >> > > > >> >
> > >> > > > >> > On Tue, Sep 19, 2023 at 1:30 PM Colin McCabe <
> > >> cmccabe@apache.org>
> > >> > > > wrote:
> > >> > > > >> >
> > >> > > > >> > > Hi Calvin,
> > >> > > > >> > >
> > >> > > > >> > > Thanks for the explanations. I like the idea of using
> none,
> > >> > > > balanced,
> > >> > > > >> > > aggressive. We also had an offline discussion about why
> it
> > is
> > >> > good
> > >> > > > to
> > >> > > > >> > use a
> > >> > > > >> > > new config key (basically, so that we can deprecate the
> old
> > >> one
> > >> > > > which
> > >> > > > >> had
> > >> > > > >> > > only false/true values in 4.0) With these changes, I am
> +1.
> > >> > > > >> > >
> > >> > > > >> > > best,
> > >> > > > >> > > Colin
> > >> > > > >> > >
> > >> > > > >> > > On Mon, Sep 18, 2023, at 15:54, Calvin Liu wrote:
> > >> > > > >> > > > Hi Colin,
> > >> > > > >> > > > Also, can we deprecate unclean.leader.election.enable
> in
> > >> 4.0?
> > >> > > > Before
> > >> > > > >> > > that,
> > >> > > > >> > > > we can have both the config unclean.recovery.strategy
> and
> > >> > > > >> > > > unclean.leader.election.enable
> > >> > > > >> > > > and using the unclean.recovery.Enabled to determine
> which
> > >> > config
> > >> > > > to
> > >> > > > >> use
> > >> > > > >> > > > during the unclean leader election.
> > >> > > > >> > > >
> > >> > > > >> > > > On Mon, Sep 18, 2023 at 3:51 PM Calvin Liu <
> > >> > caliu@confluent.io>
> > >> > > > >> wrote:
> > >> > > > >> > > >
> > >> > > > >> > > >> Hi Colin,
> > >> > > > >> > > >> For the unclean.recovery.strategy config name, how
> about
> > >> we
> > >> > use
> > >> > > > the
> > >> > > > >> > > >> following
> > >> > > > >> > > >> None. It basically means no unclean recovery will be
> > >> > performed.
> > >> > > > >> > > >> Aggressive. It means availability goes first. Whenever
> > the
> > >> > > > partition
> > >> > > > >> > > can't
> > >> > > > >> > > >> elect a durable replica, the controller will try the
> > >> unclean
> > >> > > > >> recovery.
> > >> > > > >> > > >> Balanced. It is the balance point of the availability
> > >> > > > >> > first(Aggressive)
> > >> > > > >> > > >> and least availability(None). The controller performs
> > >> unclean
> > >> > > > >> recovery
> > >> > > > >> > > when
> > >> > > > >> > > >> both ISR and ELR are empty.
> > >> > > > >> > > >>
> > >> > > > >> > > >>
> > >> > > > >> > > >> On Fri, Sep 15, 2023 at 11:42 AM Calvin Liu <
> > >> > > caliu@confluent.io>
> > >> > > > >> > wrote:
> > >> > > > >> > > >>
> > >> > > > >> > > >>> Hi Colin,
> > >> > > > >> > > >>>
> > >> > > > >> > > >>> > So, the proposal is that if someone sets
> > >> > > > >> > > "unclean.leader.election.enable
> > >> > > > >> > > >>> = true"...
> > >> > > > >> > > >>>
> > >> > > > >> > > >>>
> > >> > > > >> > > >>> The idea is to use one of the
> > >> unclean.leader.election.enable
> > >> > > and
> > >> > > > >> > > >>> unclean.recovery.strategy based on the
> > >> > > > unclean.recovery.Enabled. A
> > >> > > > >> > > possible
> > >> > > > >> > > >>> version can be
> > >> > > > >> > > >>>
> > >> > > > >> > > >>> If unclean.recovery.Enabled:
> > >> > > > >> > > >>>
> > >> > > > >> > > >>> {
> > >> > > > >> > > >>>
> > >> > > > >> > > >>> Check unclean.recovery.strategy. If set, use it.
> > >> Otherwise,
> > >> > > > check
> > >> > > > >> > > >>> unclean.leader.election.enable and translate it to
> > >> > > > >> > > >>> unclean.recovery.strategy.
> > >> > > > >> > > >>>
> > >> > > > >> > > >>> } else {
> > >> > > > >> > > >>>
> > >> > > > >> > > >>> Use unclean.leader.election.enable
> > >> > > > >> > > >>>
> > >> > > > >> > > >>> }
> > >> > > > >> > > >>>
> > >> > > > >> > > >>>
> > >> > > > >> > > >>> —--------
> > >> > > > >> > > >>>
> > >> > > > >> > > >>> >The configuration key should be
> > >> > > > >> "unclean.recovery.manager.enabled",
> > >> > > > >> > > >>> right?
> > >> > > > >> > > >>>
> > >> > > > >> > > >>>
> > >> > > > >> > > >>> I think we have two ways of choosing a leader
> > uncleanly,
> > >> > > unclean
> > >> > > > >> > leader
> > >> > > > >> > > >>> election and unclean recovery(log inspection) and we
> > try
> > >> to
> > >> > > > switch
> > >> > > > >> > > between
> > >> > > > >> > > >>> them.
> > >> > > > >> > > >>>
> > >> > > > >> > > >>> Do you mean we want to develop two ways of performing
> > the
> > >> > > > unclean
> > >> > > > >> > > >>> recovery and one of them is using “unclean recovery
> > >> > manager”?
> > >> > > I
> > >> > > > >> guess
> > >> > > > >> > > we
> > >> > > > >> > > >>> haven’t discussed the second way.
> > >> > > > >> > > >>>
> > >> > > > >> > > >>>
> > >> > > > >> > > >>> —-------
> > >> > > > >> > > >>>
> > >> > > > >> > > >>> >How do these 4 levels of overrides interact with
> your
> > >> new
> > >> > > > >> > > >>> configurations?
> > >> > > > >> > > >>>
> > >> > > > >> > > >>>
> > >> > > > >> > > >>> I do notice in the Kraft controller code, the method
> to
> > >> > check
> > >> > > > >> whether
> > >> > > > >> > > >>> perform unclean leader election is hard coded to
> false
> > >> since
> > >> > > > >> > > >>> 2021(uncleanLeaderElectionEnabledForTopic). Isn’t it
> a
> > >> good
> > >> > > > chance
> > >> > > > >> to
> > >> > > > >> > > >>> completely deprecate the
> > unclean.leader.election.enable?
> > >> We
> > >> > > > don’t
> > >> > > > >> > even
> > >> > > > >> > > have
> > >> > > > >> > > >>> to worry about the config conversion.
> > >> > > > >> > > >>>
> > >> > > > >> > > >>> On the other hand, whatever the override is, as long
> as
> > >> the
> > >> > > > >> > controller
> > >> > > > >> > > >>> can have the final effective
> > >> unclean.leader.election.enable,
> > >> > > the
> > >> > > > >> > topic
> > >> > > > >> > > >>> level config unclean.recovery.strategy, the cluster
> > level
> > >> > > config
> > >> > > > >> > > >>> unclean.recovery.Enabled, the controller can
> calculate
> > >> the
> > >> > > > correct
> > >> > > > >> > > methods
> > >> > > > >> > > >>> to use right?
> > >> > > > >> > > >>>
> > >> > > > >> > > >>>
> > >> > > > >> > > >>> On Fri, Sep 15, 2023 at 10:02 AM Colin McCabe <
> > >> > > > cmccabe@apache.org>
> > >> > > > >> > > wrote:
> > >> > > > >> > > >>>
> > >> > > > >> > > >>>> On Thu, Sep 14, 2023, at 22:23, Calvin Liu wrote:
> > >> > > > >> > > >>>> > Hi Colin
> > >> > > > >> > > >>>> > 1. I think using the new config name is more
> clear.
> > >> > > > >> > > >>>> >        a. The unclean leader election is actually
> > >> removed
> > >> > > if
> > >> > > > >> > unclean
> > >> > > > >> > > >>>> > recovery is in use.
> > >> > > > >> > > >>>> >        b. Using multiple values in
> > >> > > > >> unclean.leader.election.enable
> > >> > > > >> > is
> > >> > > > >> > > >>>> > confusing and it will be more confusing after
> people
> > >> > forget
> > >> > > > >> about
> > >> > > > >> > > this
> > >> > > > >> > > >>>> > discussion.
> > >> > > > >> > > >>>>
> > >> > > > >> > > >>>> Hi Calvin,
> > >> > > > >> > > >>>>
> > >> > > > >> > > >>>> So, the proposal is that if someone sets
> > >> > > > >> > > "unclean.leader.election.enable
> > >> > > > >> > > >>>> = true" but then sets one of your new
> configurations,
> > >> the
> > >> > > > value of
> > >> > > > >> > > >>>> unclean.leader.election.enable is ignored? That
> seems
> > >> less
> > >> > > > clear
> > >> > > > >> to
> > >> > > > >> > > me, not
> > >> > > > >> > > >>>> more. Just in general, having multiple configuration
> > >> keys
> > >> > to
> > >> > > > >> control
> > >> > > > >> > > the
> > >> > > > >> > > >>>> same thing confuses users. Basically, they are
> sitting
> > >> at a
> > >> > > > giant
> > >> > > > >> > > control
> > >> > > > >> > > >>>> panel, and some of the levers do nothing.
> > >> > > > >> > > >>>>
> > >> > > > >> > > >>>> > 2. Sorry I forgot to mention in the response that
> I
> > >> did
> > >> > add
> > >> > > > the
> > >> > > > >> > > >>>> > unclean.recovery.Enabled flag.
> > >> > > > >> > > >>>>
> > >> > > > >> > > >>>> The configuration key should be
> > >> > > > >> "unclean.recovery.manager.enabled",
> > >> > > > >> > > >>>> right? Becuase we can do "unclean recovery" without
> > the
> > >> > > > manager.
> > >> > > > >> > > Disabling
> > >> > > > >> > > >>>> the manager just means we use a different mechanism
> > for
> > >> > > > recovery.
> > >> > > > >> > > >>>>
> > >> > > > >> > > >>>> >        c. Maybe I underestimated the challenge of
> > >> > replacing
> > >> > > > the
> > >> > > > >> > > >>>> config. Any
> > >> > > > >> > > >>>> > implementation problems ahead?
> > >> > > > >> > > >>>>
> > >> > > > >> > > >>>> There are four levels of overrides for
> > >> > > > >> > unclean.leader.election.enable.
> > >> > > > >> > > >>>>
> > >> > > > >> > > >>>> 1. static configuration for node.
> > >> > > > >> > > >>>>     This goes in the configuration file, typically
> > named
> > >> > > > >> > > >>>> server.properties
> > >> > > > >> > > >>>>
> > >> > > > >> > > >>>> 2. dynamic configuration for node default
> > >> > > > >> > > >>>>   ConfigResource(type=BROKER, name="")
> > >> > > > >> > > >>>>
> > >> > > > >> > > >>>> 3. dynamic configuration for node
> > >> > > > >> > > >>>>   ConfigResource(type=BROKER, name=<controller id>)
> > >> > > > >> > > >>>>
> > >> > > > >> > > >>>> 4. dynamic configuration for topic
> > >> > > > >> > > >>>>   ConfigResource(type=TOPIC, name=<topic-name>)
> > >> > > > >> > > >>>>
> > >> > > > >> > > >>>> How do these 4 levels of overrides interact with
> your
> > >> new
> > >> > > > >> > > >>>> configurations? If the new configurations dominate
> > over
> > >> the
> > >> > > old
> > >> > > > >> > ones,
> > >> > > > >> > > it
> > >> > > > >> > > >>>> seems like this will get a lot more confusing to
> > >> implement
> > >> > > (and
> > >> > > > >> also
> > >> > > > >> > > to
> > >> > > > >> > > >>>> use.)
> > >> > > > >> > > >>>>
> > >> > > > >> > > >>>> Again, I'd recommend just adding some new values to
> > >> > > > >> > > >>>> unclean.leader.election.enable. It's simple and will
> > >> > prevent
> > >> > > > user
> > >> > > > >> > > confusion
> > >> > > > >> > > >>>> (as well as developer confusion.)
> > >> > > > >> > > >>>>
> > >> > > > >> > > >>>> best,
> > >> > > > >> > > >>>> Colin
> > >> > > > >> > > >>>>
> > >> > > > >> > > >>>>
> > >> > > > >> > > >>>> > 3. About the admin client, I mentioned 3 changes
> in
> > >> the
> > >> > > > client.
> > >> > > > >> > > >>>> Anything
> > >> > > > >> > > >>>> > else I missed in the KIP?
> > >> > > > >> > > >>>> >       a. The client will switch to using the new
> RPC
> > >> > > instead
> > >> > > > of
> > >> > > > >> > > >>>> > MetadataRequest for the topics.
> > >> > > > >> > > >>>> >       b. The TopicPartitionInfo used in
> > >> TopicDescription
> > >> > > > needs
> > >> > > > >> to
> > >> > > > >> > > add
> > >> > > > >> > > >>>> new
> > >> > > > >> > > >>>> > fields related to the ELR.
> > >> > > > >> > > >>>> >       c. The outputs will add the ELR related
> > fields.
> > >> > > > >> > > >>>> >
> > >> > > > >> > > >>>> > On Thu, Sep 14, 2023 at 9:19 PM Colin McCabe <
> > >> > > > >> cmccabe@apache.org>
> > >> > > > >> > > >>>> wrote:
> > >> > > > >> > > >>>> >
> > >> > > > >> > > >>>> >> Hi Calvin,
> > >> > > > >> > > >>>> >>
> > >> > > > >> > > >>>> >> Thanks for the changes.
> > >> > > > >> > > >>>> >>
> > >> > > > >> > > >>>> >> 1. Earlier I commented that creating
> > >> > > > >> "unclean.recovery.strategy "
> > >> > > > >> > > is
> > >> > > > >> > > >>>> not
> > >> > > > >> > > >>>> >> necessary, and we can just reuse the existing
> > >> > > > >> > > >>>> >> "unclean.leader.election.enable" configuration
> key.
> > >> > Let's
> > >> > > > >> discuss
> > >> > > > >> > > >>>> that.
> > >> > > > >> > > >>>> >>
> > >> > > > >> > > >>>> >> 2.I also don't understand why you didn't add a
> > >> > > > configuration to
> > >> > > > >> > > >>>> enable or
> > >> > > > >> > > >>>> >> disable the Unclean Recovery Manager. This seems
> > >> like a
> > >> > > very
> > >> > > > >> > simple
> > >> > > > >> > > >>>> way to
> > >> > > > >> > > >>>> >> handle the staging issue which we discussed. The
> > URM
> > >> can
> > >> > > > just
> > >> > > > >> be
> > >> > > > >> > > >>>> turned off
> > >> > > > >> > > >>>> >> until it is production ready. Let's discuss this.
> > >> > > > >> > > >>>> >>
> > >> > > > >> > > >>>> >> 3. You still need to describe the changes to
> > >> AdminClient
> > >> > > > that
> > >> > > > >> are
> > >> > > > >> > > >>>> needed
> > >> > > > >> > > >>>> >> to use DescribeTopicRequest.
> > >> > > > >> > > >>>> >>
> > >> > > > >> > > >>>> >> Keep at it. It's looking better. :)
> > >> > > > >> > > >>>> >>
> > >> > > > >> > > >>>> >> best,
> > >> > > > >> > > >>>> >> Colin
> > >> > > > >> > > >>>> >>
> > >> > > > >> > > >>>> >>
> > >> > > > >> > > >>>> >> On Thu, Sep 14, 2023, at 11:03, Calvin Liu wrote:
> > >> > > > >> > > >>>> >> > Hi Colin
> > >> > > > >> > > >>>> >> > Thanks for the comments!
> > >> > > > >> > > >>>> >> >
> > >> > > > >> > > >>>> >> > I did the following changes
> > >> > > > >> > > >>>> >> >
> > >> > > > >> > > >>>> >> >    1.
> > >> > > > >> > > >>>> >> >
> > >> > > > >> > > >>>> >> >    Simplified the API spec section to only
> > include
> > >> the
> > >> > > > diff.
> > >> > > > >> > > >>>> >> >    2.
> > >> > > > >> > > >>>> >> >
> > >> > > > >> > > >>>> >> >    Reordered the HWM requirement section.
> > >> > > > >> > > >>>> >> >    3.
> > >> > > > >> > > >>>> >> >
> > >> > > > >> > > >>>> >> >    Removed the URM implementation details to
> keep
> > >> the
> > >> > > > >> necessary
> > >> > > > >> > > >>>> >> >    characteristics to perform the unclean
> > recovery.
> > >> > > > >> > > >>>> >> >    1.
> > >> > > > >> > > >>>> >> >
> > >> > > > >> > > >>>> >> >       When to perform the unclean recovery
> > >> > > > >> > > >>>> >> >       2.
> > >> > > > >> > > >>>> >> >
> > >> > > > >> > > >>>> >> >       Under different config, how the unclean
> > >> recovery
> > >> > > > finds
> > >> > > > >> > the
> > >> > > > >> > > >>>> leader.
> > >> > > > >> > > >>>> >> >       3.
> > >> > > > >> > > >>>> >> >
> > >> > > > >> > > >>>> >> >       How the config
> > unclean.leader.election.enable
> > >> > and
> > >> > > > >> > > >>>> >> >       unclean.recovery.strategy are converted
> > when
> > >> > users
> > >> > > > >> > > >>>> enable/disable
> > >> > > > >> > > >>>> >> the
> > >> > > > >> > > >>>> >> >       unclean recovery.
> > >> > > > >> > > >>>> >> >       4.
> > >> > > > >> > > >>>> >> >
> > >> > > > >> > > >>>> >> >    More details about how we change admin
> client.
> > >> > > > >> > > >>>> >> >    5.
> > >> > > > >> > > >>>> >> >
> > >> > > > >> > > >>>> >> >    API limits on the GetReplicaLogInfoRequest
> and
> > >> > > > >> > > >>>> DescribeTopicRequest.
> > >> > > > >> > > >>>> >> >    6.
> > >> > > > >> > > >>>> >> >
> > >> > > > >> > > >>>> >> >    Two metrics added
> > >> > > > >> > > >>>> >> >    1.
> > >> > > > >> > > >>>> >> >
> > >> > > > >> > > >>>> >> >
> > >> > > >  Kafka.controller.global_under_min_isr_partition_count
> > >> > > > >> > > >>>> >> >       2.
> > >> > > > >> > > >>>> >> >
> > >> > > > >> > > >>>> >> >
> > >>  kafka.controller.unclean_recovery_finished_count
> > >> > > > >> > > >>>> >> >
> > >> > > > >> > > >>>> >> >
> > >> > > > >> > > >>>> >> > On Wed, Sep 13, 2023 at 10:46 AM Colin McCabe <
> > >> > > > >> > > cmccabe@apache.org>
> > >> > > > >> > > >>>> >> wrote:
> > >> > > > >> > > >>>> >> >
> > >> > > > >> > > >>>> >> >> On Tue, Sep 12, 2023, at 17:21, Calvin Liu
> > wrote:
> > >> > > > >> > > >>>> >> >> > Hi Colin
> > >> > > > >> > > >>>> >> >> > Thanks for the comments!
> > >> > > > >> > > >>>> >> >> >
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >> >> Hi Calvin,
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >> >> Thanks again for the KIP.
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >> >> One meta-comment: it's usually better to just
> > do a
> > >> > diff
> > >> > > > on a
> > >> > > > >> > > >>>> message
> > >> > > > >> > > >>>> >> spec
> > >> > > > >> > > >>>> >> >> file or java file if you're including changes
> to
> > >> it
> > >> > in
> > >> > > > the
> > >> > > > >> > KIP.
> > >> > > > >> > > >>>> This is
> > >> > > > >> > > >>>> >> >> easier to read than looking for "new fields
> > begin"
> > >> > etc.
> > >> > > > in
> > >> > > > >> the
> > >> > > > >> > > >>>> text, and
> > >> > > > >> > > >>>> >> >> gracefully handles the case where existing
> > fields
> > >> > were
> > >> > > > >> > changed.
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >> >> > Rewrite the Additional High Watermark
> > >> advancement
> > >> > > > >> > requirement
> > >> > > > >> > > >>>> >> >> > There was feedback on this section that some
> > >> > readers
> > >> > > > may
> > >> > > > >> not
> > >> > > > >> > > be
> > >> > > > >> > > >>>> >> familiar
> > >> > > > >> > > >>>> >> >> > with HWM and Ack=0,1,all requests. This can
> > help
> > >> > them
> > >> > > > >> > > understand
> > >> > > > >> > > >>>> the
> > >> > > > >> > > >>>> >> >> > proposal. I will rewrite this part for more
> > >> > > > readability.
> > >> > > > >> > > >>>> >> >> >
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >> >> To be clear, I wasn't suggesting dropping
> either
> > >> > > > section. I
> > >> > > > >> > > agree
> > >> > > > >> > > >>>> that
> > >> > > > >> > > >>>> >> >> they add useful background. I was just
> > suggesting
> > >> > that
> > >> > > we
> > >> > > > >> > should
> > >> > > > >> > > >>>> discuss
> > >> > > > >> > > >>>> >> >> the "acks" setting AFTER discussing the new
> high
> > >> > > > watermark
> > >> > > > >> > > >>>> advancement
> > >> > > > >> > > >>>> >> >> conditions. We also should discuss acks=0.
> While
> > >> it
> > >> > > isn't
> > >> > > > >> > > >>>> conceptually
> > >> > > > >> > > >>>> >> much
> > >> > > > >> > > >>>> >> >> different than acks=1 here, its omission from
> > this
> > >> > > > section
> > >> > > > >> is
> > >> > > > >> > > >>>> confusing.
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >> >> > Unclean recovery
> > >> > > > >> > > >>>> >> >> >
> > >> > > > >> > > >>>> >> >> > The plan is to replace the
> > >> > > > unclean.leader.election.enable
> > >> > > > >> > with
> > >> > > > >> > > >>>> >> >> > unclean.recovery.strategy. If the Unclean
> > >> Recovery
> > >> > is
> > >> > > > >> > enabled
> > >> > > > >> > > >>>> then it
> > >> > > > >> > > >>>> >> >> deals
> > >> > > > >> > > >>>> >> >> > with the three options in the
> > >> > > > unclean.recovery.strategy.
> > >> > > > >> > > >>>> >> >> >
> > >> > > > >> > > >>>> >> >> >
> > >> > > > >> > > >>>> >> >> > Let’s refine the Unclean Recovery. We have
> > >> already
> > >> > > > taken a
> > >> > > > >> > > lot of
> > >> > > > >> > > >>>> >> >> > suggestions and I hope to enhance the
> > >> durability of
> > >> > > > Kafka
> > >> > > > >> to
> > >> > > > >> > > the
> > >> > > > >> > > >>>> next
> > >> > > > >> > > >>>> >> >> level
> > >> > > > >> > > >>>> >> >> > with this KIP.
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >> >> I am OK with doing the unclean leader recovery
> > >> > > > improvements
> > >> > > > >> in
> > >> > > > >> > > >>>> this KIP.
> > >> > > > >> > > >>>> >> >> However, I think we need to really work on the
> > >> > > > configuration
> > >> > > > >> > > >>>> settings.
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >> >> Configuration overrides are often quite messy.
> > For
> > >> > > > example,
> > >> > > > >> > the
> > >> > > > >> > > >>>> cases
> > >> > > > >> > > >>>> >> >> where we have log.roll.hours and
> > >> log.roll.segment.ms
> > >> > ,
> > >> > > > the
> > >> > > > >> > user
> > >> > > > >> > > >>>> has to
> > >> > > > >> > > >>>> >> >> remember which one takes precedence, and it is
> > not
> > >> > > > obvious.
> > >> > > > >> > So,
> > >> > > > >> > > >>>> rather
> > >> > > > >> > > >>>> >> than
> > >> > > > >> > > >>>> >> >> creating a new configuration, why not add
> > >> additional
> > >> > > > values
> > >> > > > >> to
> > >> > > > >> > > >>>> >> >> "unclean.leader.election.enable"? I think this
> > >> will
> > >> > be
> > >> > > > >> simpler
> > >> > > > >> > > for
> > >> > > > >> > > >>>> >> people
> > >> > > > >> > > >>>> >> >> to understand, and simpler in the code as
> well.
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >> >> What if we continued to use
> > >> > > > "unclean.leader.election.enable"
> > >> > > > >> > but
> > >> > > > >> > > >>>> >> extended
> > >> > > > >> > > >>>> >> >> it so that it took a string? Then the string
> > could
> > >> > have
> > >> > > > >> these
> > >> > > > >> > > >>>> values:
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >> >> never
> > >> > > > >> > > >>>> >> >>     never automatically do an unclean leader
> > >> election
> > >> > > > under
> > >> > > > >> > any
> > >> > > > >> > > >>>> >> conditions
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >> >> false / default
> > >> > > > >> > > >>>> >> >>     only do an unclean leader election if
> there
> > >> may
> > >> > be
> > >> > > > >> > possible
> > >> > > > >> > > >>>> data
> > >> > > > >> > > >>>> >> loss
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >> >> true / always
> > >> > > > >> > > >>>> >> >>     always do an unclean leader election if we
> > >> can't
> > >> > > > >> > immediately
> > >> > > > >> > > >>>> elect a
> > >> > > > >> > > >>>> >> >> leader
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >> >> It's a bit awkward that false maps to default
> > >> rather
> > >> > > > than to
> > >> > > > >> > > >>>> never. But
> > >> > > > >> > > >>>> >> >> this awkwardness exists if we use two
> different
> > >> > > > >> configuration
> > >> > > > >> > > keys
> > >> > > > >> > > >>>> as
> > >> > > > >> > > >>>> >> well.
> > >> > > > >> > > >>>> >> >> The reason for the awkwardness is that we
> simply
> > >> > don't
> > >> > > > want
> > >> > > > >> > most
> > >> > > > >> > > >>>> of the
> > >> > > > >> > > >>>> >> >> people currently setting
> > >> > > > >> unclean.leader.election.enable=false
> > >> > > > >> > to
> > >> > > > >> > > >>>> get the
> > >> > > > >> > > >>>> >> >> "never" behavior. We have to bite that bullet.
> > >> Better
> > >> > > to
> > >> > > > be
> > >> > > > >> > > clear
> > >> > > > >> > > >>>> and
> > >> > > > >> > > >>>> >> >> explicit than hide it.
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >> >> Another thing that's a bit awkward is having
> two
> > >> > > > different
> > >> > > > >> > ways
> > >> > > > >> > > to
> > >> > > > >> > > >>>> do
> > >> > > > >> > > >>>> >> >> unclean leader election specified in the KIP.
> > You
> > >> > > > descirbe
> > >> > > > >> two
> > >> > > > >> > > >>>> methods:
> > >> > > > >> > > >>>> >> the
> > >> > > > >> > > >>>> >> >> simple "choose the last leader" method, and
> the
> > >> > > "unclean
> > >> > > > >> > > recovery
> > >> > > > >> > > >>>> >> manager"
> > >> > > > >> > > >>>> >> >> method. I understand why you did it this way
> --
> > >> > "choose
> > >> > > > the
> > >> > > > >> > last
> > >> > > > >> > > >>>> >> leader" is
> > >> > > > >> > > >>>> >> >> simple, and will help us deliver an
> > implementation
> > >> > > > quickly,
> > >> > > > >> > > while
> > >> > > > >> > > >>>> the
> > >> > > > >> > > >>>> >> URM
> > >> > > > >> > > >>>> >> >> is preferable in the long term. My suggestion
> > >> here is
> > >> > > to
> > >> > > > >> > > separate
> > >> > > > >> > > >>>> the
> > >> > > > >> > > >>>> >> >> decision of HOW to do unclean leader election
> > from
> > >> > the
> > >> > > > >> > decision
> > >> > > > >> > > of
> > >> > > > >> > > >>>> WHEN
> > >> > > > >> > > >>>> >> to
> > >> > > > >> > > >>>> >> >> do it.
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >> >> So in other words, have
> > >> > > "unclean.leader.election.enable"
> > >> > > > >> > specify
> > >> > > > >> > > >>>> when we
> > >> > > > >> > > >>>> >> >> do unclean leader election, and have a new
> > >> > > configuration
> > >> > > > >> like
> > >> > > > >> > > >>>> >> >> "unclean.recovery.manager.enable" to determine
> > if
> > >> we
> > >> > > use
> > >> > > > the
> > >> > > > >> > > URM.
> > >> > > > >> > > >>>> >> >> Presumably the URM will take some time to get
> > >> fully
> > >> > > > stable,
> > >> > > > >> so
> > >> > > > >> > > >>>> this can
> > >> > > > >> > > >>>> >> >> default to false for a while, and we can flip
> > the
> > >> > > > default to
> > >> > > > >> > > true
> > >> > > > >> > > >>>> when
> > >> > > > >> > > >>>> >> we
> > >> > > > >> > > >>>> >> >> feel ready.
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >> >> The URM is somewhat under-described here. I
> > think
> > >> we
> > >> > > > need a
> > >> > > > >> > few
> > >> > > > >> > > >>>> >> >> configurations here for it. For example, we
> > need a
> > >> > > > >> > > configuration to
> > >> > > > >> > > >>>> >> specify
> > >> > > > >> > > >>>> >> >> how long it should wait for a broker to
> respond
> > to
> > >> > its
> > >> > > > RPCs
> > >> > > > >> > > before
> > >> > > > >> > > >>>> >> moving
> > >> > > > >> > > >>>> >> >> on. We also need to understand how the URM
> > >> interacts
> > >> > > with
> > >> > > > >> > > >>>> >> >> unclean.leader.election.enable=always. I
> assume
> > >> that
> > >> > > with
> > >> > > > >> > > "always"
> > >> > > > >> > > >>>> we
> > >> > > > >> > > >>>> >> will
> > >> > > > >> > > >>>> >> >> just unconditionally use the URM rather than
> > >> choosing
> > >> > > > >> > randomly.
> > >> > > > >> > > >>>> But this
> > >> > > > >> > > >>>> >> >> should be spelled out in the KIP.
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >> >> >
> > >> > > > >> > > >>>> >> >> > DescribeTopicRequest
> > >> > > > >> > > >>>> >> >> >
> > >> > > > >> > > >>>> >> >> >    1.
> > >> > > > >> > > >>>> >> >> >    Yes, the plan is to replace the
> > >> MetadataRequest
> > >> > > with
> > >> > > > >> the
> > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest for the admin
> clients.
> > >> Will
> > >> > > > check
> > >> > > > >> > the
> > >> > > > >> > > >>>> details.
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >> >> Sounds good. But as I said, you need to
> specify
> > >> how
> > >> > > > >> > AdminClient
> > >> > > > >> > > >>>> >> interacts
> > >> > > > >> > > >>>> >> >> with the new request. This will involve adding
> > >> some
> > >> > > > fields
> > >> > > > >> to
> > >> > > > >> > > >>>> >> >> TopicDescription.java. And you need to specify
> > the
> > >> > > > changes
> > >> > > > >> to
> > >> > > > >> > > the
> > >> > > > >> > > >>>> >> >> kafka-topics.sh command line tool. Otherwise
> we
> > >> > cannot
> > >> > > > use
> > >> > > > >> the
> > >> > > > >> > > >>>> tool to
> > >> > > > >> > > >>>> >> see
> > >> > > > >> > > >>>> >> >> the new information.
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >> >> The new requests, DescribeTopicRequest and
> > >> > > > >> > > >>>> GetReplicaLogInfoRequest,
> > >> > > > >> > > >>>> >> need
> > >> > > > >> > > >>>> >> >> to have limits placed on them so that their
> size
> > >> > can't
> > >> > > be
> > >> > > > >> > > >>>> infinite. We
> > >> > > > >> > > >>>> >> >> don't want to propagate the current problems
> of
> > >> > > > >> > MetadataRequest,
> > >> > > > >> > > >>>> where
> > >> > > > >> > > >>>> >> >> clients can request massive responses that can
> > >> mess
> > >> > up
> > >> > > > the
> > >> > > > >> JVM
> > >> > > > >> > > when
> > >> > > > >> > > >>>> >> handled.
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >> >> Adding limits is simple for
> > >> GetReplicaLogInfoRequest
> > >> > --
> > >> > > > we
> > >> > > > >> can
> > >> > > > >> > > >>>> just say
> > >> > > > >> > > >>>> >> >> that only 2000 partitions at a time can be
> > >> requested.
> > >> > > For
> > >> > > > >> > > >>>> >> >> DescribeTopicRequest we can probably just
> limit
> > >> to 20
> > >> > > > topics
> > >> > > > >> > or
> > >> > > > >> > > >>>> >> something
> > >> > > > >> > > >>>> >> >> like that, to avoid the complexity of doing
> > >> > pagination
> > >> > > in
> > >> > > > >> this
> > >> > > > >> > > KIP.
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >> >> >    2.
> > >> > > > >> > > >>>> >> >> >    I can let the broker load the ELR info so
> > >> that
> > >> > > they
> > >> > > > can
> > >> > > > >> > > serve
> > >> > > > >> > > >>>> the
> > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest as well.
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >> >> Yes, it's fine to add to MetadataCache. In
> fact,
> > >> > you'll
> > >> > > > be
> > >> > > > >> > > loading
> > >> > > > >> > > >>>> it
> > >> > > > >> > > >>>> >> >> anyway once it's added to PartitionImage.
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >> >> >    3.
> > >> > > > >> > > >>>> >> >> >    Yeah, it does not make sense to have the
> > >> topic
> > >> > id
> > >> > > if
> > >> > > > >> > > >>>> >> >> >    DescribeTopicRequest is only used by the
> > >> admin
> > >> > > > client.
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >> >> OK. That makes things simpler. We can always
> > >> create a
> > >> > > new
> > >> > > > >> API
> > >> > > > >> > > later
> > >> > > > >> > > >>>> >> >> (hopefully not in this KIP!) to query by topic
> > ID.
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >> >> >
> > >> > > > >> > > >>>> >> >> >
> > >> > > > >> > > >>>> >> >> > Metrics
> > >> > > > >> > > >>>> >> >> >
> > >> > > > >> > > >>>> >> >> > As for overall cluster health metrics, I
> think
> > >> > > > >> under-min-ISR
> > >> > > > >> > > is
> > >> > > > >> > > >>>> still
> > >> > > > >> > > >>>> >> a
> > >> > > > >> > > >>>> >> >> > useful one. ELR is more like a safety belt.
> > When
> > >> > the
> > >> > > > ELR
> > >> > > > >> is
> > >> > > > >> > > >>>> used, the
> > >> > > > >> > > >>>> >> >> > cluster availability has already been
> > impacted.
> > >> > > > >> > > >>>> >> >> >
> > >> > > > >> > > >>>> >> >> > Maybe we can have a metric to count the
> > >> partitions
> > >> > > that
> > >> > > > >> > > sum(ISR,
> > >> > > > >> > > >>>> ELR)
> > >> > > > >> > > >>>> >> <
> > >> > > > >> > > >>>> >> >> min
> > >> > > > >> > > >>>> >> >> > ISR. What do you think?
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >> >> How about:
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >> >> A.  a metric for the totoal number of
> > >> under-min-isr
> > >> > > > >> > partitions?
> > >> > > > >> > > We
> > >> > > > >> > > >>>> don't
> > >> > > > >> > > >>>> >> >> have that in Apache Kafka at the moment.
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >> >> B. a metric for the number of unclean leader
> > >> > elections
> > >> > > we
> > >> > > > >> did
> > >> > > > >> > > (for
> > >> > > > >> > > >>>> >> >> simplicity, it can reset to 0 on controller
> > >> restart:
> > >> > we
> > >> > > > >> expect
> > >> > > > >> > > >>>> people to
> > >> > > > >> > > >>>> >> >> monitor the change over time anyway)
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >> >> best,
> > >> > > > >> > > >>>> >> >> Colin
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >> >> >
> > >> > > > >> > > >>>> >> >> > Yeah, for the ongoing unclean recoveries,
> the
> > >> > > > controller
> > >> > > > >> can
> > >> > > > >> > > >>>> keep an
> > >> > > > >> > > >>>> >> >> > accurate count through failover because
> > >> partition
> > >> > > > >> > registration
> > >> > > > >> > > >>>> can
> > >> > > > >> > > >>>> >> >> indicate
> > >> > > > >> > > >>>> >> >> > whether a recovery is needed. However, for
> the
> > >> > > happened
> > >> > > > >> > ones,
> > >> > > > >> > > >>>> unless
> > >> > > > >> > > >>>> >> we
> > >> > > > >> > > >>>> >> >> > want to persist the number somewhere, we can
> > >> only
> > >> > > > figure
> > >> > > > >> it
> > >> > > > >> > > out
> > >> > > > >> > > >>>> from
> > >> > > > >> > > >>>> >> the
> > >> > > > >> > > >>>> >> >> > log.
> > >> > > > >> > > >>>> >> >> >
> > >> > > > >> > > >>>> >> >> > On Tue, Sep 12, 2023 at 3:16 PM Colin
> McCabe <
> > >> > > > >> > > cmccabe@apache.org
> > >> > > > >> > > >>>> >
> > >> > > > >> > > >>>> >> wrote:
> > >> > > > >> > > >>>> >> >> >
> > >> > > > >> > > >>>> >> >> >> Also, we should have metrics that show what
> > is
> > >> > going
> > >> > > > on
> > >> > > > >> > with
> > >> > > > >> > > >>>> regard
> > >> > > > >> > > >>>> >> to
> > >> > > > >> > > >>>> >> >> the
> > >> > > > >> > > >>>> >> >> >> eligible replica set. I'm not sure exactly
> > >> what to
> > >> > > > >> suggest,
> > >> > > > >> > > but
> > >> > > > >> > > >>>> >> >> something
> > >> > > > >> > > >>>> >> >> >> that could identify when things are going
> > >> wrong in
> > >> > > the
> > >> > > > >> > > clsuter.
> > >> > > > >> > > >>>> >> >> >>
> > >> > > > >> > > >>>> >> >> >> For example, maybe a metric for partitions
> > >> > > containing
> > >> > > > >> > > replicas
> > >> > > > >> > > >>>> that
> > >> > > > >> > > >>>> >> are
> > >> > > > >> > > >>>> >> >> >> ineligible to be leader? That would show a
> > >> spike
> > >> > > when
> > >> > > > a
> > >> > > > >> > > broker
> > >> > > > >> > > >>>> had an
> > >> > > > >> > > >>>> >> >> >> unclean restart.
> > >> > > > >> > > >>>> >> >> >>
> > >> > > > >> > > >>>> >> >> >> Ideally, we'd also have a metric that
> > indicates
> > >> > when
> > >> > > > an
> > >> > > > >> > > unclear
> > >> > > > >> > > >>>> >> leader
> > >> > > > >> > > >>>> >> >> >> election or a recovery happened. It's a bit
> > >> tricky
> > >> > > > >> because
> > >> > > > >> > > the
> > >> > > > >> > > >>>> simple
> > >> > > > >> > > >>>> >> >> >> thing, of tracking it per controller, may
> be
> > a
> > >> bit
> > >> > > > >> > confusing
> > >> > > > >> > > >>>> during
> > >> > > > >> > > >>>> >> >> >> failovers.
> > >> > > > >> > > >>>> >> >> >>
> > >> > > > >> > > >>>> >> >> >> best,
> > >> > > > >> > > >>>> >> >> >> Colin
> > >> > > > >> > > >>>> >> >> >>
> > >> > > > >> > > >>>> >> >> >>
> > >> > > > >> > > >>>> >> >> >> On Tue, Sep 12, 2023, at 14:25, Colin
> McCabe
> > >> > wrote:
> > >> > > > >> > > >>>> >> >> >> > Hi Calvin,
> > >> > > > >> > > >>>> >> >> >> >
> > >> > > > >> > > >>>> >> >> >> > Thanks for the KIP. I think this is a
> great
> > >> > > > >> improvement.
> > >> > > > >> > > >>>> >> >> >> >
> > >> > > > >> > > >>>> >> >> >> >> Additional High Watermark advance
> > >> requirement
> > >> > > > >> > > >>>> >> >> >> >
> > >> > > > >> > > >>>> >> >> >> > Typo: change "advance" to "advancement"
> > >> > > > >> > > >>>> >> >> >> >
> > >> > > > >> > > >>>> >> >> >> >> A bit recap of some key concepts.
> > >> > > > >> > > >>>> >> >> >> >
> > >> > > > >> > > >>>> >> >> >> > Typo: change "bit" to "quick"
> > >> > > > >> > > >>>> >> >> >> >
> > >> > > > >> > > >>>> >> >> >> >> Ack=1/all produce request. It defines
> when
> > >> the
> > >> > > > Kafka
> > >> > > > >> > > server
> > >> > > > >> > > >>>> should
> > >> > > > >> > > >>>> >> >> >> respond to the produce request
> > >> > > > >> > > >>>> >> >> >> >
> > >> > > > >> > > >>>> >> >> >> > I think this section would be clearer if
> we
> > >> > talked
> > >> > > > >> about
> > >> > > > >> > > the
> > >> > > > >> > > >>>> new
> > >> > > > >> > > >>>> >> high
> > >> > > > >> > > >>>> >> >> >> > watermark advancement requirement first,
> > and
> > >> > THEN
> > >> > > > >> talked
> > >> > > > >> > > >>>> about its
> > >> > > > >> > > >>>> >> >> >> > impact on acks=0, acks=1, and
>  acks=all.
> > >> > > > acks=all
> > >> > > > >> is
> > >> > > > >> > of
> > >> > > > >> > > >>>> course
> > >> > > > >> > > >>>> >> the
> > >> > > > >> > > >>>> >> >> >> > main case we care about here, so it would
> > be
> > >> > good
> > >> > > to
> > >> > > > >> lead
> > >> > > > >> > > with
> > >> > > > >> > > >>>> >> that,
> > >> > > > >> > > >>>> >> >> >> > rather than delving into the
> technicalities
> > >> of
> > >> > > > acks=0/1
> > >> > > > >> > > first.
> > >> > > > >> > > >>>> >> >> >> >
> > >> > > > >> > > >>>> >> >> >> >> Unclean recovery
> > >> > > > >> > > >>>> >> >> >> >
> > >> > > > >> > > >>>> >> >> >> > So, here you are introducing a new
> > >> > configuration,
> > >> > > > >> > > >>>> >> >> >> > unclean.recovery.strategy. The difficult
> > >> thing
> > >> > > here
> > >> > > > is
> > >> > > > >> > that
> > >> > > > >> > > >>>> there
> > >> > > > >> > > >>>> >> is a
> > >> > > > >> > > >>>> >> >> >> > lot of overlap with
> > >> > > unclean.leader.election.enable.
> > >> > > > So
> > >> > > > >> we
> > >> > > > >> > > >>>> have 3
> > >> > > > >> > > >>>> >> >> >> > different settings for
> > >> > unclean.recovery.strategy,
> > >> > > > plus
> > >> > > > >> 2
> > >> > > > >> > > >>>> different
> > >> > > > >> > > >>>> >> >> >> > settings for
> > unclean.leader.election.enable,
> > >> > > giving
> > >> > > > a
> > >> > > > >> > cross
> > >> > > > >> > > >>>> >> product of
> > >> > > > >> > > >>>> >> >> >> > 6 different options. The following
> "unclean
> > >> > > recovery
> > >> > > > >> > > manager"
> > >> > > > >> > > >>>> >> section
> > >> > > > >> > > >>>> >> >> >> > only applies to one fo those 6 different
> > >> > > > possibilities
> > >> > > > >> (I
> > >> > > > >> > > >>>> think?)
> > >> > > > >> > > >>>> >> >> >> >
> > >> > > > >> > > >>>> >> >> >> > I simply don't think we need so many
> > >> different
> > >> > > > election
> > >> > > > >> > > types.
> > >> > > > >> > > >>>> >> Really
> > >> > > > >> > > >>>> >> >> >> > the use-cases we need are people who want
> > NO
> > >> > > unclean
> > >> > > > >> > > >>>> elections,
> > >> > > > >> > > >>>> >> people
> > >> > > > >> > > >>>> >> >> >> > who want "the reasonable thing" and
> people
> > >> who
> > >> > > want
> > >> > > > >> > > >>>> avaialbility at
> > >> > > > >> > > >>>> >> >> all
> > >> > > > >> > > >>>> >> >> >> > costs.
> > >> > > > >> > > >>>> >> >> >> >
> > >> > > > >> > > >>>> >> >> >> > Overall, I feel like the first half of
> the
> > >> KIP
> > >> > is
> > >> > > > about
> > >> > > > >> > the
> > >> > > > >> > > >>>> ELR,
> > >> > > > >> > > >>>> >> and
> > >> > > > >> > > >>>> >> >> >> > the second half is about reworking
> unclean
> > >> > leader
> > >> > > > >> > > election. It
> > >> > > > >> > > >>>> >> might
> > >> > > > >> > > >>>> >> >> be
> > >> > > > >> > > >>>> >> >> >> > better to move that second half to a
> > separate
> > >> > KIP
> > >> > > so
> > >> > > > >> that
> > >> > > > >> > > we
> > >> > > > >> > > >>>> can
> > >> > > > >> > > >>>> >> >> figure
> > >> > > > >> > > >>>> >> >> >> > it out fully. It should be fine to punt
> > this
> > >> > until
> > >> > > > >> later
> > >> > > > >> > > and
> > >> > > > >> > > >>>> just
> > >> > > > >> > > >>>> >> have
> > >> > > > >> > > >>>> >> >> >> > the current behavior on empty ELR be
> > waiting
> > >> for
> > >> > > the
> > >> > > > >> last
> > >> > > > >> > > >>>> known
> > >> > > > >> > > >>>> >> leader
> > >> > > > >> > > >>>> >> >> >> > to return. After all, that's what we do
> > >> today.
> > >> > > > >> > > >>>> >> >> >> >
> > >> > > > >> > > >>>> >> >> >> >> DescribeTopicRequest
> > >> > > > >> > > >>>> >> >> >> >
> > >> > > > >> > > >>>> >> >> >> > Is the intention for AdminClient to use
> > this
> > >> RPC
> > >> > > for
> > >> > > > >> > > >>>> >> >> >> > Admin#describeTopics ? If so, we need to
> > >> > describe
> > >> > > > all
> > >> > > > >> of
> > >> > > > >> > > the
> > >> > > > >> > > >>>> >> changes
> > >> > > > >> > > >>>> >> >> to
> > >> > > > >> > > >>>> >> >> >> > the admin client API, as well as changes
> to
> > >> > > > >> command-line
> > >> > > > >> > > >>>> tools like
> > >> > > > >> > > >>>> >> >> >> > kafka-topics.sh (if there are any). For
> > >> example,
> > >> > > you
> > >> > > > >> will
> > >> > > > >> > > >>>> probably
> > >> > > > >> > > >>>> >> >> need
> > >> > > > >> > > >>>> >> >> >> > changes to TopicDescription.java. You
> will
> > >> also
> > >> > > > need to
> > >> > > > >> > > >>>> provide
> > >> > > > >> > > >>>> >> all of
> > >> > > > >> > > >>>> >> >> >> > the things that admin client needs -- for
> > >> > example,
> > >> > > > >> > > >>>> >> >> >> > TopicAuthorizedOperations.
> > >> > > > >> > > >>>> >> >> >> >
> > >> > > > >> > > >>>> >> >> >> > I also don't think the controller should
> > >> serve
> > >> > > this
> > >> > > > >> > > request.
> > >> > > > >> > > >>>> We
> > >> > > > >> > > >>>> >> want
> > >> > > > >> > > >>>> >> >> to
> > >> > > > >> > > >>>> >> >> >> > minimize load on the controller. Just
> like
> > >> with
> > >> > > the
> > >> > > > >> other
> > >> > > > >> > > >>>> metadata
> > >> > > > >> > > >>>> >> >> >> > requests like MetadataRequest, this
> should
> > be
> > >> > > > served by
> > >> > > > >> > > >>>> brokers.
> > >> > > > >> > > >>>> >> >> >> >
> > >> > > > >> > > >>>> >> >> >> > It's a bit confusing why both topic ID
> and
> > >> topic
> > >> > > > name
> > >> > > > >> are
> > >> > > > >> > > >>>> provided
> > >> > > > >> > > >>>> >> to
> > >> > > > >> > > >>>> >> >> >> > this API. Is the intention that callers
> > >> should
> > >> > set
> > >> > > > one
> > >> > > > >> > but
> > >> > > > >> > > >>>> not the
> > >> > > > >> > > >>>> >> >> >> > other? Or both? This needs to be
> clarified.
> > >> > Also,
> > >> > > > if we
> > >> > > > >> > do
> > >> > > > >> > > >>>> want to
> > >> > > > >> > > >>>> >> >> >> > support lookups by UUID, that is another
> > >> thing
> > >> > > that
> > >> > > > >> needs
> > >> > > > >> > > to
> > >> > > > >> > > >>>> be
> > >> > > > >> > > >>>> >> added
> > >> > > > >> > > >>>> >> >> >> > to adminclient.
> > >> > > > >> > > >>>> >> >> >> >
> > >> > > > >> > > >>>> >> >> >> > In general, I feel like this should also
> > >> > probably
> > >> > > be
> > >> > > > >> its
> > >> > > > >> > > own
> > >> > > > >> > > >>>> KIP
> > >> > > > >> > > >>>> >> since
> > >> > > > >> > > >>>> >> >> >> > it's fairly complex
> > >> > > > >> > > >>>> >> >> >> >
> > >> > > > >> > > >>>> >> >> >> > best,
> > >> > > > >> > > >>>> >> >> >> > Colin
> > >> > > > >> > > >>>> >> >> >> >
> > >> > > > >> > > >>>> >> >> >> >
> > >> > > > >> > > >>>> >> >> >> > On Thu, Aug 10, 2023, at 15:46, Calvin
> Liu
> > >> > 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
> > >> > > > >> > > >>>> >> >> >>
> > >> > > > >> > > >>>> >> >>
> > >> > > > >> > > >>>> >>
> > >> > > > >> > > >>>>
> > >> > > > >> > > >>>
> > >> > > > >> > >
> > >> > > > >> >
> > >> > > > >>
> > >> > > >
> > >> > >
> > >> >
> > >>
> > >
> >
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Jun Rao <ju...@confluent.io.INVALID>.
Hi, Calvin and Justine,

Historically, when we change the record format in the log, we don't support
software version downgrading.

For the record format change in the metadata log, have we thought about
forcing the write of the latest metadata records with the old version
during MV downgrading? This will in theory allow the old version of the
software to obtain the latest metadata.

Thanks,

Jun

On Wed, Oct 4, 2023 at 9:53 AM Justine Olshan <jo...@confluent.io.invalid>
wrote:

> Sorry -- not MV but software version.
>
> On Wed, Oct 4, 2023 at 9:51 AM Justine Olshan <jo...@confluent.io>
> wrote:
>
> > Catching up with this discussion.
> >
> > I was just curious -- have we had other instances where downgrading MV is
> > not supported? I think Kafka typically tries to support downgrades, and I
> > couldn't think of other examples.
> >
> > Thanks,
> > Justine
> >
> > On Wed, Oct 4, 2023 at 9:40 AM Calvin Liu <ca...@confluent.io.invalid>
> > wrote:
> >
> >> Hi Jun,
> >> 54. Marked the software downgrading is not supported. As the old
> >> controller
> >> will not understand the new PartitionRecord and PartitionChangeRecord.
> >> Thanks!
> >>
> >> On Wed, Oct 4, 2023 at 9:12 AM Jun Rao <ju...@confluent.io.invalid>
> wrote:
> >>
> >> > Hi, Calvin,
> >> >
> >> > Thanks for the reply. Just one more comment.
> >> >
> >> > 54. It seems that downgrading MV is supported. Is downgrading the
> >> software
> >> > version supported? It would be useful to document that.
> >> >
> >> > Thanks,
> >> >
> >> > Jun
> >> >
> >> > On Tue, Oct 3, 2023 at 4:55 PM Artem Livshits
> >> > <al...@confluent.io.invalid> wrote:
> >> >
> >> > > Hi Colin,
> >> > >
> >> > > I think in your example "do_unclean_recovery" would need to do
> >> different
> >> > > things depending on the strategy.
> >> > >
> >> > > do_unclean_recovery() {
> >> > >    if (unclean.recovery.manager.enabled) {
> >> > >     if (strategy == Aggressive)
> >> > >       use UncleanRecoveryManager(waitLastKnownERL=false)  // just
> >> inspect
> >> > > logs from whoever is available
> >> > >     else
> >> > >       use  UncleanRecoveryManager(waitLastKnownERL=true)  // must
> wait
> >> > for
> >> > > at least last known ELR
> >> > >   } else {
> >> > >     if (strategy == Aggressive)
> >> > >       choose the last known leader if that is available, or a random
> >> > leader
> >> > > if not)
> >> > >     else
> >> > >       wait for last known leader to get back
> >> > >   }
> >> > > }
> >> > >
> >> > > The idea is that the Aggressive strategy would kick in as soon as we
> >> lost
> >> > > the leader and would pick a leader from whoever is available; but
> the
> >> > > Balanced will only kick in when ELR is empty and will wait for the
> >> > brokers
> >> > > that likely have most data to be available.
> >> > >
> >> > > On Tue, Oct 3, 2023 at 3:04 PM Colin McCabe <cm...@apache.org>
> >> wrote:
> >> > >
> >> > > > On Tue, Oct 3, 2023, at 10:49, Jun Rao wrote:
> >> > > > > Hi, Calvin,
> >> > > > >
> >> > > > > Thanks for the update KIP. A few more comments.
> >> > > > >
> >> > > > > 41. Why would a user choose the option to select a random
> replica
> >> as
> >> > > the
> >> > > > > leader instead of using unclean.recovery.strateg=Aggressive? It
> >> seems
> >> > > > that
> >> > > > > the latter is strictly better? If that's not the case, could we
> >> fold
> >> > > this
> >> > > > > option under unclean.recovery.strategy instead of introducing a
> >> > > separate
> >> > > > > config?
> >> > > >
> >> > > > Hi Jun,
> >> > > >
> >> > > > I thought the flow of control was:
> >> > > >
> >> > > > If there is no leader for the partition {
> >> > > >   If (there are unfenced ELR members) {
> >> > > >     choose_an_unfenced_ELR_member
> >> > > >   } else if (there are fenced ELR members AND
> strategy=Aggressive) {
> >> > > >     do_unclean_recovery
> >> > > >   } else if (there are no ELR members AND strategy != None) {
> >> > > >     do_unclean_recovery
> >> > > >   } else {
> >> > > >     do nothing about the missing leader
> >> > > >   }
> >> > > > }
> >> > > >
> >> > > > do_unclean_recovery() {
> >> > > >    if (unclean.recovery.manager.enabled) {
> >> > > >     use UncleanRecoveryManager
> >> > > >   } else {
> >> > > >     choose the last known leader if that is available, or a random
> >> > leader
> >> > > > if not)
> >> > > >   }
> >> > > > }
> >> > > >
> >> > > > However, I think this could be clarified, especially the behavior
> >> when
> >> > > > unclean.recovery.manager.enabled=false. Inuitively the goal for
> >> > > > unclean.recovery.manager.enabled=false is to be "the same as now,
> >> > mostly"
> >> > > > but it's very underspecified in the KIP, I agree.
> >> > > >
> >> > > > >
> >> > > > > 50. ElectLeadersRequest: "If more than 20 topics are included,
> >> only
> >> > the
> >> > > > > first 20 will be served. Others will be returned with
> >> > DesiredLeaders."
> >> > > > Hmm,
> >> > > > > not sure that I understand this. ElectLeadersResponse doesn't
> >> have a
> >> > > > > DesiredLeaders field.
> >> > > > >
> >> > > > > 51. GetReplicaLogInfo: "If more than 2000 partitions are
> included,
> >> > only
> >> > > > the
> >> > > > > first 2000 will be served" Do we return an error for the
> remaining
> >> > > > > partitions? Actually, should we include an errorCode field at
> the
> >> > > > partition
> >> > > > > level in GetReplicaLogInfoResponse to cover non-existing
> >> partitions
> >> > and
> >> > > > no
> >> > > > > authorization, etc?
> >> > > > >
> >> > > > > 52. The entry should matches => The entry should match
> >> > > > >
> >> > > > > 53. ElectLeadersRequest.DesiredLeaders: Should it be nullable
> >> since a
> >> > > > user
> >> > > > > may not specify DesiredLeaders?
> >> > > > >
> >> > > > > 54. Downgrade: Is that indeed possible? I thought earlier you
> said
> >> > that
> >> > > > > once the new version of the records are in the metadata log, one
> >> > can't
> >> > > > > downgrade since the old broker doesn't know how to parse the new
> >> > > version
> >> > > > of
> >> > > > > the metadata records?
> >> > > > >
> >> > > >
> >> > > > MetadataVersion downgrade is currently broken but we have fixing
> it
> >> on
> >> > > our
> >> > > > plate for Kafka 3.7.
> >> > > >
> >> > > > The way downgrade works is that "new features" are dropped,
> leaving
> >> > only
> >> > > > the old ones.
> >> > > >
> >> > > > > 55. CleanShutdownFile: Should we add a version field for future
> >> > > > extension?
> >> > > > >
> >> > > > > 56. Config changes are public facing. Could we have a separate
> >> > section
> >> > > to
> >> > > > > document all the config changes?
> >> > > >
> >> > > > +1. A separate section for this would be good.
> >> > > >
> >> > > > best,
> >> > > > Colin
> >> > > >
> >> > > > >
> >> > > > > Thanks,
> >> > > > >
> >> > > > > Jun
> >> > > > >
> >> > > > > On Mon, Sep 25, 2023 at 4:29 PM Calvin Liu
> >> > <caliu@confluent.io.invalid
> >> > > >
> >> > > > > wrote:
> >> > > > >
> >> > > > >> Hi Jun
> >> > > > >> Thanks for the comments.
> >> > > > >>
> >> > > > >> 40. If we change to None, it is not guaranteed for no data
> loss.
> >> For
> >> > > > users
> >> > > > >> who are not able to validate the data with external resources,
> >> > manual
> >> > > > >> intervention does not give a better result but a loss of
> >> > availability.
> >> > > > So
> >> > > > >> practically speaking, the Balance mode would be a better
> default
> >> > > value.
> >> > > > >>
> >> > > > >> 41. No, it represents how we want to do the unclean leader
> >> election.
> >> > > If
> >> > > > it
> >> > > > >> is false, the unclean leader election will be the old random
> way.
> >> > > > >> Otherwise, the unclean recovery will be used.
> >> > > > >>
> >> > > > >> 42. Good catch. Updated.
> >> > > > >>
> >> > > > >> 43. Only the first 20 topics will be served. Others will be
> >> returned
> >> > > > with
> >> > > > >> InvalidRequestError
> >> > > > >>
> >> > > > >> 44. The order matters. The desired leader entries match with
> the
> >> > topic
> >> > > > >> partition list by the index.
> >> > > > >>
> >> > > > >> 45. Thanks! Updated.
> >> > > > >>
> >> > > > >> 46. Good advice! Updated.
> >> > > > >>
> >> > > > >> 47.1, updated the comment. Basically it will elect the replica
> in
> >> > the
> >> > > > >> desiredLeader field to be the leader
> >> > > > >>
> >> > > > >> 47.2 We can let the admin client do the conversion. Using the
> >> > > > desiredLeader
> >> > > > >> field in the json format seems easier for users.
> >> > > > >>
> >> > > > >> 48. Once the MV version is downgraded, all the ELR related
> fields
> >> > will
> >> > > > be
> >> > > > >> removed on the next partition change. The controller will also
> >> > ignore
> >> > > > the
> >> > > > >> ELR fields. Updated the KIP.
> >> > > > >>
> >> > > > >> 49. Yes, it would be deprecated/removed.
> >> > > > >>
> >> > > > >>
> >> > > > >> On Mon, Sep 25, 2023 at 3:49 PM Jun Rao
> <jun@confluent.io.invalid
> >> >
> >> > > > wrote:
> >> > > > >>
> >> > > > >> > Hi, Calvin,
> >> > > > >> >
> >> > > > >> > Thanks for the updated KIP. Made another pass. A few more
> >> comments
> >> > > > below.
> >> > > > >> >
> >> > > > >> > 40. unclean.leader.election.enable.false ->
> >> > > > >> > unclean.recovery.strategy.Balanced: The Balanced mode could
> >> still
> >> > > > lead to
> >> > > > >> > data loss. So, I am wondering if
> >> > > unclean.leader.election.enable.false
> >> > > > >> > should map to None?
> >> > > > >> >
> >> > > > >> > 41. unclean.recovery.manager.enabled: I am not sure why we
> >> > introduce
> >> > > > this
> >> > > > >> > additional config. Is it the same as
> >> > unclean.recovery.strategy=None?
> >> > > > >> >
> >> > > > >> > 42. DescribeTopicResponse.TopicAuthorizedOperations: Should
> >> this
> >> > be
> >> > > at
> >> > > > >> the
> >> > > > >> > topic level?
> >> > > > >> >
> >> > > > >> > 43. "Limit: 20 topics max per request": Could we describe
> what
> >> > > > happens if
> >> > > > >> > the request includes more than 20 topics?
> >> > > > >> >
> >> > > > >> > 44. ElectLeadersRequest.DesiredLeaders: Could we describe
> >> whether
> >> > > the
> >> > > > >> > ordering matters?
> >> > > > >> >
> >> > > > >> > 45. GetReplicaLogInfo.TopicPartitions: "about": "The topic
> >> > > partitions
> >> > > > to
> >> > > > >> > elect leaders.": The description in "about" is incorrect.
> >> > > > >> >
> >> > > > >> > 46. GetReplicaLogInfoResponse: Should we nest partitions
> under
> >> > > > topicId to
> >> > > > >> > be consistent with other types of responses?
> >> > > > >> >
> >> > > > >> > 47. kafka-leader-election.sh:
> >> > > > >> > 47.1 Could we explain DESIGNATION?
> >> > > > >> > 47.2 desiredLeader: Should it be a list to match the field in
> >> > > > >> > ElectLeadersRequest?
> >> > > > >> >
> >> > > > >> > 48. We could add a section on downgrade?
> >> > > > >> >
> >> > > > >> > 49. LastKnownLeader: This seems only needed in the first
> phase
> >> of
> >> > > > >> > delivering ELR. Will it be removed when the complete KIP is
> >> > > delivered?
> >> > > > >> >
> >> > > > >> > Thanks,
> >> > > > >> >
> >> > > > >> > Jun
> >> > > > >> >
> >> > > > >> > On Tue, Sep 19, 2023 at 1:30 PM Colin McCabe <
> >> cmccabe@apache.org>
> >> > > > wrote:
> >> > > > >> >
> >> > > > >> > > Hi Calvin,
> >> > > > >> > >
> >> > > > >> > > Thanks for the explanations. I like the idea of using none,
> >> > > > balanced,
> >> > > > >> > > aggressive. We also had an offline discussion about why it
> is
> >> > good
> >> > > > to
> >> > > > >> > use a
> >> > > > >> > > new config key (basically, so that we can deprecate the old
> >> one
> >> > > > which
> >> > > > >> had
> >> > > > >> > > only false/true values in 4.0) With these changes, I am +1.
> >> > > > >> > >
> >> > > > >> > > best,
> >> > > > >> > > Colin
> >> > > > >> > >
> >> > > > >> > > On Mon, Sep 18, 2023, at 15:54, Calvin Liu wrote:
> >> > > > >> > > > Hi Colin,
> >> > > > >> > > > Also, can we deprecate unclean.leader.election.enable in
> >> 4.0?
> >> > > > Before
> >> > > > >> > > that,
> >> > > > >> > > > we can have both the config unclean.recovery.strategy and
> >> > > > >> > > > unclean.leader.election.enable
> >> > > > >> > > > and using the unclean.recovery.Enabled to determine which
> >> > config
> >> > > > to
> >> > > > >> use
> >> > > > >> > > > during the unclean leader election.
> >> > > > >> > > >
> >> > > > >> > > > On Mon, Sep 18, 2023 at 3:51 PM Calvin Liu <
> >> > caliu@confluent.io>
> >> > > > >> wrote:
> >> > > > >> > > >
> >> > > > >> > > >> Hi Colin,
> >> > > > >> > > >> For the unclean.recovery.strategy config name, how about
> >> we
> >> > use
> >> > > > the
> >> > > > >> > > >> following
> >> > > > >> > > >> None. It basically means no unclean recovery will be
> >> > performed.
> >> > > > >> > > >> Aggressive. It means availability goes first. Whenever
> the
> >> > > > partition
> >> > > > >> > > can't
> >> > > > >> > > >> elect a durable replica, the controller will try the
> >> unclean
> >> > > > >> recovery.
> >> > > > >> > > >> Balanced. It is the balance point of the availability
> >> > > > >> > first(Aggressive)
> >> > > > >> > > >> and least availability(None). The controller performs
> >> unclean
> >> > > > >> recovery
> >> > > > >> > > when
> >> > > > >> > > >> both ISR and ELR are empty.
> >> > > > >> > > >>
> >> > > > >> > > >>
> >> > > > >> > > >> On Fri, Sep 15, 2023 at 11:42 AM Calvin Liu <
> >> > > caliu@confluent.io>
> >> > > > >> > wrote:
> >> > > > >> > > >>
> >> > > > >> > > >>> Hi Colin,
> >> > > > >> > > >>>
> >> > > > >> > > >>> > So, the proposal is that if someone sets
> >> > > > >> > > "unclean.leader.election.enable
> >> > > > >> > > >>> = true"...
> >> > > > >> > > >>>
> >> > > > >> > > >>>
> >> > > > >> > > >>> The idea is to use one of the
> >> unclean.leader.election.enable
> >> > > and
> >> > > > >> > > >>> unclean.recovery.strategy based on the
> >> > > > unclean.recovery.Enabled. A
> >> > > > >> > > possible
> >> > > > >> > > >>> version can be
> >> > > > >> > > >>>
> >> > > > >> > > >>> If unclean.recovery.Enabled:
> >> > > > >> > > >>>
> >> > > > >> > > >>> {
> >> > > > >> > > >>>
> >> > > > >> > > >>> Check unclean.recovery.strategy. If set, use it.
> >> Otherwise,
> >> > > > check
> >> > > > >> > > >>> unclean.leader.election.enable and translate it to
> >> > > > >> > > >>> unclean.recovery.strategy.
> >> > > > >> > > >>>
> >> > > > >> > > >>> } else {
> >> > > > >> > > >>>
> >> > > > >> > > >>> Use unclean.leader.election.enable
> >> > > > >> > > >>>
> >> > > > >> > > >>> }
> >> > > > >> > > >>>
> >> > > > >> > > >>>
> >> > > > >> > > >>> —--------
> >> > > > >> > > >>>
> >> > > > >> > > >>> >The configuration key should be
> >> > > > >> "unclean.recovery.manager.enabled",
> >> > > > >> > > >>> right?
> >> > > > >> > > >>>
> >> > > > >> > > >>>
> >> > > > >> > > >>> I think we have two ways of choosing a leader
> uncleanly,
> >> > > unclean
> >> > > > >> > leader
> >> > > > >> > > >>> election and unclean recovery(log inspection) and we
> try
> >> to
> >> > > > switch
> >> > > > >> > > between
> >> > > > >> > > >>> them.
> >> > > > >> > > >>>
> >> > > > >> > > >>> Do you mean we want to develop two ways of performing
> the
> >> > > > unclean
> >> > > > >> > > >>> recovery and one of them is using “unclean recovery
> >> > manager”?
> >> > > I
> >> > > > >> guess
> >> > > > >> > > we
> >> > > > >> > > >>> haven’t discussed the second way.
> >> > > > >> > > >>>
> >> > > > >> > > >>>
> >> > > > >> > > >>> —-------
> >> > > > >> > > >>>
> >> > > > >> > > >>> >How do these 4 levels of overrides interact with your
> >> new
> >> > > > >> > > >>> configurations?
> >> > > > >> > > >>>
> >> > > > >> > > >>>
> >> > > > >> > > >>> I do notice in the Kraft controller code, the method to
> >> > check
> >> > > > >> whether
> >> > > > >> > > >>> perform unclean leader election is hard coded to false
> >> since
> >> > > > >> > > >>> 2021(uncleanLeaderElectionEnabledForTopic). Isn’t it a
> >> good
> >> > > > chance
> >> > > > >> to
> >> > > > >> > > >>> completely deprecate the
> unclean.leader.election.enable?
> >> We
> >> > > > don’t
> >> > > > >> > even
> >> > > > >> > > have
> >> > > > >> > > >>> to worry about the config conversion.
> >> > > > >> > > >>>
> >> > > > >> > > >>> On the other hand, whatever the override is, as long as
> >> the
> >> > > > >> > controller
> >> > > > >> > > >>> can have the final effective
> >> unclean.leader.election.enable,
> >> > > the
> >> > > > >> > topic
> >> > > > >> > > >>> level config unclean.recovery.strategy, the cluster
> level
> >> > > config
> >> > > > >> > > >>> unclean.recovery.Enabled, the controller can calculate
> >> the
> >> > > > correct
> >> > > > >> > > methods
> >> > > > >> > > >>> to use right?
> >> > > > >> > > >>>
> >> > > > >> > > >>>
> >> > > > >> > > >>> On Fri, Sep 15, 2023 at 10:02 AM Colin McCabe <
> >> > > > cmccabe@apache.org>
> >> > > > >> > > wrote:
> >> > > > >> > > >>>
> >> > > > >> > > >>>> On Thu, Sep 14, 2023, at 22:23, Calvin Liu wrote:
> >> > > > >> > > >>>> > Hi Colin
> >> > > > >> > > >>>> > 1. I think using the new config name is more clear.
> >> > > > >> > > >>>> >        a. The unclean leader election is actually
> >> removed
> >> > > if
> >> > > > >> > unclean
> >> > > > >> > > >>>> > recovery is in use.
> >> > > > >> > > >>>> >        b. Using multiple values in
> >> > > > >> unclean.leader.election.enable
> >> > > > >> > is
> >> > > > >> > > >>>> > confusing and it will be more confusing after people
> >> > forget
> >> > > > >> about
> >> > > > >> > > this
> >> > > > >> > > >>>> > discussion.
> >> > > > >> > > >>>>
> >> > > > >> > > >>>> Hi Calvin,
> >> > > > >> > > >>>>
> >> > > > >> > > >>>> So, the proposal is that if someone sets
> >> > > > >> > > "unclean.leader.election.enable
> >> > > > >> > > >>>> = true" but then sets one of your new configurations,
> >> the
> >> > > > value of
> >> > > > >> > > >>>> unclean.leader.election.enable is ignored? That seems
> >> less
> >> > > > clear
> >> > > > >> to
> >> > > > >> > > me, not
> >> > > > >> > > >>>> more. Just in general, having multiple configuration
> >> keys
> >> > to
> >> > > > >> control
> >> > > > >> > > the
> >> > > > >> > > >>>> same thing confuses users. Basically, they are sitting
> >> at a
> >> > > > giant
> >> > > > >> > > control
> >> > > > >> > > >>>> panel, and some of the levers do nothing.
> >> > > > >> > > >>>>
> >> > > > >> > > >>>> > 2. Sorry I forgot to mention in the response that I
> >> did
> >> > add
> >> > > > the
> >> > > > >> > > >>>> > unclean.recovery.Enabled flag.
> >> > > > >> > > >>>>
> >> > > > >> > > >>>> The configuration key should be
> >> > > > >> "unclean.recovery.manager.enabled",
> >> > > > >> > > >>>> right? Becuase we can do "unclean recovery" without
> the
> >> > > > manager.
> >> > > > >> > > Disabling
> >> > > > >> > > >>>> the manager just means we use a different mechanism
> for
> >> > > > recovery.
> >> > > > >> > > >>>>
> >> > > > >> > > >>>> >        c. Maybe I underestimated the challenge of
> >> > replacing
> >> > > > the
> >> > > > >> > > >>>> config. Any
> >> > > > >> > > >>>> > implementation problems ahead?
> >> > > > >> > > >>>>
> >> > > > >> > > >>>> There are four levels of overrides for
> >> > > > >> > unclean.leader.election.enable.
> >> > > > >> > > >>>>
> >> > > > >> > > >>>> 1. static configuration for node.
> >> > > > >> > > >>>>     This goes in the configuration file, typically
> named
> >> > > > >> > > >>>> server.properties
> >> > > > >> > > >>>>
> >> > > > >> > > >>>> 2. dynamic configuration for node default
> >> > > > >> > > >>>>   ConfigResource(type=BROKER, name="")
> >> > > > >> > > >>>>
> >> > > > >> > > >>>> 3. dynamic configuration for node
> >> > > > >> > > >>>>   ConfigResource(type=BROKER, name=<controller id>)
> >> > > > >> > > >>>>
> >> > > > >> > > >>>> 4. dynamic configuration for topic
> >> > > > >> > > >>>>   ConfigResource(type=TOPIC, name=<topic-name>)
> >> > > > >> > > >>>>
> >> > > > >> > > >>>> How do these 4 levels of overrides interact with your
> >> new
> >> > > > >> > > >>>> configurations? If the new configurations dominate
> over
> >> the
> >> > > old
> >> > > > >> > ones,
> >> > > > >> > > it
> >> > > > >> > > >>>> seems like this will get a lot more confusing to
> >> implement
> >> > > (and
> >> > > > >> also
> >> > > > >> > > to
> >> > > > >> > > >>>> use.)
> >> > > > >> > > >>>>
> >> > > > >> > > >>>> Again, I'd recommend just adding some new values to
> >> > > > >> > > >>>> unclean.leader.election.enable. It's simple and will
> >> > prevent
> >> > > > user
> >> > > > >> > > confusion
> >> > > > >> > > >>>> (as well as developer confusion.)
> >> > > > >> > > >>>>
> >> > > > >> > > >>>> best,
> >> > > > >> > > >>>> Colin
> >> > > > >> > > >>>>
> >> > > > >> > > >>>>
> >> > > > >> > > >>>> > 3. About the admin client, I mentioned 3 changes in
> >> the
> >> > > > client.
> >> > > > >> > > >>>> Anything
> >> > > > >> > > >>>> > else I missed in the KIP?
> >> > > > >> > > >>>> >       a. The client will switch to using the new RPC
> >> > > instead
> >> > > > of
> >> > > > >> > > >>>> > MetadataRequest for the topics.
> >> > > > >> > > >>>> >       b. The TopicPartitionInfo used in
> >> TopicDescription
> >> > > > needs
> >> > > > >> to
> >> > > > >> > > add
> >> > > > >> > > >>>> new
> >> > > > >> > > >>>> > fields related to the ELR.
> >> > > > >> > > >>>> >       c. The outputs will add the ELR related
> fields.
> >> > > > >> > > >>>> >
> >> > > > >> > > >>>> > On Thu, Sep 14, 2023 at 9:19 PM Colin McCabe <
> >> > > > >> cmccabe@apache.org>
> >> > > > >> > > >>>> wrote:
> >> > > > >> > > >>>> >
> >> > > > >> > > >>>> >> Hi Calvin,
> >> > > > >> > > >>>> >>
> >> > > > >> > > >>>> >> Thanks for the changes.
> >> > > > >> > > >>>> >>
> >> > > > >> > > >>>> >> 1. Earlier I commented that creating
> >> > > > >> "unclean.recovery.strategy "
> >> > > > >> > > is
> >> > > > >> > > >>>> not
> >> > > > >> > > >>>> >> necessary, and we can just reuse the existing
> >> > > > >> > > >>>> >> "unclean.leader.election.enable" configuration key.
> >> > Let's
> >> > > > >> discuss
> >> > > > >> > > >>>> that.
> >> > > > >> > > >>>> >>
> >> > > > >> > > >>>> >> 2.I also don't understand why you didn't add a
> >> > > > configuration to
> >> > > > >> > > >>>> enable or
> >> > > > >> > > >>>> >> disable the Unclean Recovery Manager. This seems
> >> like a
> >> > > very
> >> > > > >> > simple
> >> > > > >> > > >>>> way to
> >> > > > >> > > >>>> >> handle the staging issue which we discussed. The
> URM
> >> can
> >> > > > just
> >> > > > >> be
> >> > > > >> > > >>>> turned off
> >> > > > >> > > >>>> >> until it is production ready. Let's discuss this.
> >> > > > >> > > >>>> >>
> >> > > > >> > > >>>> >> 3. You still need to describe the changes to
> >> AdminClient
> >> > > > that
> >> > > > >> are
> >> > > > >> > > >>>> needed
> >> > > > >> > > >>>> >> to use DescribeTopicRequest.
> >> > > > >> > > >>>> >>
> >> > > > >> > > >>>> >> Keep at it. It's looking better. :)
> >> > > > >> > > >>>> >>
> >> > > > >> > > >>>> >> best,
> >> > > > >> > > >>>> >> Colin
> >> > > > >> > > >>>> >>
> >> > > > >> > > >>>> >>
> >> > > > >> > > >>>> >> On Thu, Sep 14, 2023, at 11:03, Calvin Liu wrote:
> >> > > > >> > > >>>> >> > Hi Colin
> >> > > > >> > > >>>> >> > Thanks for the comments!
> >> > > > >> > > >>>> >> >
> >> > > > >> > > >>>> >> > I did the following changes
> >> > > > >> > > >>>> >> >
> >> > > > >> > > >>>> >> >    1.
> >> > > > >> > > >>>> >> >
> >> > > > >> > > >>>> >> >    Simplified the API spec section to only
> include
> >> the
> >> > > > diff.
> >> > > > >> > > >>>> >> >    2.
> >> > > > >> > > >>>> >> >
> >> > > > >> > > >>>> >> >    Reordered the HWM requirement section.
> >> > > > >> > > >>>> >> >    3.
> >> > > > >> > > >>>> >> >
> >> > > > >> > > >>>> >> >    Removed the URM implementation details to keep
> >> the
> >> > > > >> necessary
> >> > > > >> > > >>>> >> >    characteristics to perform the unclean
> recovery.
> >> > > > >> > > >>>> >> >    1.
> >> > > > >> > > >>>> >> >
> >> > > > >> > > >>>> >> >       When to perform the unclean recovery
> >> > > > >> > > >>>> >> >       2.
> >> > > > >> > > >>>> >> >
> >> > > > >> > > >>>> >> >       Under different config, how the unclean
> >> recovery
> >> > > > finds
> >> > > > >> > the
> >> > > > >> > > >>>> leader.
> >> > > > >> > > >>>> >> >       3.
> >> > > > >> > > >>>> >> >
> >> > > > >> > > >>>> >> >       How the config
> unclean.leader.election.enable
> >> > and
> >> > > > >> > > >>>> >> >       unclean.recovery.strategy are converted
> when
> >> > users
> >> > > > >> > > >>>> enable/disable
> >> > > > >> > > >>>> >> the
> >> > > > >> > > >>>> >> >       unclean recovery.
> >> > > > >> > > >>>> >> >       4.
> >> > > > >> > > >>>> >> >
> >> > > > >> > > >>>> >> >    More details about how we change admin client.
> >> > > > >> > > >>>> >> >    5.
> >> > > > >> > > >>>> >> >
> >> > > > >> > > >>>> >> >    API limits on the GetReplicaLogInfoRequest and
> >> > > > >> > > >>>> DescribeTopicRequest.
> >> > > > >> > > >>>> >> >    6.
> >> > > > >> > > >>>> >> >
> >> > > > >> > > >>>> >> >    Two metrics added
> >> > > > >> > > >>>> >> >    1.
> >> > > > >> > > >>>> >> >
> >> > > > >> > > >>>> >> >
> >> > > >  Kafka.controller.global_under_min_isr_partition_count
> >> > > > >> > > >>>> >> >       2.
> >> > > > >> > > >>>> >> >
> >> > > > >> > > >>>> >> >
> >>  kafka.controller.unclean_recovery_finished_count
> >> > > > >> > > >>>> >> >
> >> > > > >> > > >>>> >> >
> >> > > > >> > > >>>> >> > On Wed, Sep 13, 2023 at 10:46 AM Colin McCabe <
> >> > > > >> > > cmccabe@apache.org>
> >> > > > >> > > >>>> >> wrote:
> >> > > > >> > > >>>> >> >
> >> > > > >> > > >>>> >> >> On Tue, Sep 12, 2023, at 17:21, Calvin Liu
> wrote:
> >> > > > >> > > >>>> >> >> > Hi Colin
> >> > > > >> > > >>>> >> >> > Thanks for the comments!
> >> > > > >> > > >>>> >> >> >
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >> >> Hi Calvin,
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >> >> Thanks again for the KIP.
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >> >> One meta-comment: it's usually better to just
> do a
> >> > diff
> >> > > > on a
> >> > > > >> > > >>>> message
> >> > > > >> > > >>>> >> spec
> >> > > > >> > > >>>> >> >> file or java file if you're including changes to
> >> it
> >> > in
> >> > > > the
> >> > > > >> > KIP.
> >> > > > >> > > >>>> This is
> >> > > > >> > > >>>> >> >> easier to read than looking for "new fields
> begin"
> >> > etc.
> >> > > > in
> >> > > > >> the
> >> > > > >> > > >>>> text, and
> >> > > > >> > > >>>> >> >> gracefully handles the case where existing
> fields
> >> > were
> >> > > > >> > changed.
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >> >> > Rewrite the Additional High Watermark
> >> advancement
> >> > > > >> > requirement
> >> > > > >> > > >>>> >> >> > There was feedback on this section that some
> >> > readers
> >> > > > may
> >> > > > >> not
> >> > > > >> > > be
> >> > > > >> > > >>>> >> familiar
> >> > > > >> > > >>>> >> >> > with HWM and Ack=0,1,all requests. This can
> help
> >> > them
> >> > > > >> > > understand
> >> > > > >> > > >>>> the
> >> > > > >> > > >>>> >> >> > proposal. I will rewrite this part for more
> >> > > > readability.
> >> > > > >> > > >>>> >> >> >
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >> >> To be clear, I wasn't suggesting dropping either
> >> > > > section. I
> >> > > > >> > > agree
> >> > > > >> > > >>>> that
> >> > > > >> > > >>>> >> >> they add useful background. I was just
> suggesting
> >> > that
> >> > > we
> >> > > > >> > should
> >> > > > >> > > >>>> discuss
> >> > > > >> > > >>>> >> >> the "acks" setting AFTER discussing the new high
> >> > > > watermark
> >> > > > >> > > >>>> advancement
> >> > > > >> > > >>>> >> >> conditions. We also should discuss acks=0. While
> >> it
> >> > > isn't
> >> > > > >> > > >>>> conceptually
> >> > > > >> > > >>>> >> much
> >> > > > >> > > >>>> >> >> different than acks=1 here, its omission from
> this
> >> > > > section
> >> > > > >> is
> >> > > > >> > > >>>> confusing.
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >> >> > Unclean recovery
> >> > > > >> > > >>>> >> >> >
> >> > > > >> > > >>>> >> >> > The plan is to replace the
> >> > > > unclean.leader.election.enable
> >> > > > >> > with
> >> > > > >> > > >>>> >> >> > unclean.recovery.strategy. If the Unclean
> >> Recovery
> >> > is
> >> > > > >> > enabled
> >> > > > >> > > >>>> then it
> >> > > > >> > > >>>> >> >> deals
> >> > > > >> > > >>>> >> >> > with the three options in the
> >> > > > unclean.recovery.strategy.
> >> > > > >> > > >>>> >> >> >
> >> > > > >> > > >>>> >> >> >
> >> > > > >> > > >>>> >> >> > Let’s refine the Unclean Recovery. We have
> >> already
> >> > > > taken a
> >> > > > >> > > lot of
> >> > > > >> > > >>>> >> >> > suggestions and I hope to enhance the
> >> durability of
> >> > > > Kafka
> >> > > > >> to
> >> > > > >> > > the
> >> > > > >> > > >>>> next
> >> > > > >> > > >>>> >> >> level
> >> > > > >> > > >>>> >> >> > with this KIP.
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >> >> I am OK with doing the unclean leader recovery
> >> > > > improvements
> >> > > > >> in
> >> > > > >> > > >>>> this KIP.
> >> > > > >> > > >>>> >> >> However, I think we need to really work on the
> >> > > > configuration
> >> > > > >> > > >>>> settings.
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >> >> Configuration overrides are often quite messy.
> For
> >> > > > example,
> >> > > > >> > the
> >> > > > >> > > >>>> cases
> >> > > > >> > > >>>> >> >> where we have log.roll.hours and
> >> log.roll.segment.ms
> >> > ,
> >> > > > the
> >> > > > >> > user
> >> > > > >> > > >>>> has to
> >> > > > >> > > >>>> >> >> remember which one takes precedence, and it is
> not
> >> > > > obvious.
> >> > > > >> > So,
> >> > > > >> > > >>>> rather
> >> > > > >> > > >>>> >> than
> >> > > > >> > > >>>> >> >> creating a new configuration, why not add
> >> additional
> >> > > > values
> >> > > > >> to
> >> > > > >> > > >>>> >> >> "unclean.leader.election.enable"? I think this
> >> will
> >> > be
> >> > > > >> simpler
> >> > > > >> > > for
> >> > > > >> > > >>>> >> people
> >> > > > >> > > >>>> >> >> to understand, and simpler in the code as well.
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >> >> What if we continued to use
> >> > > > "unclean.leader.election.enable"
> >> > > > >> > but
> >> > > > >> > > >>>> >> extended
> >> > > > >> > > >>>> >> >> it so that it took a string? Then the string
> could
> >> > have
> >> > > > >> these
> >> > > > >> > > >>>> values:
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >> >> never
> >> > > > >> > > >>>> >> >>     never automatically do an unclean leader
> >> election
> >> > > > under
> >> > > > >> > any
> >> > > > >> > > >>>> >> conditions
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >> >> false / default
> >> > > > >> > > >>>> >> >>     only do an unclean leader election if there
> >> may
> >> > be
> >> > > > >> > possible
> >> > > > >> > > >>>> data
> >> > > > >> > > >>>> >> loss
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >> >> true / always
> >> > > > >> > > >>>> >> >>     always do an unclean leader election if we
> >> can't
> >> > > > >> > immediately
> >> > > > >> > > >>>> elect a
> >> > > > >> > > >>>> >> >> leader
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >> >> It's a bit awkward that false maps to default
> >> rather
> >> > > > than to
> >> > > > >> > > >>>> never. But
> >> > > > >> > > >>>> >> >> this awkwardness exists if we use two different
> >> > > > >> configuration
> >> > > > >> > > keys
> >> > > > >> > > >>>> as
> >> > > > >> > > >>>> >> well.
> >> > > > >> > > >>>> >> >> The reason for the awkwardness is that we simply
> >> > don't
> >> > > > want
> >> > > > >> > most
> >> > > > >> > > >>>> of the
> >> > > > >> > > >>>> >> >> people currently setting
> >> > > > >> unclean.leader.election.enable=false
> >> > > > >> > to
> >> > > > >> > > >>>> get the
> >> > > > >> > > >>>> >> >> "never" behavior. We have to bite that bullet.
> >> Better
> >> > > to
> >> > > > be
> >> > > > >> > > clear
> >> > > > >> > > >>>> and
> >> > > > >> > > >>>> >> >> explicit than hide it.
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >> >> Another thing that's a bit awkward is having two
> >> > > > different
> >> > > > >> > ways
> >> > > > >> > > to
> >> > > > >> > > >>>> do
> >> > > > >> > > >>>> >> >> unclean leader election specified in the KIP.
> You
> >> > > > descirbe
> >> > > > >> two
> >> > > > >> > > >>>> methods:
> >> > > > >> > > >>>> >> the
> >> > > > >> > > >>>> >> >> simple "choose the last leader" method, and the
> >> > > "unclean
> >> > > > >> > > recovery
> >> > > > >> > > >>>> >> manager"
> >> > > > >> > > >>>> >> >> method. I understand why you did it this way --
> >> > "choose
> >> > > > the
> >> > > > >> > last
> >> > > > >> > > >>>> >> leader" is
> >> > > > >> > > >>>> >> >> simple, and will help us deliver an
> implementation
> >> > > > quickly,
> >> > > > >> > > while
> >> > > > >> > > >>>> the
> >> > > > >> > > >>>> >> URM
> >> > > > >> > > >>>> >> >> is preferable in the long term. My suggestion
> >> here is
> >> > > to
> >> > > > >> > > separate
> >> > > > >> > > >>>> the
> >> > > > >> > > >>>> >> >> decision of HOW to do unclean leader election
> from
> >> > the
> >> > > > >> > decision
> >> > > > >> > > of
> >> > > > >> > > >>>> WHEN
> >> > > > >> > > >>>> >> to
> >> > > > >> > > >>>> >> >> do it.
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >> >> So in other words, have
> >> > > "unclean.leader.election.enable"
> >> > > > >> > specify
> >> > > > >> > > >>>> when we
> >> > > > >> > > >>>> >> >> do unclean leader election, and have a new
> >> > > configuration
> >> > > > >> like
> >> > > > >> > > >>>> >> >> "unclean.recovery.manager.enable" to determine
> if
> >> we
> >> > > use
> >> > > > the
> >> > > > >> > > URM.
> >> > > > >> > > >>>> >> >> Presumably the URM will take some time to get
> >> fully
> >> > > > stable,
> >> > > > >> so
> >> > > > >> > > >>>> this can
> >> > > > >> > > >>>> >> >> default to false for a while, and we can flip
> the
> >> > > > default to
> >> > > > >> > > true
> >> > > > >> > > >>>> when
> >> > > > >> > > >>>> >> we
> >> > > > >> > > >>>> >> >> feel ready.
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >> >> The URM is somewhat under-described here. I
> think
> >> we
> >> > > > need a
> >> > > > >> > few
> >> > > > >> > > >>>> >> >> configurations here for it. For example, we
> need a
> >> > > > >> > > configuration to
> >> > > > >> > > >>>> >> specify
> >> > > > >> > > >>>> >> >> how long it should wait for a broker to respond
> to
> >> > its
> >> > > > RPCs
> >> > > > >> > > before
> >> > > > >> > > >>>> >> moving
> >> > > > >> > > >>>> >> >> on. We also need to understand how the URM
> >> interacts
> >> > > with
> >> > > > >> > > >>>> >> >> unclean.leader.election.enable=always. I assume
> >> that
> >> > > with
> >> > > > >> > > "always"
> >> > > > >> > > >>>> we
> >> > > > >> > > >>>> >> will
> >> > > > >> > > >>>> >> >> just unconditionally use the URM rather than
> >> choosing
> >> > > > >> > randomly.
> >> > > > >> > > >>>> But this
> >> > > > >> > > >>>> >> >> should be spelled out in the KIP.
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >> >> >
> >> > > > >> > > >>>> >> >> > DescribeTopicRequest
> >> > > > >> > > >>>> >> >> >
> >> > > > >> > > >>>> >> >> >    1.
> >> > > > >> > > >>>> >> >> >    Yes, the plan is to replace the
> >> MetadataRequest
> >> > > with
> >> > > > >> the
> >> > > > >> > > >>>> >> >> >    DescribeTopicRequest for the admin clients.
> >> Will
> >> > > > check
> >> > > > >> > the
> >> > > > >> > > >>>> details.
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >> >> Sounds good. But as I said, you need to specify
> >> how
> >> > > > >> > AdminClient
> >> > > > >> > > >>>> >> interacts
> >> > > > >> > > >>>> >> >> with the new request. This will involve adding
> >> some
> >> > > > fields
> >> > > > >> to
> >> > > > >> > > >>>> >> >> TopicDescription.java. And you need to specify
> the
> >> > > > changes
> >> > > > >> to
> >> > > > >> > > the
> >> > > > >> > > >>>> >> >> kafka-topics.sh command line tool. Otherwise we
> >> > cannot
> >> > > > use
> >> > > > >> the
> >> > > > >> > > >>>> tool to
> >> > > > >> > > >>>> >> see
> >> > > > >> > > >>>> >> >> the new information.
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >> >> The new requests, DescribeTopicRequest and
> >> > > > >> > > >>>> GetReplicaLogInfoRequest,
> >> > > > >> > > >>>> >> need
> >> > > > >> > > >>>> >> >> to have limits placed on them so that their size
> >> > can't
> >> > > be
> >> > > > >> > > >>>> infinite. We
> >> > > > >> > > >>>> >> >> don't want to propagate the current problems of
> >> > > > >> > MetadataRequest,
> >> > > > >> > > >>>> where
> >> > > > >> > > >>>> >> >> clients can request massive responses that can
> >> mess
> >> > up
> >> > > > the
> >> > > > >> JVM
> >> > > > >> > > when
> >> > > > >> > > >>>> >> handled.
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >> >> Adding limits is simple for
> >> GetReplicaLogInfoRequest
> >> > --
> >> > > > we
> >> > > > >> can
> >> > > > >> > > >>>> just say
> >> > > > >> > > >>>> >> >> that only 2000 partitions at a time can be
> >> requested.
> >> > > For
> >> > > > >> > > >>>> >> >> DescribeTopicRequest we can probably just limit
> >> to 20
> >> > > > topics
> >> > > > >> > or
> >> > > > >> > > >>>> >> something
> >> > > > >> > > >>>> >> >> like that, to avoid the complexity of doing
> >> > pagination
> >> > > in
> >> > > > >> this
> >> > > > >> > > KIP.
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >> >> >    2.
> >> > > > >> > > >>>> >> >> >    I can let the broker load the ELR info so
> >> that
> >> > > they
> >> > > > can
> >> > > > >> > > serve
> >> > > > >> > > >>>> the
> >> > > > >> > > >>>> >> >> >    DescribeTopicRequest as well.
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >> >> Yes, it's fine to add to MetadataCache. In fact,
> >> > you'll
> >> > > > be
> >> > > > >> > > loading
> >> > > > >> > > >>>> it
> >> > > > >> > > >>>> >> >> anyway once it's added to PartitionImage.
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >> >> >    3.
> >> > > > >> > > >>>> >> >> >    Yeah, it does not make sense to have the
> >> topic
> >> > id
> >> > > if
> >> > > > >> > > >>>> >> >> >    DescribeTopicRequest is only used by the
> >> admin
> >> > > > client.
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >> >> OK. That makes things simpler. We can always
> >> create a
> >> > > new
> >> > > > >> API
> >> > > > >> > > later
> >> > > > >> > > >>>> >> >> (hopefully not in this KIP!) to query by topic
> ID.
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >> >> >
> >> > > > >> > > >>>> >> >> >
> >> > > > >> > > >>>> >> >> > Metrics
> >> > > > >> > > >>>> >> >> >
> >> > > > >> > > >>>> >> >> > As for overall cluster health metrics, I think
> >> > > > >> under-min-ISR
> >> > > > >> > > is
> >> > > > >> > > >>>> still
> >> > > > >> > > >>>> >> a
> >> > > > >> > > >>>> >> >> > useful one. ELR is more like a safety belt.
> When
> >> > the
> >> > > > ELR
> >> > > > >> is
> >> > > > >> > > >>>> used, the
> >> > > > >> > > >>>> >> >> > cluster availability has already been
> impacted.
> >> > > > >> > > >>>> >> >> >
> >> > > > >> > > >>>> >> >> > Maybe we can have a metric to count the
> >> partitions
> >> > > that
> >> > > > >> > > sum(ISR,
> >> > > > >> > > >>>> ELR)
> >> > > > >> > > >>>> >> <
> >> > > > >> > > >>>> >> >> min
> >> > > > >> > > >>>> >> >> > ISR. What do you think?
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >> >> How about:
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >> >> A.  a metric for the totoal number of
> >> under-min-isr
> >> > > > >> > partitions?
> >> > > > >> > > We
> >> > > > >> > > >>>> don't
> >> > > > >> > > >>>> >> >> have that in Apache Kafka at the moment.
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >> >> B. a metric for the number of unclean leader
> >> > elections
> >> > > we
> >> > > > >> did
> >> > > > >> > > (for
> >> > > > >> > > >>>> >> >> simplicity, it can reset to 0 on controller
> >> restart:
> >> > we
> >> > > > >> expect
> >> > > > >> > > >>>> people to
> >> > > > >> > > >>>> >> >> monitor the change over time anyway)
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >> >> best,
> >> > > > >> > > >>>> >> >> Colin
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >> >> >
> >> > > > >> > > >>>> >> >> > Yeah, for the ongoing unclean recoveries, the
> >> > > > controller
> >> > > > >> can
> >> > > > >> > > >>>> keep an
> >> > > > >> > > >>>> >> >> > accurate count through failover because
> >> partition
> >> > > > >> > registration
> >> > > > >> > > >>>> can
> >> > > > >> > > >>>> >> >> indicate
> >> > > > >> > > >>>> >> >> > whether a recovery is needed. However, for the
> >> > > happened
> >> > > > >> > ones,
> >> > > > >> > > >>>> unless
> >> > > > >> > > >>>> >> we
> >> > > > >> > > >>>> >> >> > want to persist the number somewhere, we can
> >> only
> >> > > > figure
> >> > > > >> it
> >> > > > >> > > out
> >> > > > >> > > >>>> from
> >> > > > >> > > >>>> >> the
> >> > > > >> > > >>>> >> >> > log.
> >> > > > >> > > >>>> >> >> >
> >> > > > >> > > >>>> >> >> > On Tue, Sep 12, 2023 at 3:16 PM Colin McCabe <
> >> > > > >> > > cmccabe@apache.org
> >> > > > >> > > >>>> >
> >> > > > >> > > >>>> >> wrote:
> >> > > > >> > > >>>> >> >> >
> >> > > > >> > > >>>> >> >> >> Also, we should have metrics that show what
> is
> >> > going
> >> > > > on
> >> > > > >> > with
> >> > > > >> > > >>>> regard
> >> > > > >> > > >>>> >> to
> >> > > > >> > > >>>> >> >> the
> >> > > > >> > > >>>> >> >> >> eligible replica set. I'm not sure exactly
> >> what to
> >> > > > >> suggest,
> >> > > > >> > > but
> >> > > > >> > > >>>> >> >> something
> >> > > > >> > > >>>> >> >> >> that could identify when things are going
> >> wrong in
> >> > > the
> >> > > > >> > > clsuter.
> >> > > > >> > > >>>> >> >> >>
> >> > > > >> > > >>>> >> >> >> For example, maybe a metric for partitions
> >> > > containing
> >> > > > >> > > replicas
> >> > > > >> > > >>>> that
> >> > > > >> > > >>>> >> are
> >> > > > >> > > >>>> >> >> >> ineligible to be leader? That would show a
> >> spike
> >> > > when
> >> > > > a
> >> > > > >> > > broker
> >> > > > >> > > >>>> had an
> >> > > > >> > > >>>> >> >> >> unclean restart.
> >> > > > >> > > >>>> >> >> >>
> >> > > > >> > > >>>> >> >> >> Ideally, we'd also have a metric that
> indicates
> >> > when
> >> > > > an
> >> > > > >> > > unclear
> >> > > > >> > > >>>> >> leader
> >> > > > >> > > >>>> >> >> >> election or a recovery happened. It's a bit
> >> tricky
> >> > > > >> because
> >> > > > >> > > the
> >> > > > >> > > >>>> simple
> >> > > > >> > > >>>> >> >> >> thing, of tracking it per controller, may be
> a
> >> bit
> >> > > > >> > confusing
> >> > > > >> > > >>>> during
> >> > > > >> > > >>>> >> >> >> failovers.
> >> > > > >> > > >>>> >> >> >>
> >> > > > >> > > >>>> >> >> >> best,
> >> > > > >> > > >>>> >> >> >> Colin
> >> > > > >> > > >>>> >> >> >>
> >> > > > >> > > >>>> >> >> >>
> >> > > > >> > > >>>> >> >> >> On Tue, Sep 12, 2023, at 14:25, Colin McCabe
> >> > wrote:
> >> > > > >> > > >>>> >> >> >> > Hi Calvin,
> >> > > > >> > > >>>> >> >> >> >
> >> > > > >> > > >>>> >> >> >> > Thanks for the KIP. I think this is a great
> >> > > > >> improvement.
> >> > > > >> > > >>>> >> >> >> >
> >> > > > >> > > >>>> >> >> >> >> Additional High Watermark advance
> >> requirement
> >> > > > >> > > >>>> >> >> >> >
> >> > > > >> > > >>>> >> >> >> > Typo: change "advance" to "advancement"
> >> > > > >> > > >>>> >> >> >> >
> >> > > > >> > > >>>> >> >> >> >> A bit recap of some key concepts.
> >> > > > >> > > >>>> >> >> >> >
> >> > > > >> > > >>>> >> >> >> > Typo: change "bit" to "quick"
> >> > > > >> > > >>>> >> >> >> >
> >> > > > >> > > >>>> >> >> >> >> Ack=1/all produce request. It defines when
> >> the
> >> > > > Kafka
> >> > > > >> > > server
> >> > > > >> > > >>>> should
> >> > > > >> > > >>>> >> >> >> respond to the produce request
> >> > > > >> > > >>>> >> >> >> >
> >> > > > >> > > >>>> >> >> >> > I think this section would be clearer if we
> >> > talked
> >> > > > >> about
> >> > > > >> > > the
> >> > > > >> > > >>>> new
> >> > > > >> > > >>>> >> high
> >> > > > >> > > >>>> >> >> >> > watermark advancement requirement first,
> and
> >> > THEN
> >> > > > >> talked
> >> > > > >> > > >>>> about its
> >> > > > >> > > >>>> >> >> >> > impact on acks=0, acks=1, and     acks=all.
> >> > > > acks=all
> >> > > > >> is
> >> > > > >> > of
> >> > > > >> > > >>>> course
> >> > > > >> > > >>>> >> the
> >> > > > >> > > >>>> >> >> >> > main case we care about here, so it would
> be
> >> > good
> >> > > to
> >> > > > >> lead
> >> > > > >> > > with
> >> > > > >> > > >>>> >> that,
> >> > > > >> > > >>>> >> >> >> > rather than delving into the technicalities
> >> of
> >> > > > acks=0/1
> >> > > > >> > > first.
> >> > > > >> > > >>>> >> >> >> >
> >> > > > >> > > >>>> >> >> >> >> Unclean recovery
> >> > > > >> > > >>>> >> >> >> >
> >> > > > >> > > >>>> >> >> >> > So, here you are introducing a new
> >> > configuration,
> >> > > > >> > > >>>> >> >> >> > unclean.recovery.strategy. The difficult
> >> thing
> >> > > here
> >> > > > is
> >> > > > >> > that
> >> > > > >> > > >>>> there
> >> > > > >> > > >>>> >> is a
> >> > > > >> > > >>>> >> >> >> > lot of overlap with
> >> > > unclean.leader.election.enable.
> >> > > > So
> >> > > > >> we
> >> > > > >> > > >>>> have 3
> >> > > > >> > > >>>> >> >> >> > different settings for
> >> > unclean.recovery.strategy,
> >> > > > plus
> >> > > > >> 2
> >> > > > >> > > >>>> different
> >> > > > >> > > >>>> >> >> >> > settings for
> unclean.leader.election.enable,
> >> > > giving
> >> > > > a
> >> > > > >> > cross
> >> > > > >> > > >>>> >> product of
> >> > > > >> > > >>>> >> >> >> > 6 different options. The following "unclean
> >> > > recovery
> >> > > > >> > > manager"
> >> > > > >> > > >>>> >> section
> >> > > > >> > > >>>> >> >> >> > only applies to one fo those 6 different
> >> > > > possibilities
> >> > > > >> (I
> >> > > > >> > > >>>> think?)
> >> > > > >> > > >>>> >> >> >> >
> >> > > > >> > > >>>> >> >> >> > I simply don't think we need so many
> >> different
> >> > > > election
> >> > > > >> > > types.
> >> > > > >> > > >>>> >> Really
> >> > > > >> > > >>>> >> >> >> > the use-cases we need are people who want
> NO
> >> > > unclean
> >> > > > >> > > >>>> elections,
> >> > > > >> > > >>>> >> people
> >> > > > >> > > >>>> >> >> >> > who want "the reasonable thing" and people
> >> who
> >> > > want
> >> > > > >> > > >>>> avaialbility at
> >> > > > >> > > >>>> >> >> all
> >> > > > >> > > >>>> >> >> >> > costs.
> >> > > > >> > > >>>> >> >> >> >
> >> > > > >> > > >>>> >> >> >> > Overall, I feel like the first half of the
> >> KIP
> >> > is
> >> > > > about
> >> > > > >> > the
> >> > > > >> > > >>>> ELR,
> >> > > > >> > > >>>> >> and
> >> > > > >> > > >>>> >> >> >> > the second half is about reworking unclean
> >> > leader
> >> > > > >> > > election. It
> >> > > > >> > > >>>> >> might
> >> > > > >> > > >>>> >> >> be
> >> > > > >> > > >>>> >> >> >> > better to move that second half to a
> separate
> >> > KIP
> >> > > so
> >> > > > >> that
> >> > > > >> > > we
> >> > > > >> > > >>>> can
> >> > > > >> > > >>>> >> >> figure
> >> > > > >> > > >>>> >> >> >> > it out fully. It should be fine to punt
> this
> >> > until
> >> > > > >> later
> >> > > > >> > > and
> >> > > > >> > > >>>> just
> >> > > > >> > > >>>> >> have
> >> > > > >> > > >>>> >> >> >> > the current behavior on empty ELR be
> waiting
> >> for
> >> > > the
> >> > > > >> last
> >> > > > >> > > >>>> known
> >> > > > >> > > >>>> >> leader
> >> > > > >> > > >>>> >> >> >> > to return. After all, that's what we do
> >> today.
> >> > > > >> > > >>>> >> >> >> >
> >> > > > >> > > >>>> >> >> >> >> DescribeTopicRequest
> >> > > > >> > > >>>> >> >> >> >
> >> > > > >> > > >>>> >> >> >> > Is the intention for AdminClient to use
> this
> >> RPC
> >> > > for
> >> > > > >> > > >>>> >> >> >> > Admin#describeTopics ? If so, we need to
> >> > describe
> >> > > > all
> >> > > > >> of
> >> > > > >> > > the
> >> > > > >> > > >>>> >> changes
> >> > > > >> > > >>>> >> >> to
> >> > > > >> > > >>>> >> >> >> > the admin client API, as well as changes to
> >> > > > >> command-line
> >> > > > >> > > >>>> tools like
> >> > > > >> > > >>>> >> >> >> > kafka-topics.sh (if there are any). For
> >> example,
> >> > > you
> >> > > > >> will
> >> > > > >> > > >>>> probably
> >> > > > >> > > >>>> >> >> need
> >> > > > >> > > >>>> >> >> >> > changes to TopicDescription.java. You will
> >> also
> >> > > > need to
> >> > > > >> > > >>>> provide
> >> > > > >> > > >>>> >> all of
> >> > > > >> > > >>>> >> >> >> > the things that admin client needs -- for
> >> > example,
> >> > > > >> > > >>>> >> >> >> > TopicAuthorizedOperations.
> >> > > > >> > > >>>> >> >> >> >
> >> > > > >> > > >>>> >> >> >> > I also don't think the controller should
> >> serve
> >> > > this
> >> > > > >> > > request.
> >> > > > >> > > >>>> We
> >> > > > >> > > >>>> >> want
> >> > > > >> > > >>>> >> >> to
> >> > > > >> > > >>>> >> >> >> > minimize load on the controller. Just like
> >> with
> >> > > the
> >> > > > >> other
> >> > > > >> > > >>>> metadata
> >> > > > >> > > >>>> >> >> >> > requests like MetadataRequest, this should
> be
> >> > > > served by
> >> > > > >> > > >>>> brokers.
> >> > > > >> > > >>>> >> >> >> >
> >> > > > >> > > >>>> >> >> >> > It's a bit confusing why both topic ID and
> >> topic
> >> > > > name
> >> > > > >> are
> >> > > > >> > > >>>> provided
> >> > > > >> > > >>>> >> to
> >> > > > >> > > >>>> >> >> >> > this API. Is the intention that callers
> >> should
> >> > set
> >> > > > one
> >> > > > >> > but
> >> > > > >> > > >>>> not the
> >> > > > >> > > >>>> >> >> >> > other? Or both? This needs to be clarified.
> >> > Also,
> >> > > > if we
> >> > > > >> > do
> >> > > > >> > > >>>> want to
> >> > > > >> > > >>>> >> >> >> > support lookups by UUID, that is another
> >> thing
> >> > > that
> >> > > > >> needs
> >> > > > >> > > to
> >> > > > >> > > >>>> be
> >> > > > >> > > >>>> >> added
> >> > > > >> > > >>>> >> >> >> > to adminclient.
> >> > > > >> > > >>>> >> >> >> >
> >> > > > >> > > >>>> >> >> >> > In general, I feel like this should also
> >> > probably
> >> > > be
> >> > > > >> its
> >> > > > >> > > own
> >> > > > >> > > >>>> KIP
> >> > > > >> > > >>>> >> since
> >> > > > >> > > >>>> >> >> >> > it's fairly complex
> >> > > > >> > > >>>> >> >> >> >
> >> > > > >> > > >>>> >> >> >> > best,
> >> > > > >> > > >>>> >> >> >> > Colin
> >> > > > >> > > >>>> >> >> >> >
> >> > > > >> > > >>>> >> >> >> >
> >> > > > >> > > >>>> >> >> >> > On Thu, Aug 10, 2023, at 15:46, Calvin Liu
> >> > 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
> >> > > > >> > > >>>> >> >> >>
> >> > > > >> > > >>>> >> >>
> >> > > > >> > > >>>> >>
> >> > > > >> > > >>>>
> >> > > > >> > > >>>
> >> > > > >> > >
> >> > > > >> >
> >> > > > >>
> >> > > >
> >> > >
> >> >
> >>
> >
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
Sorry -- not MV but software version.

On Wed, Oct 4, 2023 at 9:51 AM Justine Olshan <jo...@confluent.io> wrote:

> Catching up with this discussion.
>
> I was just curious -- have we had other instances where downgrading MV is
> not supported? I think Kafka typically tries to support downgrades, and I
> couldn't think of other examples.
>
> Thanks,
> Justine
>
> On Wed, Oct 4, 2023 at 9:40 AM Calvin Liu <ca...@confluent.io.invalid>
> wrote:
>
>> Hi Jun,
>> 54. Marked the software downgrading is not supported. As the old
>> controller
>> will not understand the new PartitionRecord and PartitionChangeRecord.
>> Thanks!
>>
>> On Wed, Oct 4, 2023 at 9:12 AM Jun Rao <ju...@confluent.io.invalid> wrote:
>>
>> > Hi, Calvin,
>> >
>> > Thanks for the reply. Just one more comment.
>> >
>> > 54. It seems that downgrading MV is supported. Is downgrading the
>> software
>> > version supported? It would be useful to document that.
>> >
>> > Thanks,
>> >
>> > Jun
>> >
>> > On Tue, Oct 3, 2023 at 4:55 PM Artem Livshits
>> > <al...@confluent.io.invalid> wrote:
>> >
>> > > Hi Colin,
>> > >
>> > > I think in your example "do_unclean_recovery" would need to do
>> different
>> > > things depending on the strategy.
>> > >
>> > > do_unclean_recovery() {
>> > >    if (unclean.recovery.manager.enabled) {
>> > >     if (strategy == Aggressive)
>> > >       use UncleanRecoveryManager(waitLastKnownERL=false)  // just
>> inspect
>> > > logs from whoever is available
>> > >     else
>> > >       use  UncleanRecoveryManager(waitLastKnownERL=true)  // must wait
>> > for
>> > > at least last known ELR
>> > >   } else {
>> > >     if (strategy == Aggressive)
>> > >       choose the last known leader if that is available, or a random
>> > leader
>> > > if not)
>> > >     else
>> > >       wait for last known leader to get back
>> > >   }
>> > > }
>> > >
>> > > The idea is that the Aggressive strategy would kick in as soon as we
>> lost
>> > > the leader and would pick a leader from whoever is available; but the
>> > > Balanced will only kick in when ELR is empty and will wait for the
>> > brokers
>> > > that likely have most data to be available.
>> > >
>> > > On Tue, Oct 3, 2023 at 3:04 PM Colin McCabe <cm...@apache.org>
>> wrote:
>> > >
>> > > > On Tue, Oct 3, 2023, at 10:49, Jun Rao wrote:
>> > > > > Hi, Calvin,
>> > > > >
>> > > > > Thanks for the update KIP. A few more comments.
>> > > > >
>> > > > > 41. Why would a user choose the option to select a random replica
>> as
>> > > the
>> > > > > leader instead of using unclean.recovery.strateg=Aggressive? It
>> seems
>> > > > that
>> > > > > the latter is strictly better? If that's not the case, could we
>> fold
>> > > this
>> > > > > option under unclean.recovery.strategy instead of introducing a
>> > > separate
>> > > > > config?
>> > > >
>> > > > Hi Jun,
>> > > >
>> > > > I thought the flow of control was:
>> > > >
>> > > > If there is no leader for the partition {
>> > > >   If (there are unfenced ELR members) {
>> > > >     choose_an_unfenced_ELR_member
>> > > >   } else if (there are fenced ELR members AND strategy=Aggressive) {
>> > > >     do_unclean_recovery
>> > > >   } else if (there are no ELR members AND strategy != None) {
>> > > >     do_unclean_recovery
>> > > >   } else {
>> > > >     do nothing about the missing leader
>> > > >   }
>> > > > }
>> > > >
>> > > > do_unclean_recovery() {
>> > > >    if (unclean.recovery.manager.enabled) {
>> > > >     use UncleanRecoveryManager
>> > > >   } else {
>> > > >     choose the last known leader if that is available, or a random
>> > leader
>> > > > if not)
>> > > >   }
>> > > > }
>> > > >
>> > > > However, I think this could be clarified, especially the behavior
>> when
>> > > > unclean.recovery.manager.enabled=false. Inuitively the goal for
>> > > > unclean.recovery.manager.enabled=false is to be "the same as now,
>> > mostly"
>> > > > but it's very underspecified in the KIP, I agree.
>> > > >
>> > > > >
>> > > > > 50. ElectLeadersRequest: "If more than 20 topics are included,
>> only
>> > the
>> > > > > first 20 will be served. Others will be returned with
>> > DesiredLeaders."
>> > > > Hmm,
>> > > > > not sure that I understand this. ElectLeadersResponse doesn't
>> have a
>> > > > > DesiredLeaders field.
>> > > > >
>> > > > > 51. GetReplicaLogInfo: "If more than 2000 partitions are included,
>> > only
>> > > > the
>> > > > > first 2000 will be served" Do we return an error for the remaining
>> > > > > partitions? Actually, should we include an errorCode field at the
>> > > > partition
>> > > > > level in GetReplicaLogInfoResponse to cover non-existing
>> partitions
>> > and
>> > > > no
>> > > > > authorization, etc?
>> > > > >
>> > > > > 52. The entry should matches => The entry should match
>> > > > >
>> > > > > 53. ElectLeadersRequest.DesiredLeaders: Should it be nullable
>> since a
>> > > > user
>> > > > > may not specify DesiredLeaders?
>> > > > >
>> > > > > 54. Downgrade: Is that indeed possible? I thought earlier you said
>> > that
>> > > > > once the new version of the records are in the metadata log, one
>> > can't
>> > > > > downgrade since the old broker doesn't know how to parse the new
>> > > version
>> > > > of
>> > > > > the metadata records?
>> > > > >
>> > > >
>> > > > MetadataVersion downgrade is currently broken but we have fixing it
>> on
>> > > our
>> > > > plate for Kafka 3.7.
>> > > >
>> > > > The way downgrade works is that "new features" are dropped, leaving
>> > only
>> > > > the old ones.
>> > > >
>> > > > > 55. CleanShutdownFile: Should we add a version field for future
>> > > > extension?
>> > > > >
>> > > > > 56. Config changes are public facing. Could we have a separate
>> > section
>> > > to
>> > > > > document all the config changes?
>> > > >
>> > > > +1. A separate section for this would be good.
>> > > >
>> > > > best,
>> > > > Colin
>> > > >
>> > > > >
>> > > > > Thanks,
>> > > > >
>> > > > > Jun
>> > > > >
>> > > > > On Mon, Sep 25, 2023 at 4:29 PM Calvin Liu
>> > <caliu@confluent.io.invalid
>> > > >
>> > > > > wrote:
>> > > > >
>> > > > >> Hi Jun
>> > > > >> Thanks for the comments.
>> > > > >>
>> > > > >> 40. If we change to None, it is not guaranteed for no data loss.
>> For
>> > > > users
>> > > > >> who are not able to validate the data with external resources,
>> > manual
>> > > > >> intervention does not give a better result but a loss of
>> > availability.
>> > > > So
>> > > > >> practically speaking, the Balance mode would be a better default
>> > > value.
>> > > > >>
>> > > > >> 41. No, it represents how we want to do the unclean leader
>> election.
>> > > If
>> > > > it
>> > > > >> is false, the unclean leader election will be the old random way.
>> > > > >> Otherwise, the unclean recovery will be used.
>> > > > >>
>> > > > >> 42. Good catch. Updated.
>> > > > >>
>> > > > >> 43. Only the first 20 topics will be served. Others will be
>> returned
>> > > > with
>> > > > >> InvalidRequestError
>> > > > >>
>> > > > >> 44. The order matters. The desired leader entries match with the
>> > topic
>> > > > >> partition list by the index.
>> > > > >>
>> > > > >> 45. Thanks! Updated.
>> > > > >>
>> > > > >> 46. Good advice! Updated.
>> > > > >>
>> > > > >> 47.1, updated the comment. Basically it will elect the replica in
>> > the
>> > > > >> desiredLeader field to be the leader
>> > > > >>
>> > > > >> 47.2 We can let the admin client do the conversion. Using the
>> > > > desiredLeader
>> > > > >> field in the json format seems easier for users.
>> > > > >>
>> > > > >> 48. Once the MV version is downgraded, all the ELR related fields
>> > will
>> > > > be
>> > > > >> removed on the next partition change. The controller will also
>> > ignore
>> > > > the
>> > > > >> ELR fields. Updated the KIP.
>> > > > >>
>> > > > >> 49. Yes, it would be deprecated/removed.
>> > > > >>
>> > > > >>
>> > > > >> On Mon, Sep 25, 2023 at 3:49 PM Jun Rao <jun@confluent.io.invalid
>> >
>> > > > wrote:
>> > > > >>
>> > > > >> > Hi, Calvin,
>> > > > >> >
>> > > > >> > Thanks for the updated KIP. Made another pass. A few more
>> comments
>> > > > below.
>> > > > >> >
>> > > > >> > 40. unclean.leader.election.enable.false ->
>> > > > >> > unclean.recovery.strategy.Balanced: The Balanced mode could
>> still
>> > > > lead to
>> > > > >> > data loss. So, I am wondering if
>> > > unclean.leader.election.enable.false
>> > > > >> > should map to None?
>> > > > >> >
>> > > > >> > 41. unclean.recovery.manager.enabled: I am not sure why we
>> > introduce
>> > > > this
>> > > > >> > additional config. Is it the same as
>> > unclean.recovery.strategy=None?
>> > > > >> >
>> > > > >> > 42. DescribeTopicResponse.TopicAuthorizedOperations: Should
>> this
>> > be
>> > > at
>> > > > >> the
>> > > > >> > topic level?
>> > > > >> >
>> > > > >> > 43. "Limit: 20 topics max per request": Could we describe what
>> > > > happens if
>> > > > >> > the request includes more than 20 topics?
>> > > > >> >
>> > > > >> > 44. ElectLeadersRequest.DesiredLeaders: Could we describe
>> whether
>> > > the
>> > > > >> > ordering matters?
>> > > > >> >
>> > > > >> > 45. GetReplicaLogInfo.TopicPartitions: "about": "The topic
>> > > partitions
>> > > > to
>> > > > >> > elect leaders.": The description in "about" is incorrect.
>> > > > >> >
>> > > > >> > 46. GetReplicaLogInfoResponse: Should we nest partitions under
>> > > > topicId to
>> > > > >> > be consistent with other types of responses?
>> > > > >> >
>> > > > >> > 47. kafka-leader-election.sh:
>> > > > >> > 47.1 Could we explain DESIGNATION?
>> > > > >> > 47.2 desiredLeader: Should it be a list to match the field in
>> > > > >> > ElectLeadersRequest?
>> > > > >> >
>> > > > >> > 48. We could add a section on downgrade?
>> > > > >> >
>> > > > >> > 49. LastKnownLeader: This seems only needed in the first phase
>> of
>> > > > >> > delivering ELR. Will it be removed when the complete KIP is
>> > > delivered?
>> > > > >> >
>> > > > >> > Thanks,
>> > > > >> >
>> > > > >> > Jun
>> > > > >> >
>> > > > >> > On Tue, Sep 19, 2023 at 1:30 PM Colin McCabe <
>> cmccabe@apache.org>
>> > > > wrote:
>> > > > >> >
>> > > > >> > > Hi Calvin,
>> > > > >> > >
>> > > > >> > > Thanks for the explanations. I like the idea of using none,
>> > > > balanced,
>> > > > >> > > aggressive. We also had an offline discussion about why it is
>> > good
>> > > > to
>> > > > >> > use a
>> > > > >> > > new config key (basically, so that we can deprecate the old
>> one
>> > > > which
>> > > > >> had
>> > > > >> > > only false/true values in 4.0) With these changes, I am +1.
>> > > > >> > >
>> > > > >> > > best,
>> > > > >> > > Colin
>> > > > >> > >
>> > > > >> > > On Mon, Sep 18, 2023, at 15:54, Calvin Liu wrote:
>> > > > >> > > > Hi Colin,
>> > > > >> > > > Also, can we deprecate unclean.leader.election.enable in
>> 4.0?
>> > > > Before
>> > > > >> > > that,
>> > > > >> > > > we can have both the config unclean.recovery.strategy and
>> > > > >> > > > unclean.leader.election.enable
>> > > > >> > > > and using the unclean.recovery.Enabled to determine which
>> > config
>> > > > to
>> > > > >> use
>> > > > >> > > > during the unclean leader election.
>> > > > >> > > >
>> > > > >> > > > On Mon, Sep 18, 2023 at 3:51 PM Calvin Liu <
>> > caliu@confluent.io>
>> > > > >> wrote:
>> > > > >> > > >
>> > > > >> > > >> Hi Colin,
>> > > > >> > > >> For the unclean.recovery.strategy config name, how about
>> we
>> > use
>> > > > the
>> > > > >> > > >> following
>> > > > >> > > >> None. It basically means no unclean recovery will be
>> > performed.
>> > > > >> > > >> Aggressive. It means availability goes first. Whenever the
>> > > > partition
>> > > > >> > > can't
>> > > > >> > > >> elect a durable replica, the controller will try the
>> unclean
>> > > > >> recovery.
>> > > > >> > > >> Balanced. It is the balance point of the availability
>> > > > >> > first(Aggressive)
>> > > > >> > > >> and least availability(None). The controller performs
>> unclean
>> > > > >> recovery
>> > > > >> > > when
>> > > > >> > > >> both ISR and ELR are empty.
>> > > > >> > > >>
>> > > > >> > > >>
>> > > > >> > > >> On Fri, Sep 15, 2023 at 11:42 AM Calvin Liu <
>> > > caliu@confluent.io>
>> > > > >> > wrote:
>> > > > >> > > >>
>> > > > >> > > >>> Hi Colin,
>> > > > >> > > >>>
>> > > > >> > > >>> > So, the proposal is that if someone sets
>> > > > >> > > "unclean.leader.election.enable
>> > > > >> > > >>> = true"...
>> > > > >> > > >>>
>> > > > >> > > >>>
>> > > > >> > > >>> The idea is to use one of the
>> unclean.leader.election.enable
>> > > and
>> > > > >> > > >>> unclean.recovery.strategy based on the
>> > > > unclean.recovery.Enabled. A
>> > > > >> > > possible
>> > > > >> > > >>> version can be
>> > > > >> > > >>>
>> > > > >> > > >>> If unclean.recovery.Enabled:
>> > > > >> > > >>>
>> > > > >> > > >>> {
>> > > > >> > > >>>
>> > > > >> > > >>> Check unclean.recovery.strategy. If set, use it.
>> Otherwise,
>> > > > check
>> > > > >> > > >>> unclean.leader.election.enable and translate it to
>> > > > >> > > >>> unclean.recovery.strategy.
>> > > > >> > > >>>
>> > > > >> > > >>> } else {
>> > > > >> > > >>>
>> > > > >> > > >>> Use unclean.leader.election.enable
>> > > > >> > > >>>
>> > > > >> > > >>> }
>> > > > >> > > >>>
>> > > > >> > > >>>
>> > > > >> > > >>> —--------
>> > > > >> > > >>>
>> > > > >> > > >>> >The configuration key should be
>> > > > >> "unclean.recovery.manager.enabled",
>> > > > >> > > >>> right?
>> > > > >> > > >>>
>> > > > >> > > >>>
>> > > > >> > > >>> I think we have two ways of choosing a leader uncleanly,
>> > > unclean
>> > > > >> > leader
>> > > > >> > > >>> election and unclean recovery(log inspection) and we try
>> to
>> > > > switch
>> > > > >> > > between
>> > > > >> > > >>> them.
>> > > > >> > > >>>
>> > > > >> > > >>> Do you mean we want to develop two ways of performing the
>> > > > unclean
>> > > > >> > > >>> recovery and one of them is using “unclean recovery
>> > manager”?
>> > > I
>> > > > >> guess
>> > > > >> > > we
>> > > > >> > > >>> haven’t discussed the second way.
>> > > > >> > > >>>
>> > > > >> > > >>>
>> > > > >> > > >>> —-------
>> > > > >> > > >>>
>> > > > >> > > >>> >How do these 4 levels of overrides interact with your
>> new
>> > > > >> > > >>> configurations?
>> > > > >> > > >>>
>> > > > >> > > >>>
>> > > > >> > > >>> I do notice in the Kraft controller code, the method to
>> > check
>> > > > >> whether
>> > > > >> > > >>> perform unclean leader election is hard coded to false
>> since
>> > > > >> > > >>> 2021(uncleanLeaderElectionEnabledForTopic). Isn’t it a
>> good
>> > > > chance
>> > > > >> to
>> > > > >> > > >>> completely deprecate the unclean.leader.election.enable?
>> We
>> > > > don’t
>> > > > >> > even
>> > > > >> > > have
>> > > > >> > > >>> to worry about the config conversion.
>> > > > >> > > >>>
>> > > > >> > > >>> On the other hand, whatever the override is, as long as
>> the
>> > > > >> > controller
>> > > > >> > > >>> can have the final effective
>> unclean.leader.election.enable,
>> > > the
>> > > > >> > topic
>> > > > >> > > >>> level config unclean.recovery.strategy, the cluster level
>> > > config
>> > > > >> > > >>> unclean.recovery.Enabled, the controller can calculate
>> the
>> > > > correct
>> > > > >> > > methods
>> > > > >> > > >>> to use right?
>> > > > >> > > >>>
>> > > > >> > > >>>
>> > > > >> > > >>> On Fri, Sep 15, 2023 at 10:02 AM Colin McCabe <
>> > > > cmccabe@apache.org>
>> > > > >> > > wrote:
>> > > > >> > > >>>
>> > > > >> > > >>>> On Thu, Sep 14, 2023, at 22:23, Calvin Liu wrote:
>> > > > >> > > >>>> > Hi Colin
>> > > > >> > > >>>> > 1. I think using the new config name is more clear.
>> > > > >> > > >>>> >        a. The unclean leader election is actually
>> removed
>> > > if
>> > > > >> > unclean
>> > > > >> > > >>>> > recovery is in use.
>> > > > >> > > >>>> >        b. Using multiple values in
>> > > > >> unclean.leader.election.enable
>> > > > >> > is
>> > > > >> > > >>>> > confusing and it will be more confusing after people
>> > forget
>> > > > >> about
>> > > > >> > > this
>> > > > >> > > >>>> > discussion.
>> > > > >> > > >>>>
>> > > > >> > > >>>> Hi Calvin,
>> > > > >> > > >>>>
>> > > > >> > > >>>> So, the proposal is that if someone sets
>> > > > >> > > "unclean.leader.election.enable
>> > > > >> > > >>>> = true" but then sets one of your new configurations,
>> the
>> > > > value of
>> > > > >> > > >>>> unclean.leader.election.enable is ignored? That seems
>> less
>> > > > clear
>> > > > >> to
>> > > > >> > > me, not
>> > > > >> > > >>>> more. Just in general, having multiple configuration
>> keys
>> > to
>> > > > >> control
>> > > > >> > > the
>> > > > >> > > >>>> same thing confuses users. Basically, they are sitting
>> at a
>> > > > giant
>> > > > >> > > control
>> > > > >> > > >>>> panel, and some of the levers do nothing.
>> > > > >> > > >>>>
>> > > > >> > > >>>> > 2. Sorry I forgot to mention in the response that I
>> did
>> > add
>> > > > the
>> > > > >> > > >>>> > unclean.recovery.Enabled flag.
>> > > > >> > > >>>>
>> > > > >> > > >>>> The configuration key should be
>> > > > >> "unclean.recovery.manager.enabled",
>> > > > >> > > >>>> right? Becuase we can do "unclean recovery" without the
>> > > > manager.
>> > > > >> > > Disabling
>> > > > >> > > >>>> the manager just means we use a different mechanism for
>> > > > recovery.
>> > > > >> > > >>>>
>> > > > >> > > >>>> >        c. Maybe I underestimated the challenge of
>> > replacing
>> > > > the
>> > > > >> > > >>>> config. Any
>> > > > >> > > >>>> > implementation problems ahead?
>> > > > >> > > >>>>
>> > > > >> > > >>>> There are four levels of overrides for
>> > > > >> > unclean.leader.election.enable.
>> > > > >> > > >>>>
>> > > > >> > > >>>> 1. static configuration for node.
>> > > > >> > > >>>>     This goes in the configuration file, typically named
>> > > > >> > > >>>> server.properties
>> > > > >> > > >>>>
>> > > > >> > > >>>> 2. dynamic configuration for node default
>> > > > >> > > >>>>   ConfigResource(type=BROKER, name="")
>> > > > >> > > >>>>
>> > > > >> > > >>>> 3. dynamic configuration for node
>> > > > >> > > >>>>   ConfigResource(type=BROKER, name=<controller id>)
>> > > > >> > > >>>>
>> > > > >> > > >>>> 4. dynamic configuration for topic
>> > > > >> > > >>>>   ConfigResource(type=TOPIC, name=<topic-name>)
>> > > > >> > > >>>>
>> > > > >> > > >>>> How do these 4 levels of overrides interact with your
>> new
>> > > > >> > > >>>> configurations? If the new configurations dominate over
>> the
>> > > old
>> > > > >> > ones,
>> > > > >> > > it
>> > > > >> > > >>>> seems like this will get a lot more confusing to
>> implement
>> > > (and
>> > > > >> also
>> > > > >> > > to
>> > > > >> > > >>>> use.)
>> > > > >> > > >>>>
>> > > > >> > > >>>> Again, I'd recommend just adding some new values to
>> > > > >> > > >>>> unclean.leader.election.enable. It's simple and will
>> > prevent
>> > > > user
>> > > > >> > > confusion
>> > > > >> > > >>>> (as well as developer confusion.)
>> > > > >> > > >>>>
>> > > > >> > > >>>> best,
>> > > > >> > > >>>> Colin
>> > > > >> > > >>>>
>> > > > >> > > >>>>
>> > > > >> > > >>>> > 3. About the admin client, I mentioned 3 changes in
>> the
>> > > > client.
>> > > > >> > > >>>> Anything
>> > > > >> > > >>>> > else I missed in the KIP?
>> > > > >> > > >>>> >       a. The client will switch to using the new RPC
>> > > instead
>> > > > of
>> > > > >> > > >>>> > MetadataRequest for the topics.
>> > > > >> > > >>>> >       b. The TopicPartitionInfo used in
>> TopicDescription
>> > > > needs
>> > > > >> to
>> > > > >> > > add
>> > > > >> > > >>>> new
>> > > > >> > > >>>> > fields related to the ELR.
>> > > > >> > > >>>> >       c. The outputs will add the ELR related fields.
>> > > > >> > > >>>> >
>> > > > >> > > >>>> > On Thu, Sep 14, 2023 at 9:19 PM Colin McCabe <
>> > > > >> cmccabe@apache.org>
>> > > > >> > > >>>> wrote:
>> > > > >> > > >>>> >
>> > > > >> > > >>>> >> Hi Calvin,
>> > > > >> > > >>>> >>
>> > > > >> > > >>>> >> Thanks for the changes.
>> > > > >> > > >>>> >>
>> > > > >> > > >>>> >> 1. Earlier I commented that creating
>> > > > >> "unclean.recovery.strategy "
>> > > > >> > > is
>> > > > >> > > >>>> not
>> > > > >> > > >>>> >> necessary, and we can just reuse the existing
>> > > > >> > > >>>> >> "unclean.leader.election.enable" configuration key.
>> > Let's
>> > > > >> discuss
>> > > > >> > > >>>> that.
>> > > > >> > > >>>> >>
>> > > > >> > > >>>> >> 2.I also don't understand why you didn't add a
>> > > > configuration to
>> > > > >> > > >>>> enable or
>> > > > >> > > >>>> >> disable the Unclean Recovery Manager. This seems
>> like a
>> > > very
>> > > > >> > simple
>> > > > >> > > >>>> way to
>> > > > >> > > >>>> >> handle the staging issue which we discussed. The URM
>> can
>> > > > just
>> > > > >> be
>> > > > >> > > >>>> turned off
>> > > > >> > > >>>> >> until it is production ready. Let's discuss this.
>> > > > >> > > >>>> >>
>> > > > >> > > >>>> >> 3. You still need to describe the changes to
>> AdminClient
>> > > > that
>> > > > >> are
>> > > > >> > > >>>> needed
>> > > > >> > > >>>> >> to use DescribeTopicRequest.
>> > > > >> > > >>>> >>
>> > > > >> > > >>>> >> Keep at it. It's looking better. :)
>> > > > >> > > >>>> >>
>> > > > >> > > >>>> >> best,
>> > > > >> > > >>>> >> Colin
>> > > > >> > > >>>> >>
>> > > > >> > > >>>> >>
>> > > > >> > > >>>> >> On Thu, Sep 14, 2023, at 11:03, Calvin Liu wrote:
>> > > > >> > > >>>> >> > Hi Colin
>> > > > >> > > >>>> >> > Thanks for the comments!
>> > > > >> > > >>>> >> >
>> > > > >> > > >>>> >> > I did the following changes
>> > > > >> > > >>>> >> >
>> > > > >> > > >>>> >> >    1.
>> > > > >> > > >>>> >> >
>> > > > >> > > >>>> >> >    Simplified the API spec section to only include
>> the
>> > > > diff.
>> > > > >> > > >>>> >> >    2.
>> > > > >> > > >>>> >> >
>> > > > >> > > >>>> >> >    Reordered the HWM requirement section.
>> > > > >> > > >>>> >> >    3.
>> > > > >> > > >>>> >> >
>> > > > >> > > >>>> >> >    Removed the URM implementation details to keep
>> the
>> > > > >> necessary
>> > > > >> > > >>>> >> >    characteristics to perform the unclean recovery.
>> > > > >> > > >>>> >> >    1.
>> > > > >> > > >>>> >> >
>> > > > >> > > >>>> >> >       When to perform the unclean recovery
>> > > > >> > > >>>> >> >       2.
>> > > > >> > > >>>> >> >
>> > > > >> > > >>>> >> >       Under different config, how the unclean
>> recovery
>> > > > finds
>> > > > >> > the
>> > > > >> > > >>>> leader.
>> > > > >> > > >>>> >> >       3.
>> > > > >> > > >>>> >> >
>> > > > >> > > >>>> >> >       How the config unclean.leader.election.enable
>> > and
>> > > > >> > > >>>> >> >       unclean.recovery.strategy are converted when
>> > users
>> > > > >> > > >>>> enable/disable
>> > > > >> > > >>>> >> the
>> > > > >> > > >>>> >> >       unclean recovery.
>> > > > >> > > >>>> >> >       4.
>> > > > >> > > >>>> >> >
>> > > > >> > > >>>> >> >    More details about how we change admin client.
>> > > > >> > > >>>> >> >    5.
>> > > > >> > > >>>> >> >
>> > > > >> > > >>>> >> >    API limits on the GetReplicaLogInfoRequest and
>> > > > >> > > >>>> DescribeTopicRequest.
>> > > > >> > > >>>> >> >    6.
>> > > > >> > > >>>> >> >
>> > > > >> > > >>>> >> >    Two metrics added
>> > > > >> > > >>>> >> >    1.
>> > > > >> > > >>>> >> >
>> > > > >> > > >>>> >> >
>> > > >  Kafka.controller.global_under_min_isr_partition_count
>> > > > >> > > >>>> >> >       2.
>> > > > >> > > >>>> >> >
>> > > > >> > > >>>> >> >
>>  kafka.controller.unclean_recovery_finished_count
>> > > > >> > > >>>> >> >
>> > > > >> > > >>>> >> >
>> > > > >> > > >>>> >> > On Wed, Sep 13, 2023 at 10:46 AM Colin McCabe <
>> > > > >> > > cmccabe@apache.org>
>> > > > >> > > >>>> >> wrote:
>> > > > >> > > >>>> >> >
>> > > > >> > > >>>> >> >> On Tue, Sep 12, 2023, at 17:21, Calvin Liu wrote:
>> > > > >> > > >>>> >> >> > Hi Colin
>> > > > >> > > >>>> >> >> > Thanks for the comments!
>> > > > >> > > >>>> >> >> >
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >> >> Hi Calvin,
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >> >> Thanks again for the KIP.
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >> >> One meta-comment: it's usually better to just do a
>> > diff
>> > > > on a
>> > > > >> > > >>>> message
>> > > > >> > > >>>> >> spec
>> > > > >> > > >>>> >> >> file or java file if you're including changes to
>> it
>> > in
>> > > > the
>> > > > >> > KIP.
>> > > > >> > > >>>> This is
>> > > > >> > > >>>> >> >> easier to read than looking for "new fields begin"
>> > etc.
>> > > > in
>> > > > >> the
>> > > > >> > > >>>> text, and
>> > > > >> > > >>>> >> >> gracefully handles the case where existing fields
>> > were
>> > > > >> > changed.
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >> >> > Rewrite the Additional High Watermark
>> advancement
>> > > > >> > requirement
>> > > > >> > > >>>> >> >> > There was feedback on this section that some
>> > readers
>> > > > may
>> > > > >> not
>> > > > >> > > be
>> > > > >> > > >>>> >> familiar
>> > > > >> > > >>>> >> >> > with HWM and Ack=0,1,all requests. This can help
>> > them
>> > > > >> > > understand
>> > > > >> > > >>>> the
>> > > > >> > > >>>> >> >> > proposal. I will rewrite this part for more
>> > > > readability.
>> > > > >> > > >>>> >> >> >
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >> >> To be clear, I wasn't suggesting dropping either
>> > > > section. I
>> > > > >> > > agree
>> > > > >> > > >>>> that
>> > > > >> > > >>>> >> >> they add useful background. I was just suggesting
>> > that
>> > > we
>> > > > >> > should
>> > > > >> > > >>>> discuss
>> > > > >> > > >>>> >> >> the "acks" setting AFTER discussing the new high
>> > > > watermark
>> > > > >> > > >>>> advancement
>> > > > >> > > >>>> >> >> conditions. We also should discuss acks=0. While
>> it
>> > > isn't
>> > > > >> > > >>>> conceptually
>> > > > >> > > >>>> >> much
>> > > > >> > > >>>> >> >> different than acks=1 here, its omission from this
>> > > > section
>> > > > >> is
>> > > > >> > > >>>> confusing.
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >> >> > Unclean recovery
>> > > > >> > > >>>> >> >> >
>> > > > >> > > >>>> >> >> > The plan is to replace the
>> > > > unclean.leader.election.enable
>> > > > >> > with
>> > > > >> > > >>>> >> >> > unclean.recovery.strategy. If the Unclean
>> Recovery
>> > is
>> > > > >> > enabled
>> > > > >> > > >>>> then it
>> > > > >> > > >>>> >> >> deals
>> > > > >> > > >>>> >> >> > with the three options in the
>> > > > unclean.recovery.strategy.
>> > > > >> > > >>>> >> >> >
>> > > > >> > > >>>> >> >> >
>> > > > >> > > >>>> >> >> > Let’s refine the Unclean Recovery. We have
>> already
>> > > > taken a
>> > > > >> > > lot of
>> > > > >> > > >>>> >> >> > suggestions and I hope to enhance the
>> durability of
>> > > > Kafka
>> > > > >> to
>> > > > >> > > the
>> > > > >> > > >>>> next
>> > > > >> > > >>>> >> >> level
>> > > > >> > > >>>> >> >> > with this KIP.
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >> >> I am OK with doing the unclean leader recovery
>> > > > improvements
>> > > > >> in
>> > > > >> > > >>>> this KIP.
>> > > > >> > > >>>> >> >> However, I think we need to really work on the
>> > > > configuration
>> > > > >> > > >>>> settings.
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >> >> Configuration overrides are often quite messy. For
>> > > > example,
>> > > > >> > the
>> > > > >> > > >>>> cases
>> > > > >> > > >>>> >> >> where we have log.roll.hours and
>> log.roll.segment.ms
>> > ,
>> > > > the
>> > > > >> > user
>> > > > >> > > >>>> has to
>> > > > >> > > >>>> >> >> remember which one takes precedence, and it is not
>> > > > obvious.
>> > > > >> > So,
>> > > > >> > > >>>> rather
>> > > > >> > > >>>> >> than
>> > > > >> > > >>>> >> >> creating a new configuration, why not add
>> additional
>> > > > values
>> > > > >> to
>> > > > >> > > >>>> >> >> "unclean.leader.election.enable"? I think this
>> will
>> > be
>> > > > >> simpler
>> > > > >> > > for
>> > > > >> > > >>>> >> people
>> > > > >> > > >>>> >> >> to understand, and simpler in the code as well.
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >> >> What if we continued to use
>> > > > "unclean.leader.election.enable"
>> > > > >> > but
>> > > > >> > > >>>> >> extended
>> > > > >> > > >>>> >> >> it so that it took a string? Then the string could
>> > have
>> > > > >> these
>> > > > >> > > >>>> values:
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >> >> never
>> > > > >> > > >>>> >> >>     never automatically do an unclean leader
>> election
>> > > > under
>> > > > >> > any
>> > > > >> > > >>>> >> conditions
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >> >> false / default
>> > > > >> > > >>>> >> >>     only do an unclean leader election if there
>> may
>> > be
>> > > > >> > possible
>> > > > >> > > >>>> data
>> > > > >> > > >>>> >> loss
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >> >> true / always
>> > > > >> > > >>>> >> >>     always do an unclean leader election if we
>> can't
>> > > > >> > immediately
>> > > > >> > > >>>> elect a
>> > > > >> > > >>>> >> >> leader
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >> >> It's a bit awkward that false maps to default
>> rather
>> > > > than to
>> > > > >> > > >>>> never. But
>> > > > >> > > >>>> >> >> this awkwardness exists if we use two different
>> > > > >> configuration
>> > > > >> > > keys
>> > > > >> > > >>>> as
>> > > > >> > > >>>> >> well.
>> > > > >> > > >>>> >> >> The reason for the awkwardness is that we simply
>> > don't
>> > > > want
>> > > > >> > most
>> > > > >> > > >>>> of the
>> > > > >> > > >>>> >> >> people currently setting
>> > > > >> unclean.leader.election.enable=false
>> > > > >> > to
>> > > > >> > > >>>> get the
>> > > > >> > > >>>> >> >> "never" behavior. We have to bite that bullet.
>> Better
>> > > to
>> > > > be
>> > > > >> > > clear
>> > > > >> > > >>>> and
>> > > > >> > > >>>> >> >> explicit than hide it.
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >> >> Another thing that's a bit awkward is having two
>> > > > different
>> > > > >> > ways
>> > > > >> > > to
>> > > > >> > > >>>> do
>> > > > >> > > >>>> >> >> unclean leader election specified in the KIP. You
>> > > > descirbe
>> > > > >> two
>> > > > >> > > >>>> methods:
>> > > > >> > > >>>> >> the
>> > > > >> > > >>>> >> >> simple "choose the last leader" method, and the
>> > > "unclean
>> > > > >> > > recovery
>> > > > >> > > >>>> >> manager"
>> > > > >> > > >>>> >> >> method. I understand why you did it this way --
>> > "choose
>> > > > the
>> > > > >> > last
>> > > > >> > > >>>> >> leader" is
>> > > > >> > > >>>> >> >> simple, and will help us deliver an implementation
>> > > > quickly,
>> > > > >> > > while
>> > > > >> > > >>>> the
>> > > > >> > > >>>> >> URM
>> > > > >> > > >>>> >> >> is preferable in the long term. My suggestion
>> here is
>> > > to
>> > > > >> > > separate
>> > > > >> > > >>>> the
>> > > > >> > > >>>> >> >> decision of HOW to do unclean leader election from
>> > the
>> > > > >> > decision
>> > > > >> > > of
>> > > > >> > > >>>> WHEN
>> > > > >> > > >>>> >> to
>> > > > >> > > >>>> >> >> do it.
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >> >> So in other words, have
>> > > "unclean.leader.election.enable"
>> > > > >> > specify
>> > > > >> > > >>>> when we
>> > > > >> > > >>>> >> >> do unclean leader election, and have a new
>> > > configuration
>> > > > >> like
>> > > > >> > > >>>> >> >> "unclean.recovery.manager.enable" to determine if
>> we
>> > > use
>> > > > the
>> > > > >> > > URM.
>> > > > >> > > >>>> >> >> Presumably the URM will take some time to get
>> fully
>> > > > stable,
>> > > > >> so
>> > > > >> > > >>>> this can
>> > > > >> > > >>>> >> >> default to false for a while, and we can flip the
>> > > > default to
>> > > > >> > > true
>> > > > >> > > >>>> when
>> > > > >> > > >>>> >> we
>> > > > >> > > >>>> >> >> feel ready.
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >> >> The URM is somewhat under-described here. I think
>> we
>> > > > need a
>> > > > >> > few
>> > > > >> > > >>>> >> >> configurations here for it. For example, we need a
>> > > > >> > > configuration to
>> > > > >> > > >>>> >> specify
>> > > > >> > > >>>> >> >> how long it should wait for a broker to respond to
>> > its
>> > > > RPCs
>> > > > >> > > before
>> > > > >> > > >>>> >> moving
>> > > > >> > > >>>> >> >> on. We also need to understand how the URM
>> interacts
>> > > with
>> > > > >> > > >>>> >> >> unclean.leader.election.enable=always. I assume
>> that
>> > > with
>> > > > >> > > "always"
>> > > > >> > > >>>> we
>> > > > >> > > >>>> >> will
>> > > > >> > > >>>> >> >> just unconditionally use the URM rather than
>> choosing
>> > > > >> > randomly.
>> > > > >> > > >>>> But this
>> > > > >> > > >>>> >> >> should be spelled out in the KIP.
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >> >> >
>> > > > >> > > >>>> >> >> > DescribeTopicRequest
>> > > > >> > > >>>> >> >> >
>> > > > >> > > >>>> >> >> >    1.
>> > > > >> > > >>>> >> >> >    Yes, the plan is to replace the
>> MetadataRequest
>> > > with
>> > > > >> the
>> > > > >> > > >>>> >> >> >    DescribeTopicRequest for the admin clients.
>> Will
>> > > > check
>> > > > >> > the
>> > > > >> > > >>>> details.
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >> >> Sounds good. But as I said, you need to specify
>> how
>> > > > >> > AdminClient
>> > > > >> > > >>>> >> interacts
>> > > > >> > > >>>> >> >> with the new request. This will involve adding
>> some
>> > > > fields
>> > > > >> to
>> > > > >> > > >>>> >> >> TopicDescription.java. And you need to specify the
>> > > > changes
>> > > > >> to
>> > > > >> > > the
>> > > > >> > > >>>> >> >> kafka-topics.sh command line tool. Otherwise we
>> > cannot
>> > > > use
>> > > > >> the
>> > > > >> > > >>>> tool to
>> > > > >> > > >>>> >> see
>> > > > >> > > >>>> >> >> the new information.
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >> >> The new requests, DescribeTopicRequest and
>> > > > >> > > >>>> GetReplicaLogInfoRequest,
>> > > > >> > > >>>> >> need
>> > > > >> > > >>>> >> >> to have limits placed on them so that their size
>> > can't
>> > > be
>> > > > >> > > >>>> infinite. We
>> > > > >> > > >>>> >> >> don't want to propagate the current problems of
>> > > > >> > MetadataRequest,
>> > > > >> > > >>>> where
>> > > > >> > > >>>> >> >> clients can request massive responses that can
>> mess
>> > up
>> > > > the
>> > > > >> JVM
>> > > > >> > > when
>> > > > >> > > >>>> >> handled.
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >> >> Adding limits is simple for
>> GetReplicaLogInfoRequest
>> > --
>> > > > we
>> > > > >> can
>> > > > >> > > >>>> just say
>> > > > >> > > >>>> >> >> that only 2000 partitions at a time can be
>> requested.
>> > > For
>> > > > >> > > >>>> >> >> DescribeTopicRequest we can probably just limit
>> to 20
>> > > > topics
>> > > > >> > or
>> > > > >> > > >>>> >> something
>> > > > >> > > >>>> >> >> like that, to avoid the complexity of doing
>> > pagination
>> > > in
>> > > > >> this
>> > > > >> > > KIP.
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >> >> >    2.
>> > > > >> > > >>>> >> >> >    I can let the broker load the ELR info so
>> that
>> > > they
>> > > > can
>> > > > >> > > serve
>> > > > >> > > >>>> the
>> > > > >> > > >>>> >> >> >    DescribeTopicRequest as well.
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >> >> Yes, it's fine to add to MetadataCache. In fact,
>> > you'll
>> > > > be
>> > > > >> > > loading
>> > > > >> > > >>>> it
>> > > > >> > > >>>> >> >> anyway once it's added to PartitionImage.
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >> >> >    3.
>> > > > >> > > >>>> >> >> >    Yeah, it does not make sense to have the
>> topic
>> > id
>> > > if
>> > > > >> > > >>>> >> >> >    DescribeTopicRequest is only used by the
>> admin
>> > > > client.
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >> >> OK. That makes things simpler. We can always
>> create a
>> > > new
>> > > > >> API
>> > > > >> > > later
>> > > > >> > > >>>> >> >> (hopefully not in this KIP!) to query by topic ID.
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >> >> >
>> > > > >> > > >>>> >> >> >
>> > > > >> > > >>>> >> >> > Metrics
>> > > > >> > > >>>> >> >> >
>> > > > >> > > >>>> >> >> > As for overall cluster health metrics, I think
>> > > > >> under-min-ISR
>> > > > >> > > is
>> > > > >> > > >>>> still
>> > > > >> > > >>>> >> a
>> > > > >> > > >>>> >> >> > useful one. ELR is more like a safety belt. When
>> > the
>> > > > ELR
>> > > > >> is
>> > > > >> > > >>>> used, the
>> > > > >> > > >>>> >> >> > cluster availability has already been impacted.
>> > > > >> > > >>>> >> >> >
>> > > > >> > > >>>> >> >> > Maybe we can have a metric to count the
>> partitions
>> > > that
>> > > > >> > > sum(ISR,
>> > > > >> > > >>>> ELR)
>> > > > >> > > >>>> >> <
>> > > > >> > > >>>> >> >> min
>> > > > >> > > >>>> >> >> > ISR. What do you think?
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >> >> How about:
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >> >> A.  a metric for the totoal number of
>> under-min-isr
>> > > > >> > partitions?
>> > > > >> > > We
>> > > > >> > > >>>> don't
>> > > > >> > > >>>> >> >> have that in Apache Kafka at the moment.
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >> >> B. a metric for the number of unclean leader
>> > elections
>> > > we
>> > > > >> did
>> > > > >> > > (for
>> > > > >> > > >>>> >> >> simplicity, it can reset to 0 on controller
>> restart:
>> > we
>> > > > >> expect
>> > > > >> > > >>>> people to
>> > > > >> > > >>>> >> >> monitor the change over time anyway)
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >> >> best,
>> > > > >> > > >>>> >> >> Colin
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >> >> >
>> > > > >> > > >>>> >> >> > Yeah, for the ongoing unclean recoveries, the
>> > > > controller
>> > > > >> can
>> > > > >> > > >>>> keep an
>> > > > >> > > >>>> >> >> > accurate count through failover because
>> partition
>> > > > >> > registration
>> > > > >> > > >>>> can
>> > > > >> > > >>>> >> >> indicate
>> > > > >> > > >>>> >> >> > whether a recovery is needed. However, for the
>> > > happened
>> > > > >> > ones,
>> > > > >> > > >>>> unless
>> > > > >> > > >>>> >> we
>> > > > >> > > >>>> >> >> > want to persist the number somewhere, we can
>> only
>> > > > figure
>> > > > >> it
>> > > > >> > > out
>> > > > >> > > >>>> from
>> > > > >> > > >>>> >> the
>> > > > >> > > >>>> >> >> > log.
>> > > > >> > > >>>> >> >> >
>> > > > >> > > >>>> >> >> > On Tue, Sep 12, 2023 at 3:16 PM Colin McCabe <
>> > > > >> > > cmccabe@apache.org
>> > > > >> > > >>>> >
>> > > > >> > > >>>> >> wrote:
>> > > > >> > > >>>> >> >> >
>> > > > >> > > >>>> >> >> >> Also, we should have metrics that show what is
>> > going
>> > > > on
>> > > > >> > with
>> > > > >> > > >>>> regard
>> > > > >> > > >>>> >> to
>> > > > >> > > >>>> >> >> the
>> > > > >> > > >>>> >> >> >> eligible replica set. I'm not sure exactly
>> what to
>> > > > >> suggest,
>> > > > >> > > but
>> > > > >> > > >>>> >> >> something
>> > > > >> > > >>>> >> >> >> that could identify when things are going
>> wrong in
>> > > the
>> > > > >> > > clsuter.
>> > > > >> > > >>>> >> >> >>
>> > > > >> > > >>>> >> >> >> For example, maybe a metric for partitions
>> > > containing
>> > > > >> > > replicas
>> > > > >> > > >>>> that
>> > > > >> > > >>>> >> are
>> > > > >> > > >>>> >> >> >> ineligible to be leader? That would show a
>> spike
>> > > when
>> > > > a
>> > > > >> > > broker
>> > > > >> > > >>>> had an
>> > > > >> > > >>>> >> >> >> unclean restart.
>> > > > >> > > >>>> >> >> >>
>> > > > >> > > >>>> >> >> >> Ideally, we'd also have a metric that indicates
>> > when
>> > > > an
>> > > > >> > > unclear
>> > > > >> > > >>>> >> leader
>> > > > >> > > >>>> >> >> >> election or a recovery happened. It's a bit
>> tricky
>> > > > >> because
>> > > > >> > > the
>> > > > >> > > >>>> simple
>> > > > >> > > >>>> >> >> >> thing, of tracking it per controller, may be a
>> bit
>> > > > >> > confusing
>> > > > >> > > >>>> during
>> > > > >> > > >>>> >> >> >> failovers.
>> > > > >> > > >>>> >> >> >>
>> > > > >> > > >>>> >> >> >> best,
>> > > > >> > > >>>> >> >> >> Colin
>> > > > >> > > >>>> >> >> >>
>> > > > >> > > >>>> >> >> >>
>> > > > >> > > >>>> >> >> >> On Tue, Sep 12, 2023, at 14:25, Colin McCabe
>> > wrote:
>> > > > >> > > >>>> >> >> >> > Hi Calvin,
>> > > > >> > > >>>> >> >> >> >
>> > > > >> > > >>>> >> >> >> > Thanks for the KIP. I think this is a great
>> > > > >> improvement.
>> > > > >> > > >>>> >> >> >> >
>> > > > >> > > >>>> >> >> >> >> Additional High Watermark advance
>> requirement
>> > > > >> > > >>>> >> >> >> >
>> > > > >> > > >>>> >> >> >> > Typo: change "advance" to "advancement"
>> > > > >> > > >>>> >> >> >> >
>> > > > >> > > >>>> >> >> >> >> A bit recap of some key concepts.
>> > > > >> > > >>>> >> >> >> >
>> > > > >> > > >>>> >> >> >> > Typo: change "bit" to "quick"
>> > > > >> > > >>>> >> >> >> >
>> > > > >> > > >>>> >> >> >> >> Ack=1/all produce request. It defines when
>> the
>> > > > Kafka
>> > > > >> > > server
>> > > > >> > > >>>> should
>> > > > >> > > >>>> >> >> >> respond to the produce request
>> > > > >> > > >>>> >> >> >> >
>> > > > >> > > >>>> >> >> >> > I think this section would be clearer if we
>> > talked
>> > > > >> about
>> > > > >> > > the
>> > > > >> > > >>>> new
>> > > > >> > > >>>> >> high
>> > > > >> > > >>>> >> >> >> > watermark advancement requirement first, and
>> > THEN
>> > > > >> talked
>> > > > >> > > >>>> about its
>> > > > >> > > >>>> >> >> >> > impact on acks=0, acks=1, and     acks=all.
>> > > > acks=all
>> > > > >> is
>> > > > >> > of
>> > > > >> > > >>>> course
>> > > > >> > > >>>> >> the
>> > > > >> > > >>>> >> >> >> > main case we care about here, so it would be
>> > good
>> > > to
>> > > > >> lead
>> > > > >> > > with
>> > > > >> > > >>>> >> that,
>> > > > >> > > >>>> >> >> >> > rather than delving into the technicalities
>> of
>> > > > acks=0/1
>> > > > >> > > first.
>> > > > >> > > >>>> >> >> >> >
>> > > > >> > > >>>> >> >> >> >> Unclean recovery
>> > > > >> > > >>>> >> >> >> >
>> > > > >> > > >>>> >> >> >> > So, here you are introducing a new
>> > configuration,
>> > > > >> > > >>>> >> >> >> > unclean.recovery.strategy. The difficult
>> thing
>> > > here
>> > > > is
>> > > > >> > that
>> > > > >> > > >>>> there
>> > > > >> > > >>>> >> is a
>> > > > >> > > >>>> >> >> >> > lot of overlap with
>> > > unclean.leader.election.enable.
>> > > > So
>> > > > >> we
>> > > > >> > > >>>> have 3
>> > > > >> > > >>>> >> >> >> > different settings for
>> > unclean.recovery.strategy,
>> > > > plus
>> > > > >> 2
>> > > > >> > > >>>> different
>> > > > >> > > >>>> >> >> >> > settings for unclean.leader.election.enable,
>> > > giving
>> > > > a
>> > > > >> > cross
>> > > > >> > > >>>> >> product of
>> > > > >> > > >>>> >> >> >> > 6 different options. The following "unclean
>> > > recovery
>> > > > >> > > manager"
>> > > > >> > > >>>> >> section
>> > > > >> > > >>>> >> >> >> > only applies to one fo those 6 different
>> > > > possibilities
>> > > > >> (I
>> > > > >> > > >>>> think?)
>> > > > >> > > >>>> >> >> >> >
>> > > > >> > > >>>> >> >> >> > I simply don't think we need so many
>> different
>> > > > election
>> > > > >> > > types.
>> > > > >> > > >>>> >> Really
>> > > > >> > > >>>> >> >> >> > the use-cases we need are people who want NO
>> > > unclean
>> > > > >> > > >>>> elections,
>> > > > >> > > >>>> >> people
>> > > > >> > > >>>> >> >> >> > who want "the reasonable thing" and people
>> who
>> > > want
>> > > > >> > > >>>> avaialbility at
>> > > > >> > > >>>> >> >> all
>> > > > >> > > >>>> >> >> >> > costs.
>> > > > >> > > >>>> >> >> >> >
>> > > > >> > > >>>> >> >> >> > Overall, I feel like the first half of the
>> KIP
>> > is
>> > > > about
>> > > > >> > the
>> > > > >> > > >>>> ELR,
>> > > > >> > > >>>> >> and
>> > > > >> > > >>>> >> >> >> > the second half is about reworking unclean
>> > leader
>> > > > >> > > election. It
>> > > > >> > > >>>> >> might
>> > > > >> > > >>>> >> >> be
>> > > > >> > > >>>> >> >> >> > better to move that second half to a separate
>> > KIP
>> > > so
>> > > > >> that
>> > > > >> > > we
>> > > > >> > > >>>> can
>> > > > >> > > >>>> >> >> figure
>> > > > >> > > >>>> >> >> >> > it out fully. It should be fine to punt this
>> > until
>> > > > >> later
>> > > > >> > > and
>> > > > >> > > >>>> just
>> > > > >> > > >>>> >> have
>> > > > >> > > >>>> >> >> >> > the current behavior on empty ELR be waiting
>> for
>> > > the
>> > > > >> last
>> > > > >> > > >>>> known
>> > > > >> > > >>>> >> leader
>> > > > >> > > >>>> >> >> >> > to return. After all, that's what we do
>> today.
>> > > > >> > > >>>> >> >> >> >
>> > > > >> > > >>>> >> >> >> >> DescribeTopicRequest
>> > > > >> > > >>>> >> >> >> >
>> > > > >> > > >>>> >> >> >> > Is the intention for AdminClient to use this
>> RPC
>> > > for
>> > > > >> > > >>>> >> >> >> > Admin#describeTopics ? If so, we need to
>> > describe
>> > > > all
>> > > > >> of
>> > > > >> > > the
>> > > > >> > > >>>> >> changes
>> > > > >> > > >>>> >> >> to
>> > > > >> > > >>>> >> >> >> > the admin client API, as well as changes to
>> > > > >> command-line
>> > > > >> > > >>>> tools like
>> > > > >> > > >>>> >> >> >> > kafka-topics.sh (if there are any). For
>> example,
>> > > you
>> > > > >> will
>> > > > >> > > >>>> probably
>> > > > >> > > >>>> >> >> need
>> > > > >> > > >>>> >> >> >> > changes to TopicDescription.java. You will
>> also
>> > > > need to
>> > > > >> > > >>>> provide
>> > > > >> > > >>>> >> all of
>> > > > >> > > >>>> >> >> >> > the things that admin client needs -- for
>> > example,
>> > > > >> > > >>>> >> >> >> > TopicAuthorizedOperations.
>> > > > >> > > >>>> >> >> >> >
>> > > > >> > > >>>> >> >> >> > I also don't think the controller should
>> serve
>> > > this
>> > > > >> > > request.
>> > > > >> > > >>>> We
>> > > > >> > > >>>> >> want
>> > > > >> > > >>>> >> >> to
>> > > > >> > > >>>> >> >> >> > minimize load on the controller. Just like
>> with
>> > > the
>> > > > >> other
>> > > > >> > > >>>> metadata
>> > > > >> > > >>>> >> >> >> > requests like MetadataRequest, this should be
>> > > > served by
>> > > > >> > > >>>> brokers.
>> > > > >> > > >>>> >> >> >> >
>> > > > >> > > >>>> >> >> >> > It's a bit confusing why both topic ID and
>> topic
>> > > > name
>> > > > >> are
>> > > > >> > > >>>> provided
>> > > > >> > > >>>> >> to
>> > > > >> > > >>>> >> >> >> > this API. Is the intention that callers
>> should
>> > set
>> > > > one
>> > > > >> > but
>> > > > >> > > >>>> not the
>> > > > >> > > >>>> >> >> >> > other? Or both? This needs to be clarified.
>> > Also,
>> > > > if we
>> > > > >> > do
>> > > > >> > > >>>> want to
>> > > > >> > > >>>> >> >> >> > support lookups by UUID, that is another
>> thing
>> > > that
>> > > > >> needs
>> > > > >> > > to
>> > > > >> > > >>>> be
>> > > > >> > > >>>> >> added
>> > > > >> > > >>>> >> >> >> > to adminclient.
>> > > > >> > > >>>> >> >> >> >
>> > > > >> > > >>>> >> >> >> > In general, I feel like this should also
>> > probably
>> > > be
>> > > > >> its
>> > > > >> > > own
>> > > > >> > > >>>> KIP
>> > > > >> > > >>>> >> since
>> > > > >> > > >>>> >> >> >> > it's fairly complex
>> > > > >> > > >>>> >> >> >> >
>> > > > >> > > >>>> >> >> >> > best,
>> > > > >> > > >>>> >> >> >> > Colin
>> > > > >> > > >>>> >> >> >> >
>> > > > >> > > >>>> >> >> >> >
>> > > > >> > > >>>> >> >> >> > On Thu, Aug 10, 2023, at 15:46, Calvin Liu
>> > 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
>> > > > >> > > >>>> >> >> >>
>> > > > >> > > >>>> >> >>
>> > > > >> > > >>>> >>
>> > > > >> > > >>>>
>> > > > >> > > >>>
>> > > > >> > >
>> > > > >> >
>> > > > >>
>> > > >
>> > >
>> >
>>
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
Catching up with this discussion.

I was just curious -- have we had other instances where downgrading MV is
not supported? I think Kafka typically tries to support downgrades, and I
couldn't think of other examples.

Thanks,
Justine

On Wed, Oct 4, 2023 at 9:40 AM Calvin Liu <ca...@confluent.io.invalid>
wrote:

> Hi Jun,
> 54. Marked the software downgrading is not supported. As the old controller
> will not understand the new PartitionRecord and PartitionChangeRecord.
> Thanks!
>
> On Wed, Oct 4, 2023 at 9:12 AM Jun Rao <ju...@confluent.io.invalid> wrote:
>
> > Hi, Calvin,
> >
> > Thanks for the reply. Just one more comment.
> >
> > 54. It seems that downgrading MV is supported. Is downgrading the
> software
> > version supported? It would be useful to document that.
> >
> > Thanks,
> >
> > Jun
> >
> > On Tue, Oct 3, 2023 at 4:55 PM Artem Livshits
> > <al...@confluent.io.invalid> wrote:
> >
> > > Hi Colin,
> > >
> > > I think in your example "do_unclean_recovery" would need to do
> different
> > > things depending on the strategy.
> > >
> > > do_unclean_recovery() {
> > >    if (unclean.recovery.manager.enabled) {
> > >     if (strategy == Aggressive)
> > >       use UncleanRecoveryManager(waitLastKnownERL=false)  // just
> inspect
> > > logs from whoever is available
> > >     else
> > >       use  UncleanRecoveryManager(waitLastKnownERL=true)  // must wait
> > for
> > > at least last known ELR
> > >   } else {
> > >     if (strategy == Aggressive)
> > >       choose the last known leader if that is available, or a random
> > leader
> > > if not)
> > >     else
> > >       wait for last known leader to get back
> > >   }
> > > }
> > >
> > > The idea is that the Aggressive strategy would kick in as soon as we
> lost
> > > the leader and would pick a leader from whoever is available; but the
> > > Balanced will only kick in when ELR is empty and will wait for the
> > brokers
> > > that likely have most data to be available.
> > >
> > > On Tue, Oct 3, 2023 at 3:04 PM Colin McCabe <cm...@apache.org>
> wrote:
> > >
> > > > On Tue, Oct 3, 2023, at 10:49, Jun Rao wrote:
> > > > > Hi, Calvin,
> > > > >
> > > > > Thanks for the update KIP. A few more comments.
> > > > >
> > > > > 41. Why would a user choose the option to select a random replica
> as
> > > the
> > > > > leader instead of using unclean.recovery.strateg=Aggressive? It
> seems
> > > > that
> > > > > the latter is strictly better? If that's not the case, could we
> fold
> > > this
> > > > > option under unclean.recovery.strategy instead of introducing a
> > > separate
> > > > > config?
> > > >
> > > > Hi Jun,
> > > >
> > > > I thought the flow of control was:
> > > >
> > > > If there is no leader for the partition {
> > > >   If (there are unfenced ELR members) {
> > > >     choose_an_unfenced_ELR_member
> > > >   } else if (there are fenced ELR members AND strategy=Aggressive) {
> > > >     do_unclean_recovery
> > > >   } else if (there are no ELR members AND strategy != None) {
> > > >     do_unclean_recovery
> > > >   } else {
> > > >     do nothing about the missing leader
> > > >   }
> > > > }
> > > >
> > > > do_unclean_recovery() {
> > > >    if (unclean.recovery.manager.enabled) {
> > > >     use UncleanRecoveryManager
> > > >   } else {
> > > >     choose the last known leader if that is available, or a random
> > leader
> > > > if not)
> > > >   }
> > > > }
> > > >
> > > > However, I think this could be clarified, especially the behavior
> when
> > > > unclean.recovery.manager.enabled=false. Inuitively the goal for
> > > > unclean.recovery.manager.enabled=false is to be "the same as now,
> > mostly"
> > > > but it's very underspecified in the KIP, I agree.
> > > >
> > > > >
> > > > > 50. ElectLeadersRequest: "If more than 20 topics are included, only
> > the
> > > > > first 20 will be served. Others will be returned with
> > DesiredLeaders."
> > > > Hmm,
> > > > > not sure that I understand this. ElectLeadersResponse doesn't have
> a
> > > > > DesiredLeaders field.
> > > > >
> > > > > 51. GetReplicaLogInfo: "If more than 2000 partitions are included,
> > only
> > > > the
> > > > > first 2000 will be served" Do we return an error for the remaining
> > > > > partitions? Actually, should we include an errorCode field at the
> > > > partition
> > > > > level in GetReplicaLogInfoResponse to cover non-existing partitions
> > and
> > > > no
> > > > > authorization, etc?
> > > > >
> > > > > 52. The entry should matches => The entry should match
> > > > >
> > > > > 53. ElectLeadersRequest.DesiredLeaders: Should it be nullable
> since a
> > > > user
> > > > > may not specify DesiredLeaders?
> > > > >
> > > > > 54. Downgrade: Is that indeed possible? I thought earlier you said
> > that
> > > > > once the new version of the records are in the metadata log, one
> > can't
> > > > > downgrade since the old broker doesn't know how to parse the new
> > > version
> > > > of
> > > > > the metadata records?
> > > > >
> > > >
> > > > MetadataVersion downgrade is currently broken but we have fixing it
> on
> > > our
> > > > plate for Kafka 3.7.
> > > >
> > > > The way downgrade works is that "new features" are dropped, leaving
> > only
> > > > the old ones.
> > > >
> > > > > 55. CleanShutdownFile: Should we add a version field for future
> > > > extension?
> > > > >
> > > > > 56. Config changes are public facing. Could we have a separate
> > section
> > > to
> > > > > document all the config changes?
> > > >
> > > > +1. A separate section for this would be good.
> > > >
> > > > best,
> > > > Colin
> > > >
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > > On Mon, Sep 25, 2023 at 4:29 PM Calvin Liu
> > <caliu@confluent.io.invalid
> > > >
> > > > > wrote:
> > > > >
> > > > >> Hi Jun
> > > > >> Thanks for the comments.
> > > > >>
> > > > >> 40. If we change to None, it is not guaranteed for no data loss.
> For
> > > > users
> > > > >> who are not able to validate the data with external resources,
> > manual
> > > > >> intervention does not give a better result but a loss of
> > availability.
> > > > So
> > > > >> practically speaking, the Balance mode would be a better default
> > > value.
> > > > >>
> > > > >> 41. No, it represents how we want to do the unclean leader
> election.
> > > If
> > > > it
> > > > >> is false, the unclean leader election will be the old random way.
> > > > >> Otherwise, the unclean recovery will be used.
> > > > >>
> > > > >> 42. Good catch. Updated.
> > > > >>
> > > > >> 43. Only the first 20 topics will be served. Others will be
> returned
> > > > with
> > > > >> InvalidRequestError
> > > > >>
> > > > >> 44. The order matters. The desired leader entries match with the
> > topic
> > > > >> partition list by the index.
> > > > >>
> > > > >> 45. Thanks! Updated.
> > > > >>
> > > > >> 46. Good advice! Updated.
> > > > >>
> > > > >> 47.1, updated the comment. Basically it will elect the replica in
> > the
> > > > >> desiredLeader field to be the leader
> > > > >>
> > > > >> 47.2 We can let the admin client do the conversion. Using the
> > > > desiredLeader
> > > > >> field in the json format seems easier for users.
> > > > >>
> > > > >> 48. Once the MV version is downgraded, all the ELR related fields
> > will
> > > > be
> > > > >> removed on the next partition change. The controller will also
> > ignore
> > > > the
> > > > >> ELR fields. Updated the KIP.
> > > > >>
> > > > >> 49. Yes, it would be deprecated/removed.
> > > > >>
> > > > >>
> > > > >> On Mon, Sep 25, 2023 at 3:49 PM Jun Rao <jun@confluent.io.invalid
> >
> > > > wrote:
> > > > >>
> > > > >> > Hi, Calvin,
> > > > >> >
> > > > >> > Thanks for the updated KIP. Made another pass. A few more
> comments
> > > > below.
> > > > >> >
> > > > >> > 40. unclean.leader.election.enable.false ->
> > > > >> > unclean.recovery.strategy.Balanced: The Balanced mode could
> still
> > > > lead to
> > > > >> > data loss. So, I am wondering if
> > > unclean.leader.election.enable.false
> > > > >> > should map to None?
> > > > >> >
> > > > >> > 41. unclean.recovery.manager.enabled: I am not sure why we
> > introduce
> > > > this
> > > > >> > additional config. Is it the same as
> > unclean.recovery.strategy=None?
> > > > >> >
> > > > >> > 42. DescribeTopicResponse.TopicAuthorizedOperations: Should this
> > be
> > > at
> > > > >> the
> > > > >> > topic level?
> > > > >> >
> > > > >> > 43. "Limit: 20 topics max per request": Could we describe what
> > > > happens if
> > > > >> > the request includes more than 20 topics?
> > > > >> >
> > > > >> > 44. ElectLeadersRequest.DesiredLeaders: Could we describe
> whether
> > > the
> > > > >> > ordering matters?
> > > > >> >
> > > > >> > 45. GetReplicaLogInfo.TopicPartitions: "about": "The topic
> > > partitions
> > > > to
> > > > >> > elect leaders.": The description in "about" is incorrect.
> > > > >> >
> > > > >> > 46. GetReplicaLogInfoResponse: Should we nest partitions under
> > > > topicId to
> > > > >> > be consistent with other types of responses?
> > > > >> >
> > > > >> > 47. kafka-leader-election.sh:
> > > > >> > 47.1 Could we explain DESIGNATION?
> > > > >> > 47.2 desiredLeader: Should it be a list to match the field in
> > > > >> > ElectLeadersRequest?
> > > > >> >
> > > > >> > 48. We could add a section on downgrade?
> > > > >> >
> > > > >> > 49. LastKnownLeader: This seems only needed in the first phase
> of
> > > > >> > delivering ELR. Will it be removed when the complete KIP is
> > > delivered?
> > > > >> >
> > > > >> > Thanks,
> > > > >> >
> > > > >> > Jun
> > > > >> >
> > > > >> > On Tue, Sep 19, 2023 at 1:30 PM Colin McCabe <
> cmccabe@apache.org>
> > > > wrote:
> > > > >> >
> > > > >> > > Hi Calvin,
> > > > >> > >
> > > > >> > > Thanks for the explanations. I like the idea of using none,
> > > > balanced,
> > > > >> > > aggressive. We also had an offline discussion about why it is
> > good
> > > > to
> > > > >> > use a
> > > > >> > > new config key (basically, so that we can deprecate the old
> one
> > > > which
> > > > >> had
> > > > >> > > only false/true values in 4.0) With these changes, I am +1.
> > > > >> > >
> > > > >> > > best,
> > > > >> > > Colin
> > > > >> > >
> > > > >> > > On Mon, Sep 18, 2023, at 15:54, Calvin Liu wrote:
> > > > >> > > > Hi Colin,
> > > > >> > > > Also, can we deprecate unclean.leader.election.enable in
> 4.0?
> > > > Before
> > > > >> > > that,
> > > > >> > > > we can have both the config unclean.recovery.strategy and
> > > > >> > > > unclean.leader.election.enable
> > > > >> > > > and using the unclean.recovery.Enabled to determine which
> > config
> > > > to
> > > > >> use
> > > > >> > > > during the unclean leader election.
> > > > >> > > >
> > > > >> > > > On Mon, Sep 18, 2023 at 3:51 PM Calvin Liu <
> > caliu@confluent.io>
> > > > >> wrote:
> > > > >> > > >
> > > > >> > > >> Hi Colin,
> > > > >> > > >> For the unclean.recovery.strategy config name, how about we
> > use
> > > > the
> > > > >> > > >> following
> > > > >> > > >> None. It basically means no unclean recovery will be
> > performed.
> > > > >> > > >> Aggressive. It means availability goes first. Whenever the
> > > > partition
> > > > >> > > can't
> > > > >> > > >> elect a durable replica, the controller will try the
> unclean
> > > > >> recovery.
> > > > >> > > >> Balanced. It is the balance point of the availability
> > > > >> > first(Aggressive)
> > > > >> > > >> and least availability(None). The controller performs
> unclean
> > > > >> recovery
> > > > >> > > when
> > > > >> > > >> both ISR and ELR are empty.
> > > > >> > > >>
> > > > >> > > >>
> > > > >> > > >> On Fri, Sep 15, 2023 at 11:42 AM Calvin Liu <
> > > caliu@confluent.io>
> > > > >> > wrote:
> > > > >> > > >>
> > > > >> > > >>> Hi Colin,
> > > > >> > > >>>
> > > > >> > > >>> > So, the proposal is that if someone sets
> > > > >> > > "unclean.leader.election.enable
> > > > >> > > >>> = true"...
> > > > >> > > >>>
> > > > >> > > >>>
> > > > >> > > >>> The idea is to use one of the
> unclean.leader.election.enable
> > > and
> > > > >> > > >>> unclean.recovery.strategy based on the
> > > > unclean.recovery.Enabled. A
> > > > >> > > possible
> > > > >> > > >>> version can be
> > > > >> > > >>>
> > > > >> > > >>> If unclean.recovery.Enabled:
> > > > >> > > >>>
> > > > >> > > >>> {
> > > > >> > > >>>
> > > > >> > > >>> Check unclean.recovery.strategy. If set, use it.
> Otherwise,
> > > > check
> > > > >> > > >>> unclean.leader.election.enable and translate it to
> > > > >> > > >>> unclean.recovery.strategy.
> > > > >> > > >>>
> > > > >> > > >>> } else {
> > > > >> > > >>>
> > > > >> > > >>> Use unclean.leader.election.enable
> > > > >> > > >>>
> > > > >> > > >>> }
> > > > >> > > >>>
> > > > >> > > >>>
> > > > >> > > >>> —--------
> > > > >> > > >>>
> > > > >> > > >>> >The configuration key should be
> > > > >> "unclean.recovery.manager.enabled",
> > > > >> > > >>> right?
> > > > >> > > >>>
> > > > >> > > >>>
> > > > >> > > >>> I think we have two ways of choosing a leader uncleanly,
> > > unclean
> > > > >> > leader
> > > > >> > > >>> election and unclean recovery(log inspection) and we try
> to
> > > > switch
> > > > >> > > between
> > > > >> > > >>> them.
> > > > >> > > >>>
> > > > >> > > >>> Do you mean we want to develop two ways of performing the
> > > > unclean
> > > > >> > > >>> recovery and one of them is using “unclean recovery
> > manager”?
> > > I
> > > > >> guess
> > > > >> > > we
> > > > >> > > >>> haven’t discussed the second way.
> > > > >> > > >>>
> > > > >> > > >>>
> > > > >> > > >>> —-------
> > > > >> > > >>>
> > > > >> > > >>> >How do these 4 levels of overrides interact with your new
> > > > >> > > >>> configurations?
> > > > >> > > >>>
> > > > >> > > >>>
> > > > >> > > >>> I do notice in the Kraft controller code, the method to
> > check
> > > > >> whether
> > > > >> > > >>> perform unclean leader election is hard coded to false
> since
> > > > >> > > >>> 2021(uncleanLeaderElectionEnabledForTopic). Isn’t it a
> good
> > > > chance
> > > > >> to
> > > > >> > > >>> completely deprecate the unclean.leader.election.enable?
> We
> > > > don’t
> > > > >> > even
> > > > >> > > have
> > > > >> > > >>> to worry about the config conversion.
> > > > >> > > >>>
> > > > >> > > >>> On the other hand, whatever the override is, as long as
> the
> > > > >> > controller
> > > > >> > > >>> can have the final effective
> unclean.leader.election.enable,
> > > the
> > > > >> > topic
> > > > >> > > >>> level config unclean.recovery.strategy, the cluster level
> > > config
> > > > >> > > >>> unclean.recovery.Enabled, the controller can calculate the
> > > > correct
> > > > >> > > methods
> > > > >> > > >>> to use right?
> > > > >> > > >>>
> > > > >> > > >>>
> > > > >> > > >>> On Fri, Sep 15, 2023 at 10:02 AM Colin McCabe <
> > > > cmccabe@apache.org>
> > > > >> > > wrote:
> > > > >> > > >>>
> > > > >> > > >>>> On Thu, Sep 14, 2023, at 22:23, Calvin Liu wrote:
> > > > >> > > >>>> > Hi Colin
> > > > >> > > >>>> > 1. I think using the new config name is more clear.
> > > > >> > > >>>> >        a. The unclean leader election is actually
> removed
> > > if
> > > > >> > unclean
> > > > >> > > >>>> > recovery is in use.
> > > > >> > > >>>> >        b. Using multiple values in
> > > > >> unclean.leader.election.enable
> > > > >> > is
> > > > >> > > >>>> > confusing and it will be more confusing after people
> > forget
> > > > >> about
> > > > >> > > this
> > > > >> > > >>>> > discussion.
> > > > >> > > >>>>
> > > > >> > > >>>> Hi Calvin,
> > > > >> > > >>>>
> > > > >> > > >>>> So, the proposal is that if someone sets
> > > > >> > > "unclean.leader.election.enable
> > > > >> > > >>>> = true" but then sets one of your new configurations, the
> > > > value of
> > > > >> > > >>>> unclean.leader.election.enable is ignored? That seems
> less
> > > > clear
> > > > >> to
> > > > >> > > me, not
> > > > >> > > >>>> more. Just in general, having multiple configuration keys
> > to
> > > > >> control
> > > > >> > > the
> > > > >> > > >>>> same thing confuses users. Basically, they are sitting
> at a
> > > > giant
> > > > >> > > control
> > > > >> > > >>>> panel, and some of the levers do nothing.
> > > > >> > > >>>>
> > > > >> > > >>>> > 2. Sorry I forgot to mention in the response that I did
> > add
> > > > the
> > > > >> > > >>>> > unclean.recovery.Enabled flag.
> > > > >> > > >>>>
> > > > >> > > >>>> The configuration key should be
> > > > >> "unclean.recovery.manager.enabled",
> > > > >> > > >>>> right? Becuase we can do "unclean recovery" without the
> > > > manager.
> > > > >> > > Disabling
> > > > >> > > >>>> the manager just means we use a different mechanism for
> > > > recovery.
> > > > >> > > >>>>
> > > > >> > > >>>> >        c. Maybe I underestimated the challenge of
> > replacing
> > > > the
> > > > >> > > >>>> config. Any
> > > > >> > > >>>> > implementation problems ahead?
> > > > >> > > >>>>
> > > > >> > > >>>> There are four levels of overrides for
> > > > >> > unclean.leader.election.enable.
> > > > >> > > >>>>
> > > > >> > > >>>> 1. static configuration for node.
> > > > >> > > >>>>     This goes in the configuration file, typically named
> > > > >> > > >>>> server.properties
> > > > >> > > >>>>
> > > > >> > > >>>> 2. dynamic configuration for node default
> > > > >> > > >>>>   ConfigResource(type=BROKER, name="")
> > > > >> > > >>>>
> > > > >> > > >>>> 3. dynamic configuration for node
> > > > >> > > >>>>   ConfigResource(type=BROKER, name=<controller id>)
> > > > >> > > >>>>
> > > > >> > > >>>> 4. dynamic configuration for topic
> > > > >> > > >>>>   ConfigResource(type=TOPIC, name=<topic-name>)
> > > > >> > > >>>>
> > > > >> > > >>>> How do these 4 levels of overrides interact with your new
> > > > >> > > >>>> configurations? If the new configurations dominate over
> the
> > > old
> > > > >> > ones,
> > > > >> > > it
> > > > >> > > >>>> seems like this will get a lot more confusing to
> implement
> > > (and
> > > > >> also
> > > > >> > > to
> > > > >> > > >>>> use.)
> > > > >> > > >>>>
> > > > >> > > >>>> Again, I'd recommend just adding some new values to
> > > > >> > > >>>> unclean.leader.election.enable. It's simple and will
> > prevent
> > > > user
> > > > >> > > confusion
> > > > >> > > >>>> (as well as developer confusion.)
> > > > >> > > >>>>
> > > > >> > > >>>> best,
> > > > >> > > >>>> Colin
> > > > >> > > >>>>
> > > > >> > > >>>>
> > > > >> > > >>>> > 3. About the admin client, I mentioned 3 changes in the
> > > > client.
> > > > >> > > >>>> Anything
> > > > >> > > >>>> > else I missed in the KIP?
> > > > >> > > >>>> >       a. The client will switch to using the new RPC
> > > instead
> > > > of
> > > > >> > > >>>> > MetadataRequest for the topics.
> > > > >> > > >>>> >       b. The TopicPartitionInfo used in
> TopicDescription
> > > > needs
> > > > >> to
> > > > >> > > add
> > > > >> > > >>>> new
> > > > >> > > >>>> > fields related to the ELR.
> > > > >> > > >>>> >       c. The outputs will add the ELR related fields.
> > > > >> > > >>>> >
> > > > >> > > >>>> > On Thu, Sep 14, 2023 at 9:19 PM Colin McCabe <
> > > > >> cmccabe@apache.org>
> > > > >> > > >>>> wrote:
> > > > >> > > >>>> >
> > > > >> > > >>>> >> Hi Calvin,
> > > > >> > > >>>> >>
> > > > >> > > >>>> >> Thanks for the changes.
> > > > >> > > >>>> >>
> > > > >> > > >>>> >> 1. Earlier I commented that creating
> > > > >> "unclean.recovery.strategy "
> > > > >> > > is
> > > > >> > > >>>> not
> > > > >> > > >>>> >> necessary, and we can just reuse the existing
> > > > >> > > >>>> >> "unclean.leader.election.enable" configuration key.
> > Let's
> > > > >> discuss
> > > > >> > > >>>> that.
> > > > >> > > >>>> >>
> > > > >> > > >>>> >> 2.I also don't understand why you didn't add a
> > > > configuration to
> > > > >> > > >>>> enable or
> > > > >> > > >>>> >> disable the Unclean Recovery Manager. This seems like
> a
> > > very
> > > > >> > simple
> > > > >> > > >>>> way to
> > > > >> > > >>>> >> handle the staging issue which we discussed. The URM
> can
> > > > just
> > > > >> be
> > > > >> > > >>>> turned off
> > > > >> > > >>>> >> until it is production ready. Let's discuss this.
> > > > >> > > >>>> >>
> > > > >> > > >>>> >> 3. You still need to describe the changes to
> AdminClient
> > > > that
> > > > >> are
> > > > >> > > >>>> needed
> > > > >> > > >>>> >> to use DescribeTopicRequest.
> > > > >> > > >>>> >>
> > > > >> > > >>>> >> Keep at it. It's looking better. :)
> > > > >> > > >>>> >>
> > > > >> > > >>>> >> best,
> > > > >> > > >>>> >> Colin
> > > > >> > > >>>> >>
> > > > >> > > >>>> >>
> > > > >> > > >>>> >> On Thu, Sep 14, 2023, at 11:03, Calvin Liu wrote:
> > > > >> > > >>>> >> > Hi Colin
> > > > >> > > >>>> >> > Thanks for the comments!
> > > > >> > > >>>> >> >
> > > > >> > > >>>> >> > I did the following changes
> > > > >> > > >>>> >> >
> > > > >> > > >>>> >> >    1.
> > > > >> > > >>>> >> >
> > > > >> > > >>>> >> >    Simplified the API spec section to only include
> the
> > > > diff.
> > > > >> > > >>>> >> >    2.
> > > > >> > > >>>> >> >
> > > > >> > > >>>> >> >    Reordered the HWM requirement section.
> > > > >> > > >>>> >> >    3.
> > > > >> > > >>>> >> >
> > > > >> > > >>>> >> >    Removed the URM implementation details to keep
> the
> > > > >> necessary
> > > > >> > > >>>> >> >    characteristics to perform the unclean recovery.
> > > > >> > > >>>> >> >    1.
> > > > >> > > >>>> >> >
> > > > >> > > >>>> >> >       When to perform the unclean recovery
> > > > >> > > >>>> >> >       2.
> > > > >> > > >>>> >> >
> > > > >> > > >>>> >> >       Under different config, how the unclean
> recovery
> > > > finds
> > > > >> > the
> > > > >> > > >>>> leader.
> > > > >> > > >>>> >> >       3.
> > > > >> > > >>>> >> >
> > > > >> > > >>>> >> >       How the config unclean.leader.election.enable
> > and
> > > > >> > > >>>> >> >       unclean.recovery.strategy are converted when
> > users
> > > > >> > > >>>> enable/disable
> > > > >> > > >>>> >> the
> > > > >> > > >>>> >> >       unclean recovery.
> > > > >> > > >>>> >> >       4.
> > > > >> > > >>>> >> >
> > > > >> > > >>>> >> >    More details about how we change admin client.
> > > > >> > > >>>> >> >    5.
> > > > >> > > >>>> >> >
> > > > >> > > >>>> >> >    API limits on the GetReplicaLogInfoRequest and
> > > > >> > > >>>> DescribeTopicRequest.
> > > > >> > > >>>> >> >    6.
> > > > >> > > >>>> >> >
> > > > >> > > >>>> >> >    Two metrics added
> > > > >> > > >>>> >> >    1.
> > > > >> > > >>>> >> >
> > > > >> > > >>>> >> >
> > > >  Kafka.controller.global_under_min_isr_partition_count
> > > > >> > > >>>> >> >       2.
> > > > >> > > >>>> >> >
> > > > >> > > >>>> >> >
>  kafka.controller.unclean_recovery_finished_count
> > > > >> > > >>>> >> >
> > > > >> > > >>>> >> >
> > > > >> > > >>>> >> > On Wed, Sep 13, 2023 at 10:46 AM Colin McCabe <
> > > > >> > > cmccabe@apache.org>
> > > > >> > > >>>> >> wrote:
> > > > >> > > >>>> >> >
> > > > >> > > >>>> >> >> On Tue, Sep 12, 2023, at 17:21, Calvin Liu wrote:
> > > > >> > > >>>> >> >> > Hi Colin
> > > > >> > > >>>> >> >> > Thanks for the comments!
> > > > >> > > >>>> >> >> >
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >> >> Hi Calvin,
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >> >> Thanks again for the KIP.
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >> >> One meta-comment: it's usually better to just do a
> > diff
> > > > on a
> > > > >> > > >>>> message
> > > > >> > > >>>> >> spec
> > > > >> > > >>>> >> >> file or java file if you're including changes to it
> > in
> > > > the
> > > > >> > KIP.
> > > > >> > > >>>> This is
> > > > >> > > >>>> >> >> easier to read than looking for "new fields begin"
> > etc.
> > > > in
> > > > >> the
> > > > >> > > >>>> text, and
> > > > >> > > >>>> >> >> gracefully handles the case where existing fields
> > were
> > > > >> > changed.
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >> >> > Rewrite the Additional High Watermark advancement
> > > > >> > requirement
> > > > >> > > >>>> >> >> > There was feedback on this section that some
> > readers
> > > > may
> > > > >> not
> > > > >> > > be
> > > > >> > > >>>> >> familiar
> > > > >> > > >>>> >> >> > with HWM and Ack=0,1,all requests. This can help
> > them
> > > > >> > > understand
> > > > >> > > >>>> the
> > > > >> > > >>>> >> >> > proposal. I will rewrite this part for more
> > > > readability.
> > > > >> > > >>>> >> >> >
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >> >> To be clear, I wasn't suggesting dropping either
> > > > section. I
> > > > >> > > agree
> > > > >> > > >>>> that
> > > > >> > > >>>> >> >> they add useful background. I was just suggesting
> > that
> > > we
> > > > >> > should
> > > > >> > > >>>> discuss
> > > > >> > > >>>> >> >> the "acks" setting AFTER discussing the new high
> > > > watermark
> > > > >> > > >>>> advancement
> > > > >> > > >>>> >> >> conditions. We also should discuss acks=0. While it
> > > isn't
> > > > >> > > >>>> conceptually
> > > > >> > > >>>> >> much
> > > > >> > > >>>> >> >> different than acks=1 here, its omission from this
> > > > section
> > > > >> is
> > > > >> > > >>>> confusing.
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >> >> > Unclean recovery
> > > > >> > > >>>> >> >> >
> > > > >> > > >>>> >> >> > The plan is to replace the
> > > > unclean.leader.election.enable
> > > > >> > with
> > > > >> > > >>>> >> >> > unclean.recovery.strategy. If the Unclean
> Recovery
> > is
> > > > >> > enabled
> > > > >> > > >>>> then it
> > > > >> > > >>>> >> >> deals
> > > > >> > > >>>> >> >> > with the three options in the
> > > > unclean.recovery.strategy.
> > > > >> > > >>>> >> >> >
> > > > >> > > >>>> >> >> >
> > > > >> > > >>>> >> >> > Let’s refine the Unclean Recovery. We have
> already
> > > > taken a
> > > > >> > > lot of
> > > > >> > > >>>> >> >> > suggestions and I hope to enhance the durability
> of
> > > > Kafka
> > > > >> to
> > > > >> > > the
> > > > >> > > >>>> next
> > > > >> > > >>>> >> >> level
> > > > >> > > >>>> >> >> > with this KIP.
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >> >> I am OK with doing the unclean leader recovery
> > > > improvements
> > > > >> in
> > > > >> > > >>>> this KIP.
> > > > >> > > >>>> >> >> However, I think we need to really work on the
> > > > configuration
> > > > >> > > >>>> settings.
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >> >> Configuration overrides are often quite messy. For
> > > > example,
> > > > >> > the
> > > > >> > > >>>> cases
> > > > >> > > >>>> >> >> where we have log.roll.hours and
> log.roll.segment.ms
> > ,
> > > > the
> > > > >> > user
> > > > >> > > >>>> has to
> > > > >> > > >>>> >> >> remember which one takes precedence, and it is not
> > > > obvious.
> > > > >> > So,
> > > > >> > > >>>> rather
> > > > >> > > >>>> >> than
> > > > >> > > >>>> >> >> creating a new configuration, why not add
> additional
> > > > values
> > > > >> to
> > > > >> > > >>>> >> >> "unclean.leader.election.enable"? I think this will
> > be
> > > > >> simpler
> > > > >> > > for
> > > > >> > > >>>> >> people
> > > > >> > > >>>> >> >> to understand, and simpler in the code as well.
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >> >> What if we continued to use
> > > > "unclean.leader.election.enable"
> > > > >> > but
> > > > >> > > >>>> >> extended
> > > > >> > > >>>> >> >> it so that it took a string? Then the string could
> > have
> > > > >> these
> > > > >> > > >>>> values:
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >> >> never
> > > > >> > > >>>> >> >>     never automatically do an unclean leader
> election
> > > > under
> > > > >> > any
> > > > >> > > >>>> >> conditions
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >> >> false / default
> > > > >> > > >>>> >> >>     only do an unclean leader election if there may
> > be
> > > > >> > possible
> > > > >> > > >>>> data
> > > > >> > > >>>> >> loss
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >> >> true / always
> > > > >> > > >>>> >> >>     always do an unclean leader election if we
> can't
> > > > >> > immediately
> > > > >> > > >>>> elect a
> > > > >> > > >>>> >> >> leader
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >> >> It's a bit awkward that false maps to default
> rather
> > > > than to
> > > > >> > > >>>> never. But
> > > > >> > > >>>> >> >> this awkwardness exists if we use two different
> > > > >> configuration
> > > > >> > > keys
> > > > >> > > >>>> as
> > > > >> > > >>>> >> well.
> > > > >> > > >>>> >> >> The reason for the awkwardness is that we simply
> > don't
> > > > want
> > > > >> > most
> > > > >> > > >>>> of the
> > > > >> > > >>>> >> >> people currently setting
> > > > >> unclean.leader.election.enable=false
> > > > >> > to
> > > > >> > > >>>> get the
> > > > >> > > >>>> >> >> "never" behavior. We have to bite that bullet.
> Better
> > > to
> > > > be
> > > > >> > > clear
> > > > >> > > >>>> and
> > > > >> > > >>>> >> >> explicit than hide it.
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >> >> Another thing that's a bit awkward is having two
> > > > different
> > > > >> > ways
> > > > >> > > to
> > > > >> > > >>>> do
> > > > >> > > >>>> >> >> unclean leader election specified in the KIP. You
> > > > descirbe
> > > > >> two
> > > > >> > > >>>> methods:
> > > > >> > > >>>> >> the
> > > > >> > > >>>> >> >> simple "choose the last leader" method, and the
> > > "unclean
> > > > >> > > recovery
> > > > >> > > >>>> >> manager"
> > > > >> > > >>>> >> >> method. I understand why you did it this way --
> > "choose
> > > > the
> > > > >> > last
> > > > >> > > >>>> >> leader" is
> > > > >> > > >>>> >> >> simple, and will help us deliver an implementation
> > > > quickly,
> > > > >> > > while
> > > > >> > > >>>> the
> > > > >> > > >>>> >> URM
> > > > >> > > >>>> >> >> is preferable in the long term. My suggestion here
> is
> > > to
> > > > >> > > separate
> > > > >> > > >>>> the
> > > > >> > > >>>> >> >> decision of HOW to do unclean leader election from
> > the
> > > > >> > decision
> > > > >> > > of
> > > > >> > > >>>> WHEN
> > > > >> > > >>>> >> to
> > > > >> > > >>>> >> >> do it.
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >> >> So in other words, have
> > > "unclean.leader.election.enable"
> > > > >> > specify
> > > > >> > > >>>> when we
> > > > >> > > >>>> >> >> do unclean leader election, and have a new
> > > configuration
> > > > >> like
> > > > >> > > >>>> >> >> "unclean.recovery.manager.enable" to determine if
> we
> > > use
> > > > the
> > > > >> > > URM.
> > > > >> > > >>>> >> >> Presumably the URM will take some time to get fully
> > > > stable,
> > > > >> so
> > > > >> > > >>>> this can
> > > > >> > > >>>> >> >> default to false for a while, and we can flip the
> > > > default to
> > > > >> > > true
> > > > >> > > >>>> when
> > > > >> > > >>>> >> we
> > > > >> > > >>>> >> >> feel ready.
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >> >> The URM is somewhat under-described here. I think
> we
> > > > need a
> > > > >> > few
> > > > >> > > >>>> >> >> configurations here for it. For example, we need a
> > > > >> > > configuration to
> > > > >> > > >>>> >> specify
> > > > >> > > >>>> >> >> how long it should wait for a broker to respond to
> > its
> > > > RPCs
> > > > >> > > before
> > > > >> > > >>>> >> moving
> > > > >> > > >>>> >> >> on. We also need to understand how the URM
> interacts
> > > with
> > > > >> > > >>>> >> >> unclean.leader.election.enable=always. I assume
> that
> > > with
> > > > >> > > "always"
> > > > >> > > >>>> we
> > > > >> > > >>>> >> will
> > > > >> > > >>>> >> >> just unconditionally use the URM rather than
> choosing
> > > > >> > randomly.
> > > > >> > > >>>> But this
> > > > >> > > >>>> >> >> should be spelled out in the KIP.
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >> >> >
> > > > >> > > >>>> >> >> > DescribeTopicRequest
> > > > >> > > >>>> >> >> >
> > > > >> > > >>>> >> >> >    1.
> > > > >> > > >>>> >> >> >    Yes, the plan is to replace the
> MetadataRequest
> > > with
> > > > >> the
> > > > >> > > >>>> >> >> >    DescribeTopicRequest for the admin clients.
> Will
> > > > check
> > > > >> > the
> > > > >> > > >>>> details.
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >> >> Sounds good. But as I said, you need to specify how
> > > > >> > AdminClient
> > > > >> > > >>>> >> interacts
> > > > >> > > >>>> >> >> with the new request. This will involve adding some
> > > > fields
> > > > >> to
> > > > >> > > >>>> >> >> TopicDescription.java. And you need to specify the
> > > > changes
> > > > >> to
> > > > >> > > the
> > > > >> > > >>>> >> >> kafka-topics.sh command line tool. Otherwise we
> > cannot
> > > > use
> > > > >> the
> > > > >> > > >>>> tool to
> > > > >> > > >>>> >> see
> > > > >> > > >>>> >> >> the new information.
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >> >> The new requests, DescribeTopicRequest and
> > > > >> > > >>>> GetReplicaLogInfoRequest,
> > > > >> > > >>>> >> need
> > > > >> > > >>>> >> >> to have limits placed on them so that their size
> > can't
> > > be
> > > > >> > > >>>> infinite. We
> > > > >> > > >>>> >> >> don't want to propagate the current problems of
> > > > >> > MetadataRequest,
> > > > >> > > >>>> where
> > > > >> > > >>>> >> >> clients can request massive responses that can mess
> > up
> > > > the
> > > > >> JVM
> > > > >> > > when
> > > > >> > > >>>> >> handled.
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >> >> Adding limits is simple for
> GetReplicaLogInfoRequest
> > --
> > > > we
> > > > >> can
> > > > >> > > >>>> just say
> > > > >> > > >>>> >> >> that only 2000 partitions at a time can be
> requested.
> > > For
> > > > >> > > >>>> >> >> DescribeTopicRequest we can probably just limit to
> 20
> > > > topics
> > > > >> > or
> > > > >> > > >>>> >> something
> > > > >> > > >>>> >> >> like that, to avoid the complexity of doing
> > pagination
> > > in
> > > > >> this
> > > > >> > > KIP.
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >> >> >    2.
> > > > >> > > >>>> >> >> >    I can let the broker load the ELR info so that
> > > they
> > > > can
> > > > >> > > serve
> > > > >> > > >>>> the
> > > > >> > > >>>> >> >> >    DescribeTopicRequest as well.
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >> >> Yes, it's fine to add to MetadataCache. In fact,
> > you'll
> > > > be
> > > > >> > > loading
> > > > >> > > >>>> it
> > > > >> > > >>>> >> >> anyway once it's added to PartitionImage.
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >> >> >    3.
> > > > >> > > >>>> >> >> >    Yeah, it does not make sense to have the topic
> > id
> > > if
> > > > >> > > >>>> >> >> >    DescribeTopicRequest is only used by the admin
> > > > client.
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >> >> OK. That makes things simpler. We can always
> create a
> > > new
> > > > >> API
> > > > >> > > later
> > > > >> > > >>>> >> >> (hopefully not in this KIP!) to query by topic ID.
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >> >> >
> > > > >> > > >>>> >> >> >
> > > > >> > > >>>> >> >> > Metrics
> > > > >> > > >>>> >> >> >
> > > > >> > > >>>> >> >> > As for overall cluster health metrics, I think
> > > > >> under-min-ISR
> > > > >> > > is
> > > > >> > > >>>> still
> > > > >> > > >>>> >> a
> > > > >> > > >>>> >> >> > useful one. ELR is more like a safety belt. When
> > the
> > > > ELR
> > > > >> is
> > > > >> > > >>>> used, the
> > > > >> > > >>>> >> >> > cluster availability has already been impacted.
> > > > >> > > >>>> >> >> >
> > > > >> > > >>>> >> >> > Maybe we can have a metric to count the
> partitions
> > > that
> > > > >> > > sum(ISR,
> > > > >> > > >>>> ELR)
> > > > >> > > >>>> >> <
> > > > >> > > >>>> >> >> min
> > > > >> > > >>>> >> >> > ISR. What do you think?
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >> >> How about:
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >> >> A.  a metric for the totoal number of under-min-isr
> > > > >> > partitions?
> > > > >> > > We
> > > > >> > > >>>> don't
> > > > >> > > >>>> >> >> have that in Apache Kafka at the moment.
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >> >> B. a metric for the number of unclean leader
> > elections
> > > we
> > > > >> did
> > > > >> > > (for
> > > > >> > > >>>> >> >> simplicity, it can reset to 0 on controller
> restart:
> > we
> > > > >> expect
> > > > >> > > >>>> people to
> > > > >> > > >>>> >> >> monitor the change over time anyway)
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >> >> best,
> > > > >> > > >>>> >> >> Colin
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >> >> >
> > > > >> > > >>>> >> >> > Yeah, for the ongoing unclean recoveries, the
> > > > controller
> > > > >> can
> > > > >> > > >>>> keep an
> > > > >> > > >>>> >> >> > accurate count through failover because partition
> > > > >> > registration
> > > > >> > > >>>> can
> > > > >> > > >>>> >> >> indicate
> > > > >> > > >>>> >> >> > whether a recovery is needed. However, for the
> > > happened
> > > > >> > ones,
> > > > >> > > >>>> unless
> > > > >> > > >>>> >> we
> > > > >> > > >>>> >> >> > want to persist the number somewhere, we can only
> > > > figure
> > > > >> it
> > > > >> > > out
> > > > >> > > >>>> from
> > > > >> > > >>>> >> the
> > > > >> > > >>>> >> >> > log.
> > > > >> > > >>>> >> >> >
> > > > >> > > >>>> >> >> > On Tue, Sep 12, 2023 at 3:16 PM Colin McCabe <
> > > > >> > > cmccabe@apache.org
> > > > >> > > >>>> >
> > > > >> > > >>>> >> wrote:
> > > > >> > > >>>> >> >> >
> > > > >> > > >>>> >> >> >> Also, we should have metrics that show what is
> > going
> > > > on
> > > > >> > with
> > > > >> > > >>>> regard
> > > > >> > > >>>> >> to
> > > > >> > > >>>> >> >> the
> > > > >> > > >>>> >> >> >> eligible replica set. I'm not sure exactly what
> to
> > > > >> suggest,
> > > > >> > > but
> > > > >> > > >>>> >> >> something
> > > > >> > > >>>> >> >> >> that could identify when things are going wrong
> in
> > > the
> > > > >> > > clsuter.
> > > > >> > > >>>> >> >> >>
> > > > >> > > >>>> >> >> >> For example, maybe a metric for partitions
> > > containing
> > > > >> > > replicas
> > > > >> > > >>>> that
> > > > >> > > >>>> >> are
> > > > >> > > >>>> >> >> >> ineligible to be leader? That would show a spike
> > > when
> > > > a
> > > > >> > > broker
> > > > >> > > >>>> had an
> > > > >> > > >>>> >> >> >> unclean restart.
> > > > >> > > >>>> >> >> >>
> > > > >> > > >>>> >> >> >> Ideally, we'd also have a metric that indicates
> > when
> > > > an
> > > > >> > > unclear
> > > > >> > > >>>> >> leader
> > > > >> > > >>>> >> >> >> election or a recovery happened. It's a bit
> tricky
> > > > >> because
> > > > >> > > the
> > > > >> > > >>>> simple
> > > > >> > > >>>> >> >> >> thing, of tracking it per controller, may be a
> bit
> > > > >> > confusing
> > > > >> > > >>>> during
> > > > >> > > >>>> >> >> >> failovers.
> > > > >> > > >>>> >> >> >>
> > > > >> > > >>>> >> >> >> best,
> > > > >> > > >>>> >> >> >> Colin
> > > > >> > > >>>> >> >> >>
> > > > >> > > >>>> >> >> >>
> > > > >> > > >>>> >> >> >> On Tue, Sep 12, 2023, at 14:25, Colin McCabe
> > wrote:
> > > > >> > > >>>> >> >> >> > Hi Calvin,
> > > > >> > > >>>> >> >> >> >
> > > > >> > > >>>> >> >> >> > Thanks for the KIP. I think this is a great
> > > > >> improvement.
> > > > >> > > >>>> >> >> >> >
> > > > >> > > >>>> >> >> >> >> Additional High Watermark advance requirement
> > > > >> > > >>>> >> >> >> >
> > > > >> > > >>>> >> >> >> > Typo: change "advance" to "advancement"
> > > > >> > > >>>> >> >> >> >
> > > > >> > > >>>> >> >> >> >> A bit recap of some key concepts.
> > > > >> > > >>>> >> >> >> >
> > > > >> > > >>>> >> >> >> > Typo: change "bit" to "quick"
> > > > >> > > >>>> >> >> >> >
> > > > >> > > >>>> >> >> >> >> Ack=1/all produce request. It defines when
> the
> > > > Kafka
> > > > >> > > server
> > > > >> > > >>>> should
> > > > >> > > >>>> >> >> >> respond to the produce request
> > > > >> > > >>>> >> >> >> >
> > > > >> > > >>>> >> >> >> > I think this section would be clearer if we
> > talked
> > > > >> about
> > > > >> > > the
> > > > >> > > >>>> new
> > > > >> > > >>>> >> high
> > > > >> > > >>>> >> >> >> > watermark advancement requirement first, and
> > THEN
> > > > >> talked
> > > > >> > > >>>> about its
> > > > >> > > >>>> >> >> >> > impact on acks=0, acks=1, and     acks=all.
> > > > acks=all
> > > > >> is
> > > > >> > of
> > > > >> > > >>>> course
> > > > >> > > >>>> >> the
> > > > >> > > >>>> >> >> >> > main case we care about here, so it would be
> > good
> > > to
> > > > >> lead
> > > > >> > > with
> > > > >> > > >>>> >> that,
> > > > >> > > >>>> >> >> >> > rather than delving into the technicalities of
> > > > acks=0/1
> > > > >> > > first.
> > > > >> > > >>>> >> >> >> >
> > > > >> > > >>>> >> >> >> >> Unclean recovery
> > > > >> > > >>>> >> >> >> >
> > > > >> > > >>>> >> >> >> > So, here you are introducing a new
> > configuration,
> > > > >> > > >>>> >> >> >> > unclean.recovery.strategy. The difficult thing
> > > here
> > > > is
> > > > >> > that
> > > > >> > > >>>> there
> > > > >> > > >>>> >> is a
> > > > >> > > >>>> >> >> >> > lot of overlap with
> > > unclean.leader.election.enable.
> > > > So
> > > > >> we
> > > > >> > > >>>> have 3
> > > > >> > > >>>> >> >> >> > different settings for
> > unclean.recovery.strategy,
> > > > plus
> > > > >> 2
> > > > >> > > >>>> different
> > > > >> > > >>>> >> >> >> > settings for unclean.leader.election.enable,
> > > giving
> > > > a
> > > > >> > cross
> > > > >> > > >>>> >> product of
> > > > >> > > >>>> >> >> >> > 6 different options. The following "unclean
> > > recovery
> > > > >> > > manager"
> > > > >> > > >>>> >> section
> > > > >> > > >>>> >> >> >> > only applies to one fo those 6 different
> > > > possibilities
> > > > >> (I
> > > > >> > > >>>> think?)
> > > > >> > > >>>> >> >> >> >
> > > > >> > > >>>> >> >> >> > I simply don't think we need so many different
> > > > election
> > > > >> > > types.
> > > > >> > > >>>> >> Really
> > > > >> > > >>>> >> >> >> > the use-cases we need are people who want NO
> > > unclean
> > > > >> > > >>>> elections,
> > > > >> > > >>>> >> people
> > > > >> > > >>>> >> >> >> > who want "the reasonable thing" and people who
> > > want
> > > > >> > > >>>> avaialbility at
> > > > >> > > >>>> >> >> all
> > > > >> > > >>>> >> >> >> > costs.
> > > > >> > > >>>> >> >> >> >
> > > > >> > > >>>> >> >> >> > Overall, I feel like the first half of the KIP
> > is
> > > > about
> > > > >> > the
> > > > >> > > >>>> ELR,
> > > > >> > > >>>> >> and
> > > > >> > > >>>> >> >> >> > the second half is about reworking unclean
> > leader
> > > > >> > > election. It
> > > > >> > > >>>> >> might
> > > > >> > > >>>> >> >> be
> > > > >> > > >>>> >> >> >> > better to move that second half to a separate
> > KIP
> > > so
> > > > >> that
> > > > >> > > we
> > > > >> > > >>>> can
> > > > >> > > >>>> >> >> figure
> > > > >> > > >>>> >> >> >> > it out fully. It should be fine to punt this
> > until
> > > > >> later
> > > > >> > > and
> > > > >> > > >>>> just
> > > > >> > > >>>> >> have
> > > > >> > > >>>> >> >> >> > the current behavior on empty ELR be waiting
> for
> > > the
> > > > >> last
> > > > >> > > >>>> known
> > > > >> > > >>>> >> leader
> > > > >> > > >>>> >> >> >> > to return. After all, that's what we do today.
> > > > >> > > >>>> >> >> >> >
> > > > >> > > >>>> >> >> >> >> DescribeTopicRequest
> > > > >> > > >>>> >> >> >> >
> > > > >> > > >>>> >> >> >> > Is the intention for AdminClient to use this
> RPC
> > > for
> > > > >> > > >>>> >> >> >> > Admin#describeTopics ? If so, we need to
> > describe
> > > > all
> > > > >> of
> > > > >> > > the
> > > > >> > > >>>> >> changes
> > > > >> > > >>>> >> >> to
> > > > >> > > >>>> >> >> >> > the admin client API, as well as changes to
> > > > >> command-line
> > > > >> > > >>>> tools like
> > > > >> > > >>>> >> >> >> > kafka-topics.sh (if there are any). For
> example,
> > > you
> > > > >> will
> > > > >> > > >>>> probably
> > > > >> > > >>>> >> >> need
> > > > >> > > >>>> >> >> >> > changes to TopicDescription.java. You will
> also
> > > > need to
> > > > >> > > >>>> provide
> > > > >> > > >>>> >> all of
> > > > >> > > >>>> >> >> >> > the things that admin client needs -- for
> > example,
> > > > >> > > >>>> >> >> >> > TopicAuthorizedOperations.
> > > > >> > > >>>> >> >> >> >
> > > > >> > > >>>> >> >> >> > I also don't think the controller should serve
> > > this
> > > > >> > > request.
> > > > >> > > >>>> We
> > > > >> > > >>>> >> want
> > > > >> > > >>>> >> >> to
> > > > >> > > >>>> >> >> >> > minimize load on the controller. Just like
> with
> > > the
> > > > >> other
> > > > >> > > >>>> metadata
> > > > >> > > >>>> >> >> >> > requests like MetadataRequest, this should be
> > > > served by
> > > > >> > > >>>> brokers.
> > > > >> > > >>>> >> >> >> >
> > > > >> > > >>>> >> >> >> > It's a bit confusing why both topic ID and
> topic
> > > > name
> > > > >> are
> > > > >> > > >>>> provided
> > > > >> > > >>>> >> to
> > > > >> > > >>>> >> >> >> > this API. Is the intention that callers should
> > set
> > > > one
> > > > >> > but
> > > > >> > > >>>> not the
> > > > >> > > >>>> >> >> >> > other? Or both? This needs to be clarified.
> > Also,
> > > > if we
> > > > >> > do
> > > > >> > > >>>> want to
> > > > >> > > >>>> >> >> >> > support lookups by UUID, that is another thing
> > > that
> > > > >> needs
> > > > >> > > to
> > > > >> > > >>>> be
> > > > >> > > >>>> >> added
> > > > >> > > >>>> >> >> >> > to adminclient.
> > > > >> > > >>>> >> >> >> >
> > > > >> > > >>>> >> >> >> > In general, I feel like this should also
> > probably
> > > be
> > > > >> its
> > > > >> > > own
> > > > >> > > >>>> KIP
> > > > >> > > >>>> >> since
> > > > >> > > >>>> >> >> >> > it's fairly complex
> > > > >> > > >>>> >> >> >> >
> > > > >> > > >>>> >> >> >> > best,
> > > > >> > > >>>> >> >> >> > Colin
> > > > >> > > >>>> >> >> >> >
> > > > >> > > >>>> >> >> >> >
> > > > >> > > >>>> >> >> >> > On Thu, Aug 10, 2023, at 15:46, Calvin Liu
> > 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
> > > > >> > > >>>> >> >> >>
> > > > >> > > >>>> >> >>
> > > > >> > > >>>> >>
> > > > >> > > >>>>
> > > > >> > > >>>
> > > > >> > >
> > > > >> >
> > > > >>
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Calvin Liu <ca...@confluent.io.INVALID>.
Hi Jun,
54. Marked the software downgrading is not supported. As the old controller
will not understand the new PartitionRecord and PartitionChangeRecord.
Thanks!

On Wed, Oct 4, 2023 at 9:12 AM Jun Rao <ju...@confluent.io.invalid> wrote:

> Hi, Calvin,
>
> Thanks for the reply. Just one more comment.
>
> 54. It seems that downgrading MV is supported. Is downgrading the software
> version supported? It would be useful to document that.
>
> Thanks,
>
> Jun
>
> On Tue, Oct 3, 2023 at 4:55 PM Artem Livshits
> <al...@confluent.io.invalid> wrote:
>
> > Hi Colin,
> >
> > I think in your example "do_unclean_recovery" would need to do different
> > things depending on the strategy.
> >
> > do_unclean_recovery() {
> >    if (unclean.recovery.manager.enabled) {
> >     if (strategy == Aggressive)
> >       use UncleanRecoveryManager(waitLastKnownERL=false)  // just inspect
> > logs from whoever is available
> >     else
> >       use  UncleanRecoveryManager(waitLastKnownERL=true)  // must wait
> for
> > at least last known ELR
> >   } else {
> >     if (strategy == Aggressive)
> >       choose the last known leader if that is available, or a random
> leader
> > if not)
> >     else
> >       wait for last known leader to get back
> >   }
> > }
> >
> > The idea is that the Aggressive strategy would kick in as soon as we lost
> > the leader and would pick a leader from whoever is available; but the
> > Balanced will only kick in when ELR is empty and will wait for the
> brokers
> > that likely have most data to be available.
> >
> > On Tue, Oct 3, 2023 at 3:04 PM Colin McCabe <cm...@apache.org> wrote:
> >
> > > On Tue, Oct 3, 2023, at 10:49, Jun Rao wrote:
> > > > Hi, Calvin,
> > > >
> > > > Thanks for the update KIP. A few more comments.
> > > >
> > > > 41. Why would a user choose the option to select a random replica as
> > the
> > > > leader instead of using unclean.recovery.strateg=Aggressive? It seems
> > > that
> > > > the latter is strictly better? If that's not the case, could we fold
> > this
> > > > option under unclean.recovery.strategy instead of introducing a
> > separate
> > > > config?
> > >
> > > Hi Jun,
> > >
> > > I thought the flow of control was:
> > >
> > > If there is no leader for the partition {
> > >   If (there are unfenced ELR members) {
> > >     choose_an_unfenced_ELR_member
> > >   } else if (there are fenced ELR members AND strategy=Aggressive) {
> > >     do_unclean_recovery
> > >   } else if (there are no ELR members AND strategy != None) {
> > >     do_unclean_recovery
> > >   } else {
> > >     do nothing about the missing leader
> > >   }
> > > }
> > >
> > > do_unclean_recovery() {
> > >    if (unclean.recovery.manager.enabled) {
> > >     use UncleanRecoveryManager
> > >   } else {
> > >     choose the last known leader if that is available, or a random
> leader
> > > if not)
> > >   }
> > > }
> > >
> > > However, I think this could be clarified, especially the behavior when
> > > unclean.recovery.manager.enabled=false. Inuitively the goal for
> > > unclean.recovery.manager.enabled=false is to be "the same as now,
> mostly"
> > > but it's very underspecified in the KIP, I agree.
> > >
> > > >
> > > > 50. ElectLeadersRequest: "If more than 20 topics are included, only
> the
> > > > first 20 will be served. Others will be returned with
> DesiredLeaders."
> > > Hmm,
> > > > not sure that I understand this. ElectLeadersResponse doesn't have a
> > > > DesiredLeaders field.
> > > >
> > > > 51. GetReplicaLogInfo: "If more than 2000 partitions are included,
> only
> > > the
> > > > first 2000 will be served" Do we return an error for the remaining
> > > > partitions? Actually, should we include an errorCode field at the
> > > partition
> > > > level in GetReplicaLogInfoResponse to cover non-existing partitions
> and
> > > no
> > > > authorization, etc?
> > > >
> > > > 52. The entry should matches => The entry should match
> > > >
> > > > 53. ElectLeadersRequest.DesiredLeaders: Should it be nullable since a
> > > user
> > > > may not specify DesiredLeaders?
> > > >
> > > > 54. Downgrade: Is that indeed possible? I thought earlier you said
> that
> > > > once the new version of the records are in the metadata log, one
> can't
> > > > downgrade since the old broker doesn't know how to parse the new
> > version
> > > of
> > > > the metadata records?
> > > >
> > >
> > > MetadataVersion downgrade is currently broken but we have fixing it on
> > our
> > > plate for Kafka 3.7.
> > >
> > > The way downgrade works is that "new features" are dropped, leaving
> only
> > > the old ones.
> > >
> > > > 55. CleanShutdownFile: Should we add a version field for future
> > > extension?
> > > >
> > > > 56. Config changes are public facing. Could we have a separate
> section
> > to
> > > > document all the config changes?
> > >
> > > +1. A separate section for this would be good.
> > >
> > > best,
> > > Colin
> > >
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > > On Mon, Sep 25, 2023 at 4:29 PM Calvin Liu
> <caliu@confluent.io.invalid
> > >
> > > > wrote:
> > > >
> > > >> Hi Jun
> > > >> Thanks for the comments.
> > > >>
> > > >> 40. If we change to None, it is not guaranteed for no data loss. For
> > > users
> > > >> who are not able to validate the data with external resources,
> manual
> > > >> intervention does not give a better result but a loss of
> availability.
> > > So
> > > >> practically speaking, the Balance mode would be a better default
> > value.
> > > >>
> > > >> 41. No, it represents how we want to do the unclean leader election.
> > If
> > > it
> > > >> is false, the unclean leader election will be the old random way.
> > > >> Otherwise, the unclean recovery will be used.
> > > >>
> > > >> 42. Good catch. Updated.
> > > >>
> > > >> 43. Only the first 20 topics will be served. Others will be returned
> > > with
> > > >> InvalidRequestError
> > > >>
> > > >> 44. The order matters. The desired leader entries match with the
> topic
> > > >> partition list by the index.
> > > >>
> > > >> 45. Thanks! Updated.
> > > >>
> > > >> 46. Good advice! Updated.
> > > >>
> > > >> 47.1, updated the comment. Basically it will elect the replica in
> the
> > > >> desiredLeader field to be the leader
> > > >>
> > > >> 47.2 We can let the admin client do the conversion. Using the
> > > desiredLeader
> > > >> field in the json format seems easier for users.
> > > >>
> > > >> 48. Once the MV version is downgraded, all the ELR related fields
> will
> > > be
> > > >> removed on the next partition change. The controller will also
> ignore
> > > the
> > > >> ELR fields. Updated the KIP.
> > > >>
> > > >> 49. Yes, it would be deprecated/removed.
> > > >>
> > > >>
> > > >> On Mon, Sep 25, 2023 at 3:49 PM Jun Rao <ju...@confluent.io.invalid>
> > > wrote:
> > > >>
> > > >> > Hi, Calvin,
> > > >> >
> > > >> > Thanks for the updated KIP. Made another pass. A few more comments
> > > below.
> > > >> >
> > > >> > 40. unclean.leader.election.enable.false ->
> > > >> > unclean.recovery.strategy.Balanced: The Balanced mode could still
> > > lead to
> > > >> > data loss. So, I am wondering if
> > unclean.leader.election.enable.false
> > > >> > should map to None?
> > > >> >
> > > >> > 41. unclean.recovery.manager.enabled: I am not sure why we
> introduce
> > > this
> > > >> > additional config. Is it the same as
> unclean.recovery.strategy=None?
> > > >> >
> > > >> > 42. DescribeTopicResponse.TopicAuthorizedOperations: Should this
> be
> > at
> > > >> the
> > > >> > topic level?
> > > >> >
> > > >> > 43. "Limit: 20 topics max per request": Could we describe what
> > > happens if
> > > >> > the request includes more than 20 topics?
> > > >> >
> > > >> > 44. ElectLeadersRequest.DesiredLeaders: Could we describe whether
> > the
> > > >> > ordering matters?
> > > >> >
> > > >> > 45. GetReplicaLogInfo.TopicPartitions: "about": "The topic
> > partitions
> > > to
> > > >> > elect leaders.": The description in "about" is incorrect.
> > > >> >
> > > >> > 46. GetReplicaLogInfoResponse: Should we nest partitions under
> > > topicId to
> > > >> > be consistent with other types of responses?
> > > >> >
> > > >> > 47. kafka-leader-election.sh:
> > > >> > 47.1 Could we explain DESIGNATION?
> > > >> > 47.2 desiredLeader: Should it be a list to match the field in
> > > >> > ElectLeadersRequest?
> > > >> >
> > > >> > 48. We could add a section on downgrade?
> > > >> >
> > > >> > 49. LastKnownLeader: This seems only needed in the first phase of
> > > >> > delivering ELR. Will it be removed when the complete KIP is
> > delivered?
> > > >> >
> > > >> > Thanks,
> > > >> >
> > > >> > Jun
> > > >> >
> > > >> > On Tue, Sep 19, 2023 at 1:30 PM Colin McCabe <cm...@apache.org>
> > > wrote:
> > > >> >
> > > >> > > Hi Calvin,
> > > >> > >
> > > >> > > Thanks for the explanations. I like the idea of using none,
> > > balanced,
> > > >> > > aggressive. We also had an offline discussion about why it is
> good
> > > to
> > > >> > use a
> > > >> > > new config key (basically, so that we can deprecate the old one
> > > which
> > > >> had
> > > >> > > only false/true values in 4.0) With these changes, I am +1.
> > > >> > >
> > > >> > > best,
> > > >> > > Colin
> > > >> > >
> > > >> > > On Mon, Sep 18, 2023, at 15:54, Calvin Liu wrote:
> > > >> > > > Hi Colin,
> > > >> > > > Also, can we deprecate unclean.leader.election.enable in 4.0?
> > > Before
> > > >> > > that,
> > > >> > > > we can have both the config unclean.recovery.strategy and
> > > >> > > > unclean.leader.election.enable
> > > >> > > > and using the unclean.recovery.Enabled to determine which
> config
> > > to
> > > >> use
> > > >> > > > during the unclean leader election.
> > > >> > > >
> > > >> > > > On Mon, Sep 18, 2023 at 3:51 PM Calvin Liu <
> caliu@confluent.io>
> > > >> wrote:
> > > >> > > >
> > > >> > > >> Hi Colin,
> > > >> > > >> For the unclean.recovery.strategy config name, how about we
> use
> > > the
> > > >> > > >> following
> > > >> > > >> None. It basically means no unclean recovery will be
> performed.
> > > >> > > >> Aggressive. It means availability goes first. Whenever the
> > > partition
> > > >> > > can't
> > > >> > > >> elect a durable replica, the controller will try the unclean
> > > >> recovery.
> > > >> > > >> Balanced. It is the balance point of the availability
> > > >> > first(Aggressive)
> > > >> > > >> and least availability(None). The controller performs unclean
> > > >> recovery
> > > >> > > when
> > > >> > > >> both ISR and ELR are empty.
> > > >> > > >>
> > > >> > > >>
> > > >> > > >> On Fri, Sep 15, 2023 at 11:42 AM Calvin Liu <
> > caliu@confluent.io>
> > > >> > wrote:
> > > >> > > >>
> > > >> > > >>> Hi Colin,
> > > >> > > >>>
> > > >> > > >>> > So, the proposal is that if someone sets
> > > >> > > "unclean.leader.election.enable
> > > >> > > >>> = true"...
> > > >> > > >>>
> > > >> > > >>>
> > > >> > > >>> The idea is to use one of the unclean.leader.election.enable
> > and
> > > >> > > >>> unclean.recovery.strategy based on the
> > > unclean.recovery.Enabled. A
> > > >> > > possible
> > > >> > > >>> version can be
> > > >> > > >>>
> > > >> > > >>> If unclean.recovery.Enabled:
> > > >> > > >>>
> > > >> > > >>> {
> > > >> > > >>>
> > > >> > > >>> Check unclean.recovery.strategy. If set, use it. Otherwise,
> > > check
> > > >> > > >>> unclean.leader.election.enable and translate it to
> > > >> > > >>> unclean.recovery.strategy.
> > > >> > > >>>
> > > >> > > >>> } else {
> > > >> > > >>>
> > > >> > > >>> Use unclean.leader.election.enable
> > > >> > > >>>
> > > >> > > >>> }
> > > >> > > >>>
> > > >> > > >>>
> > > >> > > >>> —--------
> > > >> > > >>>
> > > >> > > >>> >The configuration key should be
> > > >> "unclean.recovery.manager.enabled",
> > > >> > > >>> right?
> > > >> > > >>>
> > > >> > > >>>
> > > >> > > >>> I think we have two ways of choosing a leader uncleanly,
> > unclean
> > > >> > leader
> > > >> > > >>> election and unclean recovery(log inspection) and we try to
> > > switch
> > > >> > > between
> > > >> > > >>> them.
> > > >> > > >>>
> > > >> > > >>> Do you mean we want to develop two ways of performing the
> > > unclean
> > > >> > > >>> recovery and one of them is using “unclean recovery
> manager”?
> > I
> > > >> guess
> > > >> > > we
> > > >> > > >>> haven’t discussed the second way.
> > > >> > > >>>
> > > >> > > >>>
> > > >> > > >>> —-------
> > > >> > > >>>
> > > >> > > >>> >How do these 4 levels of overrides interact with your new
> > > >> > > >>> configurations?
> > > >> > > >>>
> > > >> > > >>>
> > > >> > > >>> I do notice in the Kraft controller code, the method to
> check
> > > >> whether
> > > >> > > >>> perform unclean leader election is hard coded to false since
> > > >> > > >>> 2021(uncleanLeaderElectionEnabledForTopic). Isn’t it a good
> > > chance
> > > >> to
> > > >> > > >>> completely deprecate the unclean.leader.election.enable? We
> > > don’t
> > > >> > even
> > > >> > > have
> > > >> > > >>> to worry about the config conversion.
> > > >> > > >>>
> > > >> > > >>> On the other hand, whatever the override is, as long as the
> > > >> > controller
> > > >> > > >>> can have the final effective unclean.leader.election.enable,
> > the
> > > >> > topic
> > > >> > > >>> level config unclean.recovery.strategy, the cluster level
> > config
> > > >> > > >>> unclean.recovery.Enabled, the controller can calculate the
> > > correct
> > > >> > > methods
> > > >> > > >>> to use right?
> > > >> > > >>>
> > > >> > > >>>
> > > >> > > >>> On Fri, Sep 15, 2023 at 10:02 AM Colin McCabe <
> > > cmccabe@apache.org>
> > > >> > > wrote:
> > > >> > > >>>
> > > >> > > >>>> On Thu, Sep 14, 2023, at 22:23, Calvin Liu wrote:
> > > >> > > >>>> > Hi Colin
> > > >> > > >>>> > 1. I think using the new config name is more clear.
> > > >> > > >>>> >        a. The unclean leader election is actually removed
> > if
> > > >> > unclean
> > > >> > > >>>> > recovery is in use.
> > > >> > > >>>> >        b. Using multiple values in
> > > >> unclean.leader.election.enable
> > > >> > is
> > > >> > > >>>> > confusing and it will be more confusing after people
> forget
> > > >> about
> > > >> > > this
> > > >> > > >>>> > discussion.
> > > >> > > >>>>
> > > >> > > >>>> Hi Calvin,
> > > >> > > >>>>
> > > >> > > >>>> So, the proposal is that if someone sets
> > > >> > > "unclean.leader.election.enable
> > > >> > > >>>> = true" but then sets one of your new configurations, the
> > > value of
> > > >> > > >>>> unclean.leader.election.enable is ignored? That seems less
> > > clear
> > > >> to
> > > >> > > me, not
> > > >> > > >>>> more. Just in general, having multiple configuration keys
> to
> > > >> control
> > > >> > > the
> > > >> > > >>>> same thing confuses users. Basically, they are sitting at a
> > > giant
> > > >> > > control
> > > >> > > >>>> panel, and some of the levers do nothing.
> > > >> > > >>>>
> > > >> > > >>>> > 2. Sorry I forgot to mention in the response that I did
> add
> > > the
> > > >> > > >>>> > unclean.recovery.Enabled flag.
> > > >> > > >>>>
> > > >> > > >>>> The configuration key should be
> > > >> "unclean.recovery.manager.enabled",
> > > >> > > >>>> right? Becuase we can do "unclean recovery" without the
> > > manager.
> > > >> > > Disabling
> > > >> > > >>>> the manager just means we use a different mechanism for
> > > recovery.
> > > >> > > >>>>
> > > >> > > >>>> >        c. Maybe I underestimated the challenge of
> replacing
> > > the
> > > >> > > >>>> config. Any
> > > >> > > >>>> > implementation problems ahead?
> > > >> > > >>>>
> > > >> > > >>>> There are four levels of overrides for
> > > >> > unclean.leader.election.enable.
> > > >> > > >>>>
> > > >> > > >>>> 1. static configuration for node.
> > > >> > > >>>>     This goes in the configuration file, typically named
> > > >> > > >>>> server.properties
> > > >> > > >>>>
> > > >> > > >>>> 2. dynamic configuration for node default
> > > >> > > >>>>   ConfigResource(type=BROKER, name="")
> > > >> > > >>>>
> > > >> > > >>>> 3. dynamic configuration for node
> > > >> > > >>>>   ConfigResource(type=BROKER, name=<controller id>)
> > > >> > > >>>>
> > > >> > > >>>> 4. dynamic configuration for topic
> > > >> > > >>>>   ConfigResource(type=TOPIC, name=<topic-name>)
> > > >> > > >>>>
> > > >> > > >>>> How do these 4 levels of overrides interact with your new
> > > >> > > >>>> configurations? If the new configurations dominate over the
> > old
> > > >> > ones,
> > > >> > > it
> > > >> > > >>>> seems like this will get a lot more confusing to implement
> > (and
> > > >> also
> > > >> > > to
> > > >> > > >>>> use.)
> > > >> > > >>>>
> > > >> > > >>>> Again, I'd recommend just adding some new values to
> > > >> > > >>>> unclean.leader.election.enable. It's simple and will
> prevent
> > > user
> > > >> > > confusion
> > > >> > > >>>> (as well as developer confusion.)
> > > >> > > >>>>
> > > >> > > >>>> best,
> > > >> > > >>>> Colin
> > > >> > > >>>>
> > > >> > > >>>>
> > > >> > > >>>> > 3. About the admin client, I mentioned 3 changes in the
> > > client.
> > > >> > > >>>> Anything
> > > >> > > >>>> > else I missed in the KIP?
> > > >> > > >>>> >       a. The client will switch to using the new RPC
> > instead
> > > of
> > > >> > > >>>> > MetadataRequest for the topics.
> > > >> > > >>>> >       b. The TopicPartitionInfo used in TopicDescription
> > > needs
> > > >> to
> > > >> > > add
> > > >> > > >>>> new
> > > >> > > >>>> > fields related to the ELR.
> > > >> > > >>>> >       c. The outputs will add the ELR related fields.
> > > >> > > >>>> >
> > > >> > > >>>> > On Thu, Sep 14, 2023 at 9:19 PM Colin McCabe <
> > > >> cmccabe@apache.org>
> > > >> > > >>>> wrote:
> > > >> > > >>>> >
> > > >> > > >>>> >> Hi Calvin,
> > > >> > > >>>> >>
> > > >> > > >>>> >> Thanks for the changes.
> > > >> > > >>>> >>
> > > >> > > >>>> >> 1. Earlier I commented that creating
> > > >> "unclean.recovery.strategy "
> > > >> > > is
> > > >> > > >>>> not
> > > >> > > >>>> >> necessary, and we can just reuse the existing
> > > >> > > >>>> >> "unclean.leader.election.enable" configuration key.
> Let's
> > > >> discuss
> > > >> > > >>>> that.
> > > >> > > >>>> >>
> > > >> > > >>>> >> 2.I also don't understand why you didn't add a
> > > configuration to
> > > >> > > >>>> enable or
> > > >> > > >>>> >> disable the Unclean Recovery Manager. This seems like a
> > very
> > > >> > simple
> > > >> > > >>>> way to
> > > >> > > >>>> >> handle the staging issue which we discussed. The URM can
> > > just
> > > >> be
> > > >> > > >>>> turned off
> > > >> > > >>>> >> until it is production ready. Let's discuss this.
> > > >> > > >>>> >>
> > > >> > > >>>> >> 3. You still need to describe the changes to AdminClient
> > > that
> > > >> are
> > > >> > > >>>> needed
> > > >> > > >>>> >> to use DescribeTopicRequest.
> > > >> > > >>>> >>
> > > >> > > >>>> >> Keep at it. It's looking better. :)
> > > >> > > >>>> >>
> > > >> > > >>>> >> best,
> > > >> > > >>>> >> Colin
> > > >> > > >>>> >>
> > > >> > > >>>> >>
> > > >> > > >>>> >> On Thu, Sep 14, 2023, at 11:03, Calvin Liu wrote:
> > > >> > > >>>> >> > Hi Colin
> > > >> > > >>>> >> > Thanks for the comments!
> > > >> > > >>>> >> >
> > > >> > > >>>> >> > I did the following changes
> > > >> > > >>>> >> >
> > > >> > > >>>> >> >    1.
> > > >> > > >>>> >> >
> > > >> > > >>>> >> >    Simplified the API spec section to only include the
> > > diff.
> > > >> > > >>>> >> >    2.
> > > >> > > >>>> >> >
> > > >> > > >>>> >> >    Reordered the HWM requirement section.
> > > >> > > >>>> >> >    3.
> > > >> > > >>>> >> >
> > > >> > > >>>> >> >    Removed the URM implementation details to keep the
> > > >> necessary
> > > >> > > >>>> >> >    characteristics to perform the unclean recovery.
> > > >> > > >>>> >> >    1.
> > > >> > > >>>> >> >
> > > >> > > >>>> >> >       When to perform the unclean recovery
> > > >> > > >>>> >> >       2.
> > > >> > > >>>> >> >
> > > >> > > >>>> >> >       Under different config, how the unclean recovery
> > > finds
> > > >> > the
> > > >> > > >>>> leader.
> > > >> > > >>>> >> >       3.
> > > >> > > >>>> >> >
> > > >> > > >>>> >> >       How the config unclean.leader.election.enable
> and
> > > >> > > >>>> >> >       unclean.recovery.strategy are converted when
> users
> > > >> > > >>>> enable/disable
> > > >> > > >>>> >> the
> > > >> > > >>>> >> >       unclean recovery.
> > > >> > > >>>> >> >       4.
> > > >> > > >>>> >> >
> > > >> > > >>>> >> >    More details about how we change admin client.
> > > >> > > >>>> >> >    5.
> > > >> > > >>>> >> >
> > > >> > > >>>> >> >    API limits on the GetReplicaLogInfoRequest and
> > > >> > > >>>> DescribeTopicRequest.
> > > >> > > >>>> >> >    6.
> > > >> > > >>>> >> >
> > > >> > > >>>> >> >    Two metrics added
> > > >> > > >>>> >> >    1.
> > > >> > > >>>> >> >
> > > >> > > >>>> >> >
> > >  Kafka.controller.global_under_min_isr_partition_count
> > > >> > > >>>> >> >       2.
> > > >> > > >>>> >> >
> > > >> > > >>>> >> >       kafka.controller.unclean_recovery_finished_count
> > > >> > > >>>> >> >
> > > >> > > >>>> >> >
> > > >> > > >>>> >> > On Wed, Sep 13, 2023 at 10:46 AM Colin McCabe <
> > > >> > > cmccabe@apache.org>
> > > >> > > >>>> >> wrote:
> > > >> > > >>>> >> >
> > > >> > > >>>> >> >> On Tue, Sep 12, 2023, at 17:21, Calvin Liu wrote:
> > > >> > > >>>> >> >> > Hi Colin
> > > >> > > >>>> >> >> > Thanks for the comments!
> > > >> > > >>>> >> >> >
> > > >> > > >>>> >> >>
> > > >> > > >>>> >> >> Hi Calvin,
> > > >> > > >>>> >> >>
> > > >> > > >>>> >> >> Thanks again for the KIP.
> > > >> > > >>>> >> >>
> > > >> > > >>>> >> >> One meta-comment: it's usually better to just do a
> diff
> > > on a
> > > >> > > >>>> message
> > > >> > > >>>> >> spec
> > > >> > > >>>> >> >> file or java file if you're including changes to it
> in
> > > the
> > > >> > KIP.
> > > >> > > >>>> This is
> > > >> > > >>>> >> >> easier to read than looking for "new fields begin"
> etc.
> > > in
> > > >> the
> > > >> > > >>>> text, and
> > > >> > > >>>> >> >> gracefully handles the case where existing fields
> were
> > > >> > changed.
> > > >> > > >>>> >> >>
> > > >> > > >>>> >> >> > Rewrite the Additional High Watermark advancement
> > > >> > requirement
> > > >> > > >>>> >> >> > There was feedback on this section that some
> readers
> > > may
> > > >> not
> > > >> > > be
> > > >> > > >>>> >> familiar
> > > >> > > >>>> >> >> > with HWM and Ack=0,1,all requests. This can help
> them
> > > >> > > understand
> > > >> > > >>>> the
> > > >> > > >>>> >> >> > proposal. I will rewrite this part for more
> > > readability.
> > > >> > > >>>> >> >> >
> > > >> > > >>>> >> >>
> > > >> > > >>>> >> >> To be clear, I wasn't suggesting dropping either
> > > section. I
> > > >> > > agree
> > > >> > > >>>> that
> > > >> > > >>>> >> >> they add useful background. I was just suggesting
> that
> > we
> > > >> > should
> > > >> > > >>>> discuss
> > > >> > > >>>> >> >> the "acks" setting AFTER discussing the new high
> > > watermark
> > > >> > > >>>> advancement
> > > >> > > >>>> >> >> conditions. We also should discuss acks=0. While it
> > isn't
> > > >> > > >>>> conceptually
> > > >> > > >>>> >> much
> > > >> > > >>>> >> >> different than acks=1 here, its omission from this
> > > section
> > > >> is
> > > >> > > >>>> confusing.
> > > >> > > >>>> >> >>
> > > >> > > >>>> >> >> > Unclean recovery
> > > >> > > >>>> >> >> >
> > > >> > > >>>> >> >> > The plan is to replace the
> > > unclean.leader.election.enable
> > > >> > with
> > > >> > > >>>> >> >> > unclean.recovery.strategy. If the Unclean Recovery
> is
> > > >> > enabled
> > > >> > > >>>> then it
> > > >> > > >>>> >> >> deals
> > > >> > > >>>> >> >> > with the three options in the
> > > unclean.recovery.strategy.
> > > >> > > >>>> >> >> >
> > > >> > > >>>> >> >> >
> > > >> > > >>>> >> >> > Let’s refine the Unclean Recovery. We have already
> > > taken a
> > > >> > > lot of
> > > >> > > >>>> >> >> > suggestions and I hope to enhance the durability of
> > > Kafka
> > > >> to
> > > >> > > the
> > > >> > > >>>> next
> > > >> > > >>>> >> >> level
> > > >> > > >>>> >> >> > with this KIP.
> > > >> > > >>>> >> >>
> > > >> > > >>>> >> >> I am OK with doing the unclean leader recovery
> > > improvements
> > > >> in
> > > >> > > >>>> this KIP.
> > > >> > > >>>> >> >> However, I think we need to really work on the
> > > configuration
> > > >> > > >>>> settings.
> > > >> > > >>>> >> >>
> > > >> > > >>>> >> >> Configuration overrides are often quite messy. For
> > > example,
> > > >> > the
> > > >> > > >>>> cases
> > > >> > > >>>> >> >> where we have log.roll.hours and log.roll.segment.ms
> ,
> > > the
> > > >> > user
> > > >> > > >>>> has to
> > > >> > > >>>> >> >> remember which one takes precedence, and it is not
> > > obvious.
> > > >> > So,
> > > >> > > >>>> rather
> > > >> > > >>>> >> than
> > > >> > > >>>> >> >> creating a new configuration, why not add additional
> > > values
> > > >> to
> > > >> > > >>>> >> >> "unclean.leader.election.enable"? I think this will
> be
> > > >> simpler
> > > >> > > for
> > > >> > > >>>> >> people
> > > >> > > >>>> >> >> to understand, and simpler in the code as well.
> > > >> > > >>>> >> >>
> > > >> > > >>>> >> >> What if we continued to use
> > > "unclean.leader.election.enable"
> > > >> > but
> > > >> > > >>>> >> extended
> > > >> > > >>>> >> >> it so that it took a string? Then the string could
> have
> > > >> these
> > > >> > > >>>> values:
> > > >> > > >>>> >> >>
> > > >> > > >>>> >> >> never
> > > >> > > >>>> >> >>     never automatically do an unclean leader election
> > > under
> > > >> > any
> > > >> > > >>>> >> conditions
> > > >> > > >>>> >> >>
> > > >> > > >>>> >> >> false / default
> > > >> > > >>>> >> >>     only do an unclean leader election if there may
> be
> > > >> > possible
> > > >> > > >>>> data
> > > >> > > >>>> >> loss
> > > >> > > >>>> >> >>
> > > >> > > >>>> >> >> true / always
> > > >> > > >>>> >> >>     always do an unclean leader election if we can't
> > > >> > immediately
> > > >> > > >>>> elect a
> > > >> > > >>>> >> >> leader
> > > >> > > >>>> >> >>
> > > >> > > >>>> >> >> It's a bit awkward that false maps to default rather
> > > than to
> > > >> > > >>>> never. But
> > > >> > > >>>> >> >> this awkwardness exists if we use two different
> > > >> configuration
> > > >> > > keys
> > > >> > > >>>> as
> > > >> > > >>>> >> well.
> > > >> > > >>>> >> >> The reason for the awkwardness is that we simply
> don't
> > > want
> > > >> > most
> > > >> > > >>>> of the
> > > >> > > >>>> >> >> people currently setting
> > > >> unclean.leader.election.enable=false
> > > >> > to
> > > >> > > >>>> get the
> > > >> > > >>>> >> >> "never" behavior. We have to bite that bullet. Better
> > to
> > > be
> > > >> > > clear
> > > >> > > >>>> and
> > > >> > > >>>> >> >> explicit than hide it.
> > > >> > > >>>> >> >>
> > > >> > > >>>> >> >> Another thing that's a bit awkward is having two
> > > different
> > > >> > ways
> > > >> > > to
> > > >> > > >>>> do
> > > >> > > >>>> >> >> unclean leader election specified in the KIP. You
> > > descirbe
> > > >> two
> > > >> > > >>>> methods:
> > > >> > > >>>> >> the
> > > >> > > >>>> >> >> simple "choose the last leader" method, and the
> > "unclean
> > > >> > > recovery
> > > >> > > >>>> >> manager"
> > > >> > > >>>> >> >> method. I understand why you did it this way --
> "choose
> > > the
> > > >> > last
> > > >> > > >>>> >> leader" is
> > > >> > > >>>> >> >> simple, and will help us deliver an implementation
> > > quickly,
> > > >> > > while
> > > >> > > >>>> the
> > > >> > > >>>> >> URM
> > > >> > > >>>> >> >> is preferable in the long term. My suggestion here is
> > to
> > > >> > > separate
> > > >> > > >>>> the
> > > >> > > >>>> >> >> decision of HOW to do unclean leader election from
> the
> > > >> > decision
> > > >> > > of
> > > >> > > >>>> WHEN
> > > >> > > >>>> >> to
> > > >> > > >>>> >> >> do it.
> > > >> > > >>>> >> >>
> > > >> > > >>>> >> >> So in other words, have
> > "unclean.leader.election.enable"
> > > >> > specify
> > > >> > > >>>> when we
> > > >> > > >>>> >> >> do unclean leader election, and have a new
> > configuration
> > > >> like
> > > >> > > >>>> >> >> "unclean.recovery.manager.enable" to determine if we
> > use
> > > the
> > > >> > > URM.
> > > >> > > >>>> >> >> Presumably the URM will take some time to get fully
> > > stable,
> > > >> so
> > > >> > > >>>> this can
> > > >> > > >>>> >> >> default to false for a while, and we can flip the
> > > default to
> > > >> > > true
> > > >> > > >>>> when
> > > >> > > >>>> >> we
> > > >> > > >>>> >> >> feel ready.
> > > >> > > >>>> >> >>
> > > >> > > >>>> >> >> The URM is somewhat under-described here. I think we
> > > need a
> > > >> > few
> > > >> > > >>>> >> >> configurations here for it. For example, we need a
> > > >> > > configuration to
> > > >> > > >>>> >> specify
> > > >> > > >>>> >> >> how long it should wait for a broker to respond to
> its
> > > RPCs
> > > >> > > before
> > > >> > > >>>> >> moving
> > > >> > > >>>> >> >> on. We also need to understand how the URM interacts
> > with
> > > >> > > >>>> >> >> unclean.leader.election.enable=always. I assume that
> > with
> > > >> > > "always"
> > > >> > > >>>> we
> > > >> > > >>>> >> will
> > > >> > > >>>> >> >> just unconditionally use the URM rather than choosing
> > > >> > randomly.
> > > >> > > >>>> But this
> > > >> > > >>>> >> >> should be spelled out in the KIP.
> > > >> > > >>>> >> >>
> > > >> > > >>>> >> >> >
> > > >> > > >>>> >> >> > DescribeTopicRequest
> > > >> > > >>>> >> >> >
> > > >> > > >>>> >> >> >    1.
> > > >> > > >>>> >> >> >    Yes, the plan is to replace the MetadataRequest
> > with
> > > >> the
> > > >> > > >>>> >> >> >    DescribeTopicRequest for the admin clients. Will
> > > check
> > > >> > the
> > > >> > > >>>> details.
> > > >> > > >>>> >> >>
> > > >> > > >>>> >> >> Sounds good. But as I said, you need to specify how
> > > >> > AdminClient
> > > >> > > >>>> >> interacts
> > > >> > > >>>> >> >> with the new request. This will involve adding some
> > > fields
> > > >> to
> > > >> > > >>>> >> >> TopicDescription.java. And you need to specify the
> > > changes
> > > >> to
> > > >> > > the
> > > >> > > >>>> >> >> kafka-topics.sh command line tool. Otherwise we
> cannot
> > > use
> > > >> the
> > > >> > > >>>> tool to
> > > >> > > >>>> >> see
> > > >> > > >>>> >> >> the new information.
> > > >> > > >>>> >> >>
> > > >> > > >>>> >> >> The new requests, DescribeTopicRequest and
> > > >> > > >>>> GetReplicaLogInfoRequest,
> > > >> > > >>>> >> need
> > > >> > > >>>> >> >> to have limits placed on them so that their size
> can't
> > be
> > > >> > > >>>> infinite. We
> > > >> > > >>>> >> >> don't want to propagate the current problems of
> > > >> > MetadataRequest,
> > > >> > > >>>> where
> > > >> > > >>>> >> >> clients can request massive responses that can mess
> up
> > > the
> > > >> JVM
> > > >> > > when
> > > >> > > >>>> >> handled.
> > > >> > > >>>> >> >>
> > > >> > > >>>> >> >> Adding limits is simple for GetReplicaLogInfoRequest
> --
> > > we
> > > >> can
> > > >> > > >>>> just say
> > > >> > > >>>> >> >> that only 2000 partitions at a time can be requested.
> > For
> > > >> > > >>>> >> >> DescribeTopicRequest we can probably just limit to 20
> > > topics
> > > >> > or
> > > >> > > >>>> >> something
> > > >> > > >>>> >> >> like that, to avoid the complexity of doing
> pagination
> > in
> > > >> this
> > > >> > > KIP.
> > > >> > > >>>> >> >>
> > > >> > > >>>> >> >> >    2.
> > > >> > > >>>> >> >> >    I can let the broker load the ELR info so that
> > they
> > > can
> > > >> > > serve
> > > >> > > >>>> the
> > > >> > > >>>> >> >> >    DescribeTopicRequest as well.
> > > >> > > >>>> >> >>
> > > >> > > >>>> >> >> Yes, it's fine to add to MetadataCache. In fact,
> you'll
> > > be
> > > >> > > loading
> > > >> > > >>>> it
> > > >> > > >>>> >> >> anyway once it's added to PartitionImage.
> > > >> > > >>>> >> >>
> > > >> > > >>>> >> >> >    3.
> > > >> > > >>>> >> >> >    Yeah, it does not make sense to have the topic
> id
> > if
> > > >> > > >>>> >> >> >    DescribeTopicRequest is only used by the admin
> > > client.
> > > >> > > >>>> >> >>
> > > >> > > >>>> >> >> OK. That makes things simpler. We can always create a
> > new
> > > >> API
> > > >> > > later
> > > >> > > >>>> >> >> (hopefully not in this KIP!) to query by topic ID.
> > > >> > > >>>> >> >>
> > > >> > > >>>> >> >> >
> > > >> > > >>>> >> >> >
> > > >> > > >>>> >> >> > Metrics
> > > >> > > >>>> >> >> >
> > > >> > > >>>> >> >> > As for overall cluster health metrics, I think
> > > >> under-min-ISR
> > > >> > > is
> > > >> > > >>>> still
> > > >> > > >>>> >> a
> > > >> > > >>>> >> >> > useful one. ELR is more like a safety belt. When
> the
> > > ELR
> > > >> is
> > > >> > > >>>> used, the
> > > >> > > >>>> >> >> > cluster availability has already been impacted.
> > > >> > > >>>> >> >> >
> > > >> > > >>>> >> >> > Maybe we can have a metric to count the partitions
> > that
> > > >> > > sum(ISR,
> > > >> > > >>>> ELR)
> > > >> > > >>>> >> <
> > > >> > > >>>> >> >> min
> > > >> > > >>>> >> >> > ISR. What do you think?
> > > >> > > >>>> >> >>
> > > >> > > >>>> >> >> How about:
> > > >> > > >>>> >> >>
> > > >> > > >>>> >> >> A.  a metric for the totoal number of under-min-isr
> > > >> > partitions?
> > > >> > > We
> > > >> > > >>>> don't
> > > >> > > >>>> >> >> have that in Apache Kafka at the moment.
> > > >> > > >>>> >> >>
> > > >> > > >>>> >> >> B. a metric for the number of unclean leader
> elections
> > we
> > > >> did
> > > >> > > (for
> > > >> > > >>>> >> >> simplicity, it can reset to 0 on controller restart:
> we
> > > >> expect
> > > >> > > >>>> people to
> > > >> > > >>>> >> >> monitor the change over time anyway)
> > > >> > > >>>> >> >>
> > > >> > > >>>> >> >> best,
> > > >> > > >>>> >> >> Colin
> > > >> > > >>>> >> >>
> > > >> > > >>>> >> >>
> > > >> > > >>>> >> >> >
> > > >> > > >>>> >> >> > Yeah, for the ongoing unclean recoveries, the
> > > controller
> > > >> can
> > > >> > > >>>> keep an
> > > >> > > >>>> >> >> > accurate count through failover because partition
> > > >> > registration
> > > >> > > >>>> can
> > > >> > > >>>> >> >> indicate
> > > >> > > >>>> >> >> > whether a recovery is needed. However, for the
> > happened
> > > >> > ones,
> > > >> > > >>>> unless
> > > >> > > >>>> >> we
> > > >> > > >>>> >> >> > want to persist the number somewhere, we can only
> > > figure
> > > >> it
> > > >> > > out
> > > >> > > >>>> from
> > > >> > > >>>> >> the
> > > >> > > >>>> >> >> > log.
> > > >> > > >>>> >> >> >
> > > >> > > >>>> >> >> > On Tue, Sep 12, 2023 at 3:16 PM Colin McCabe <
> > > >> > > cmccabe@apache.org
> > > >> > > >>>> >
> > > >> > > >>>> >> wrote:
> > > >> > > >>>> >> >> >
> > > >> > > >>>> >> >> >> Also, we should have metrics that show what is
> going
> > > on
> > > >> > with
> > > >> > > >>>> regard
> > > >> > > >>>> >> to
> > > >> > > >>>> >> >> the
> > > >> > > >>>> >> >> >> eligible replica set. I'm not sure exactly what to
> > > >> suggest,
> > > >> > > but
> > > >> > > >>>> >> >> something
> > > >> > > >>>> >> >> >> that could identify when things are going wrong in
> > the
> > > >> > > clsuter.
> > > >> > > >>>> >> >> >>
> > > >> > > >>>> >> >> >> For example, maybe a metric for partitions
> > containing
> > > >> > > replicas
> > > >> > > >>>> that
> > > >> > > >>>> >> are
> > > >> > > >>>> >> >> >> ineligible to be leader? That would show a spike
> > when
> > > a
> > > >> > > broker
> > > >> > > >>>> had an
> > > >> > > >>>> >> >> >> unclean restart.
> > > >> > > >>>> >> >> >>
> > > >> > > >>>> >> >> >> Ideally, we'd also have a metric that indicates
> when
> > > an
> > > >> > > unclear
> > > >> > > >>>> >> leader
> > > >> > > >>>> >> >> >> election or a recovery happened. It's a bit tricky
> > > >> because
> > > >> > > the
> > > >> > > >>>> simple
> > > >> > > >>>> >> >> >> thing, of tracking it per controller, may be a bit
> > > >> > confusing
> > > >> > > >>>> during
> > > >> > > >>>> >> >> >> failovers.
> > > >> > > >>>> >> >> >>
> > > >> > > >>>> >> >> >> best,
> > > >> > > >>>> >> >> >> Colin
> > > >> > > >>>> >> >> >>
> > > >> > > >>>> >> >> >>
> > > >> > > >>>> >> >> >> On Tue, Sep 12, 2023, at 14:25, Colin McCabe
> wrote:
> > > >> > > >>>> >> >> >> > Hi Calvin,
> > > >> > > >>>> >> >> >> >
> > > >> > > >>>> >> >> >> > Thanks for the KIP. I think this is a great
> > > >> improvement.
> > > >> > > >>>> >> >> >> >
> > > >> > > >>>> >> >> >> >> Additional High Watermark advance requirement
> > > >> > > >>>> >> >> >> >
> > > >> > > >>>> >> >> >> > Typo: change "advance" to "advancement"
> > > >> > > >>>> >> >> >> >
> > > >> > > >>>> >> >> >> >> A bit recap of some key concepts.
> > > >> > > >>>> >> >> >> >
> > > >> > > >>>> >> >> >> > Typo: change "bit" to "quick"
> > > >> > > >>>> >> >> >> >
> > > >> > > >>>> >> >> >> >> Ack=1/all produce request. It defines when the
> > > Kafka
> > > >> > > server
> > > >> > > >>>> should
> > > >> > > >>>> >> >> >> respond to the produce request
> > > >> > > >>>> >> >> >> >
> > > >> > > >>>> >> >> >> > I think this section would be clearer if we
> talked
> > > >> about
> > > >> > > the
> > > >> > > >>>> new
> > > >> > > >>>> >> high
> > > >> > > >>>> >> >> >> > watermark advancement requirement first, and
> THEN
> > > >> talked
> > > >> > > >>>> about its
> > > >> > > >>>> >> >> >> > impact on acks=0, acks=1, and     acks=all.
> > > acks=all
> > > >> is
> > > >> > of
> > > >> > > >>>> course
> > > >> > > >>>> >> the
> > > >> > > >>>> >> >> >> > main case we care about here, so it would be
> good
> > to
> > > >> lead
> > > >> > > with
> > > >> > > >>>> >> that,
> > > >> > > >>>> >> >> >> > rather than delving into the technicalities of
> > > acks=0/1
> > > >> > > first.
> > > >> > > >>>> >> >> >> >
> > > >> > > >>>> >> >> >> >> Unclean recovery
> > > >> > > >>>> >> >> >> >
> > > >> > > >>>> >> >> >> > So, here you are introducing a new
> configuration,
> > > >> > > >>>> >> >> >> > unclean.recovery.strategy. The difficult thing
> > here
> > > is
> > > >> > that
> > > >> > > >>>> there
> > > >> > > >>>> >> is a
> > > >> > > >>>> >> >> >> > lot of overlap with
> > unclean.leader.election.enable.
> > > So
> > > >> we
> > > >> > > >>>> have 3
> > > >> > > >>>> >> >> >> > different settings for
> unclean.recovery.strategy,
> > > plus
> > > >> 2
> > > >> > > >>>> different
> > > >> > > >>>> >> >> >> > settings for unclean.leader.election.enable,
> > giving
> > > a
> > > >> > cross
> > > >> > > >>>> >> product of
> > > >> > > >>>> >> >> >> > 6 different options. The following "unclean
> > recovery
> > > >> > > manager"
> > > >> > > >>>> >> section
> > > >> > > >>>> >> >> >> > only applies to one fo those 6 different
> > > possibilities
> > > >> (I
> > > >> > > >>>> think?)
> > > >> > > >>>> >> >> >> >
> > > >> > > >>>> >> >> >> > I simply don't think we need so many different
> > > election
> > > >> > > types.
> > > >> > > >>>> >> Really
> > > >> > > >>>> >> >> >> > the use-cases we need are people who want NO
> > unclean
> > > >> > > >>>> elections,
> > > >> > > >>>> >> people
> > > >> > > >>>> >> >> >> > who want "the reasonable thing" and people who
> > want
> > > >> > > >>>> avaialbility at
> > > >> > > >>>> >> >> all
> > > >> > > >>>> >> >> >> > costs.
> > > >> > > >>>> >> >> >> >
> > > >> > > >>>> >> >> >> > Overall, I feel like the first half of the KIP
> is
> > > about
> > > >> > the
> > > >> > > >>>> ELR,
> > > >> > > >>>> >> and
> > > >> > > >>>> >> >> >> > the second half is about reworking unclean
> leader
> > > >> > > election. It
> > > >> > > >>>> >> might
> > > >> > > >>>> >> >> be
> > > >> > > >>>> >> >> >> > better to move that second half to a separate
> KIP
> > so
> > > >> that
> > > >> > > we
> > > >> > > >>>> can
> > > >> > > >>>> >> >> figure
> > > >> > > >>>> >> >> >> > it out fully. It should be fine to punt this
> until
> > > >> later
> > > >> > > and
> > > >> > > >>>> just
> > > >> > > >>>> >> have
> > > >> > > >>>> >> >> >> > the current behavior on empty ELR be waiting for
> > the
> > > >> last
> > > >> > > >>>> known
> > > >> > > >>>> >> leader
> > > >> > > >>>> >> >> >> > to return. After all, that's what we do today.
> > > >> > > >>>> >> >> >> >
> > > >> > > >>>> >> >> >> >> DescribeTopicRequest
> > > >> > > >>>> >> >> >> >
> > > >> > > >>>> >> >> >> > Is the intention for AdminClient to use this RPC
> > for
> > > >> > > >>>> >> >> >> > Admin#describeTopics ? If so, we need to
> describe
> > > all
> > > >> of
> > > >> > > the
> > > >> > > >>>> >> changes
> > > >> > > >>>> >> >> to
> > > >> > > >>>> >> >> >> > the admin client API, as well as changes to
> > > >> command-line
> > > >> > > >>>> tools like
> > > >> > > >>>> >> >> >> > kafka-topics.sh (if there are any). For example,
> > you
> > > >> will
> > > >> > > >>>> probably
> > > >> > > >>>> >> >> need
> > > >> > > >>>> >> >> >> > changes to TopicDescription.java. You will also
> > > need to
> > > >> > > >>>> provide
> > > >> > > >>>> >> all of
> > > >> > > >>>> >> >> >> > the things that admin client needs -- for
> example,
> > > >> > > >>>> >> >> >> > TopicAuthorizedOperations.
> > > >> > > >>>> >> >> >> >
> > > >> > > >>>> >> >> >> > I also don't think the controller should serve
> > this
> > > >> > > request.
> > > >> > > >>>> We
> > > >> > > >>>> >> want
> > > >> > > >>>> >> >> to
> > > >> > > >>>> >> >> >> > minimize load on the controller. Just like with
> > the
> > > >> other
> > > >> > > >>>> metadata
> > > >> > > >>>> >> >> >> > requests like MetadataRequest, this should be
> > > served by
> > > >> > > >>>> brokers.
> > > >> > > >>>> >> >> >> >
> > > >> > > >>>> >> >> >> > It's a bit confusing why both topic ID and topic
> > > name
> > > >> are
> > > >> > > >>>> provided
> > > >> > > >>>> >> to
> > > >> > > >>>> >> >> >> > this API. Is the intention that callers should
> set
> > > one
> > > >> > but
> > > >> > > >>>> not the
> > > >> > > >>>> >> >> >> > other? Or both? This needs to be clarified.
> Also,
> > > if we
> > > >> > do
> > > >> > > >>>> want to
> > > >> > > >>>> >> >> >> > support lookups by UUID, that is another thing
> > that
> > > >> needs
> > > >> > > to
> > > >> > > >>>> be
> > > >> > > >>>> >> added
> > > >> > > >>>> >> >> >> > to adminclient.
> > > >> > > >>>> >> >> >> >
> > > >> > > >>>> >> >> >> > In general, I feel like this should also
> probably
> > be
> > > >> its
> > > >> > > own
> > > >> > > >>>> KIP
> > > >> > > >>>> >> since
> > > >> > > >>>> >> >> >> > it's fairly complex
> > > >> > > >>>> >> >> >> >
> > > >> > > >>>> >> >> >> > best,
> > > >> > > >>>> >> >> >> > Colin
> > > >> > > >>>> >> >> >> >
> > > >> > > >>>> >> >> >> >
> > > >> > > >>>> >> >> >> > On Thu, Aug 10, 2023, at 15:46, Calvin Liu
> 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
> > > >> > > >>>> >> >> >>
> > > >> > > >>>> >> >>
> > > >> > > >>>> >>
> > > >> > > >>>>
> > > >> > > >>>
> > > >> > >
> > > >> >
> > > >>
> > >
> >
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

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

Thanks for the reply. Just one more comment.

54. It seems that downgrading MV is supported. Is downgrading the software
version supported? It would be useful to document that.

Thanks,

Jun

On Tue, Oct 3, 2023 at 4:55 PM Artem Livshits
<al...@confluent.io.invalid> wrote:

> Hi Colin,
>
> I think in your example "do_unclean_recovery" would need to do different
> things depending on the strategy.
>
> do_unclean_recovery() {
>    if (unclean.recovery.manager.enabled) {
>     if (strategy == Aggressive)
>       use UncleanRecoveryManager(waitLastKnownERL=false)  // just inspect
> logs from whoever is available
>     else
>       use  UncleanRecoveryManager(waitLastKnownERL=true)  // must wait for
> at least last known ELR
>   } else {
>     if (strategy == Aggressive)
>       choose the last known leader if that is available, or a random leader
> if not)
>     else
>       wait for last known leader to get back
>   }
> }
>
> The idea is that the Aggressive strategy would kick in as soon as we lost
> the leader and would pick a leader from whoever is available; but the
> Balanced will only kick in when ELR is empty and will wait for the brokers
> that likely have most data to be available.
>
> On Tue, Oct 3, 2023 at 3:04 PM Colin McCabe <cm...@apache.org> wrote:
>
> > On Tue, Oct 3, 2023, at 10:49, Jun Rao wrote:
> > > Hi, Calvin,
> > >
> > > Thanks for the update KIP. A few more comments.
> > >
> > > 41. Why would a user choose the option to select a random replica as
> the
> > > leader instead of using unclean.recovery.strateg=Aggressive? It seems
> > that
> > > the latter is strictly better? If that's not the case, could we fold
> this
> > > option under unclean.recovery.strategy instead of introducing a
> separate
> > > config?
> >
> > Hi Jun,
> >
> > I thought the flow of control was:
> >
> > If there is no leader for the partition {
> >   If (there are unfenced ELR members) {
> >     choose_an_unfenced_ELR_member
> >   } else if (there are fenced ELR members AND strategy=Aggressive) {
> >     do_unclean_recovery
> >   } else if (there are no ELR members AND strategy != None) {
> >     do_unclean_recovery
> >   } else {
> >     do nothing about the missing leader
> >   }
> > }
> >
> > do_unclean_recovery() {
> >    if (unclean.recovery.manager.enabled) {
> >     use UncleanRecoveryManager
> >   } else {
> >     choose the last known leader if that is available, or a random leader
> > if not)
> >   }
> > }
> >
> > However, I think this could be clarified, especially the behavior when
> > unclean.recovery.manager.enabled=false. Inuitively the goal for
> > unclean.recovery.manager.enabled=false is to be "the same as now, mostly"
> > but it's very underspecified in the KIP, I agree.
> >
> > >
> > > 50. ElectLeadersRequest: "If more than 20 topics are included, only the
> > > first 20 will be served. Others will be returned with DesiredLeaders."
> > Hmm,
> > > not sure that I understand this. ElectLeadersResponse doesn't have a
> > > DesiredLeaders field.
> > >
> > > 51. GetReplicaLogInfo: "If more than 2000 partitions are included, only
> > the
> > > first 2000 will be served" Do we return an error for the remaining
> > > partitions? Actually, should we include an errorCode field at the
> > partition
> > > level in GetReplicaLogInfoResponse to cover non-existing partitions and
> > no
> > > authorization, etc?
> > >
> > > 52. The entry should matches => The entry should match
> > >
> > > 53. ElectLeadersRequest.DesiredLeaders: Should it be nullable since a
> > user
> > > may not specify DesiredLeaders?
> > >
> > > 54. Downgrade: Is that indeed possible? I thought earlier you said that
> > > once the new version of the records are in the metadata log, one can't
> > > downgrade since the old broker doesn't know how to parse the new
> version
> > of
> > > the metadata records?
> > >
> >
> > MetadataVersion downgrade is currently broken but we have fixing it on
> our
> > plate for Kafka 3.7.
> >
> > The way downgrade works is that "new features" are dropped, leaving only
> > the old ones.
> >
> > > 55. CleanShutdownFile: Should we add a version field for future
> > extension?
> > >
> > > 56. Config changes are public facing. Could we have a separate section
> to
> > > document all the config changes?
> >
> > +1. A separate section for this would be good.
> >
> > best,
> > Colin
> >
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Mon, Sep 25, 2023 at 4:29 PM Calvin Liu <caliu@confluent.io.invalid
> >
> > > wrote:
> > >
> > >> Hi Jun
> > >> Thanks for the comments.
> > >>
> > >> 40. If we change to None, it is not guaranteed for no data loss. For
> > users
> > >> who are not able to validate the data with external resources, manual
> > >> intervention does not give a better result but a loss of availability.
> > So
> > >> practically speaking, the Balance mode would be a better default
> value.
> > >>
> > >> 41. No, it represents how we want to do the unclean leader election.
> If
> > it
> > >> is false, the unclean leader election will be the old random way.
> > >> Otherwise, the unclean recovery will be used.
> > >>
> > >> 42. Good catch. Updated.
> > >>
> > >> 43. Only the first 20 topics will be served. Others will be returned
> > with
> > >> InvalidRequestError
> > >>
> > >> 44. The order matters. The desired leader entries match with the topic
> > >> partition list by the index.
> > >>
> > >> 45. Thanks! Updated.
> > >>
> > >> 46. Good advice! Updated.
> > >>
> > >> 47.1, updated the comment. Basically it will elect the replica in the
> > >> desiredLeader field to be the leader
> > >>
> > >> 47.2 We can let the admin client do the conversion. Using the
> > desiredLeader
> > >> field in the json format seems easier for users.
> > >>
> > >> 48. Once the MV version is downgraded, all the ELR related fields will
> > be
> > >> removed on the next partition change. The controller will also ignore
> > the
> > >> ELR fields. Updated the KIP.
> > >>
> > >> 49. Yes, it would be deprecated/removed.
> > >>
> > >>
> > >> On Mon, Sep 25, 2023 at 3:49 PM Jun Rao <ju...@confluent.io.invalid>
> > wrote:
> > >>
> > >> > Hi, Calvin,
> > >> >
> > >> > Thanks for the updated KIP. Made another pass. A few more comments
> > below.
> > >> >
> > >> > 40. unclean.leader.election.enable.false ->
> > >> > unclean.recovery.strategy.Balanced: The Balanced mode could still
> > lead to
> > >> > data loss. So, I am wondering if
> unclean.leader.election.enable.false
> > >> > should map to None?
> > >> >
> > >> > 41. unclean.recovery.manager.enabled: I am not sure why we introduce
> > this
> > >> > additional config. Is it the same as unclean.recovery.strategy=None?
> > >> >
> > >> > 42. DescribeTopicResponse.TopicAuthorizedOperations: Should this be
> at
> > >> the
> > >> > topic level?
> > >> >
> > >> > 43. "Limit: 20 topics max per request": Could we describe what
> > happens if
> > >> > the request includes more than 20 topics?
> > >> >
> > >> > 44. ElectLeadersRequest.DesiredLeaders: Could we describe whether
> the
> > >> > ordering matters?
> > >> >
> > >> > 45. GetReplicaLogInfo.TopicPartitions: "about": "The topic
> partitions
> > to
> > >> > elect leaders.": The description in "about" is incorrect.
> > >> >
> > >> > 46. GetReplicaLogInfoResponse: Should we nest partitions under
> > topicId to
> > >> > be consistent with other types of responses?
> > >> >
> > >> > 47. kafka-leader-election.sh:
> > >> > 47.1 Could we explain DESIGNATION?
> > >> > 47.2 desiredLeader: Should it be a list to match the field in
> > >> > ElectLeadersRequest?
> > >> >
> > >> > 48. We could add a section on downgrade?
> > >> >
> > >> > 49. LastKnownLeader: This seems only needed in the first phase of
> > >> > delivering ELR. Will it be removed when the complete KIP is
> delivered?
> > >> >
> > >> > Thanks,
> > >> >
> > >> > Jun
> > >> >
> > >> > On Tue, Sep 19, 2023 at 1:30 PM Colin McCabe <cm...@apache.org>
> > wrote:
> > >> >
> > >> > > Hi Calvin,
> > >> > >
> > >> > > Thanks for the explanations. I like the idea of using none,
> > balanced,
> > >> > > aggressive. We also had an offline discussion about why it is good
> > to
> > >> > use a
> > >> > > new config key (basically, so that we can deprecate the old one
> > which
> > >> had
> > >> > > only false/true values in 4.0) With these changes, I am +1.
> > >> > >
> > >> > > best,
> > >> > > Colin
> > >> > >
> > >> > > On Mon, Sep 18, 2023, at 15:54, Calvin Liu wrote:
> > >> > > > Hi Colin,
> > >> > > > Also, can we deprecate unclean.leader.election.enable in 4.0?
> > Before
> > >> > > that,
> > >> > > > we can have both the config unclean.recovery.strategy and
> > >> > > > unclean.leader.election.enable
> > >> > > > and using the unclean.recovery.Enabled to determine which config
> > to
> > >> use
> > >> > > > during the unclean leader election.
> > >> > > >
> > >> > > > On Mon, Sep 18, 2023 at 3:51 PM Calvin Liu <ca...@confluent.io>
> > >> wrote:
> > >> > > >
> > >> > > >> Hi Colin,
> > >> > > >> For the unclean.recovery.strategy config name, how about we use
> > the
> > >> > > >> following
> > >> > > >> None. It basically means no unclean recovery will be performed.
> > >> > > >> Aggressive. It means availability goes first. Whenever the
> > partition
> > >> > > can't
> > >> > > >> elect a durable replica, the controller will try the unclean
> > >> recovery.
> > >> > > >> Balanced. It is the balance point of the availability
> > >> > first(Aggressive)
> > >> > > >> and least availability(None). The controller performs unclean
> > >> recovery
> > >> > > when
> > >> > > >> both ISR and ELR are empty.
> > >> > > >>
> > >> > > >>
> > >> > > >> On Fri, Sep 15, 2023 at 11:42 AM Calvin Liu <
> caliu@confluent.io>
> > >> > wrote:
> > >> > > >>
> > >> > > >>> Hi Colin,
> > >> > > >>>
> > >> > > >>> > So, the proposal is that if someone sets
> > >> > > "unclean.leader.election.enable
> > >> > > >>> = true"...
> > >> > > >>>
> > >> > > >>>
> > >> > > >>> The idea is to use one of the unclean.leader.election.enable
> and
> > >> > > >>> unclean.recovery.strategy based on the
> > unclean.recovery.Enabled. A
> > >> > > possible
> > >> > > >>> version can be
> > >> > > >>>
> > >> > > >>> If unclean.recovery.Enabled:
> > >> > > >>>
> > >> > > >>> {
> > >> > > >>>
> > >> > > >>> Check unclean.recovery.strategy. If set, use it. Otherwise,
> > check
> > >> > > >>> unclean.leader.election.enable and translate it to
> > >> > > >>> unclean.recovery.strategy.
> > >> > > >>>
> > >> > > >>> } else {
> > >> > > >>>
> > >> > > >>> Use unclean.leader.election.enable
> > >> > > >>>
> > >> > > >>> }
> > >> > > >>>
> > >> > > >>>
> > >> > > >>> —--------
> > >> > > >>>
> > >> > > >>> >The configuration key should be
> > >> "unclean.recovery.manager.enabled",
> > >> > > >>> right?
> > >> > > >>>
> > >> > > >>>
> > >> > > >>> I think we have two ways of choosing a leader uncleanly,
> unclean
> > >> > leader
> > >> > > >>> election and unclean recovery(log inspection) and we try to
> > switch
> > >> > > between
> > >> > > >>> them.
> > >> > > >>>
> > >> > > >>> Do you mean we want to develop two ways of performing the
> > unclean
> > >> > > >>> recovery and one of them is using “unclean recovery manager”?
> I
> > >> guess
> > >> > > we
> > >> > > >>> haven’t discussed the second way.
> > >> > > >>>
> > >> > > >>>
> > >> > > >>> —-------
> > >> > > >>>
> > >> > > >>> >How do these 4 levels of overrides interact with your new
> > >> > > >>> configurations?
> > >> > > >>>
> > >> > > >>>
> > >> > > >>> I do notice in the Kraft controller code, the method to check
> > >> whether
> > >> > > >>> perform unclean leader election is hard coded to false since
> > >> > > >>> 2021(uncleanLeaderElectionEnabledForTopic). Isn’t it a good
> > chance
> > >> to
> > >> > > >>> completely deprecate the unclean.leader.election.enable? We
> > don’t
> > >> > even
> > >> > > have
> > >> > > >>> to worry about the config conversion.
> > >> > > >>>
> > >> > > >>> On the other hand, whatever the override is, as long as the
> > >> > controller
> > >> > > >>> can have the final effective unclean.leader.election.enable,
> the
> > >> > topic
> > >> > > >>> level config unclean.recovery.strategy, the cluster level
> config
> > >> > > >>> unclean.recovery.Enabled, the controller can calculate the
> > correct
> > >> > > methods
> > >> > > >>> to use right?
> > >> > > >>>
> > >> > > >>>
> > >> > > >>> On Fri, Sep 15, 2023 at 10:02 AM Colin McCabe <
> > cmccabe@apache.org>
> > >> > > wrote:
> > >> > > >>>
> > >> > > >>>> On Thu, Sep 14, 2023, at 22:23, Calvin Liu wrote:
> > >> > > >>>> > Hi Colin
> > >> > > >>>> > 1. I think using the new config name is more clear.
> > >> > > >>>> >        a. The unclean leader election is actually removed
> if
> > >> > unclean
> > >> > > >>>> > recovery is in use.
> > >> > > >>>> >        b. Using multiple values in
> > >> unclean.leader.election.enable
> > >> > is
> > >> > > >>>> > confusing and it will be more confusing after people forget
> > >> about
> > >> > > this
> > >> > > >>>> > discussion.
> > >> > > >>>>
> > >> > > >>>> Hi Calvin,
> > >> > > >>>>
> > >> > > >>>> So, the proposal is that if someone sets
> > >> > > "unclean.leader.election.enable
> > >> > > >>>> = true" but then sets one of your new configurations, the
> > value of
> > >> > > >>>> unclean.leader.election.enable is ignored? That seems less
> > clear
> > >> to
> > >> > > me, not
> > >> > > >>>> more. Just in general, having multiple configuration keys to
> > >> control
> > >> > > the
> > >> > > >>>> same thing confuses users. Basically, they are sitting at a
> > giant
> > >> > > control
> > >> > > >>>> panel, and some of the levers do nothing.
> > >> > > >>>>
> > >> > > >>>> > 2. Sorry I forgot to mention in the response that I did add
> > the
> > >> > > >>>> > unclean.recovery.Enabled flag.
> > >> > > >>>>
> > >> > > >>>> The configuration key should be
> > >> "unclean.recovery.manager.enabled",
> > >> > > >>>> right? Becuase we can do "unclean recovery" without the
> > manager.
> > >> > > Disabling
> > >> > > >>>> the manager just means we use a different mechanism for
> > recovery.
> > >> > > >>>>
> > >> > > >>>> >        c. Maybe I underestimated the challenge of replacing
> > the
> > >> > > >>>> config. Any
> > >> > > >>>> > implementation problems ahead?
> > >> > > >>>>
> > >> > > >>>> There are four levels of overrides for
> > >> > unclean.leader.election.enable.
> > >> > > >>>>
> > >> > > >>>> 1. static configuration for node.
> > >> > > >>>>     This goes in the configuration file, typically named
> > >> > > >>>> server.properties
> > >> > > >>>>
> > >> > > >>>> 2. dynamic configuration for node default
> > >> > > >>>>   ConfigResource(type=BROKER, name="")
> > >> > > >>>>
> > >> > > >>>> 3. dynamic configuration for node
> > >> > > >>>>   ConfigResource(type=BROKER, name=<controller id>)
> > >> > > >>>>
> > >> > > >>>> 4. dynamic configuration for topic
> > >> > > >>>>   ConfigResource(type=TOPIC, name=<topic-name>)
> > >> > > >>>>
> > >> > > >>>> How do these 4 levels of overrides interact with your new
> > >> > > >>>> configurations? If the new configurations dominate over the
> old
> > >> > ones,
> > >> > > it
> > >> > > >>>> seems like this will get a lot more confusing to implement
> (and
> > >> also
> > >> > > to
> > >> > > >>>> use.)
> > >> > > >>>>
> > >> > > >>>> Again, I'd recommend just adding some new values to
> > >> > > >>>> unclean.leader.election.enable. It's simple and will prevent
> > user
> > >> > > confusion
> > >> > > >>>> (as well as developer confusion.)
> > >> > > >>>>
> > >> > > >>>> best,
> > >> > > >>>> Colin
> > >> > > >>>>
> > >> > > >>>>
> > >> > > >>>> > 3. About the admin client, I mentioned 3 changes in the
> > client.
> > >> > > >>>> Anything
> > >> > > >>>> > else I missed in the KIP?
> > >> > > >>>> >       a. The client will switch to using the new RPC
> instead
> > of
> > >> > > >>>> > MetadataRequest for the topics.
> > >> > > >>>> >       b. The TopicPartitionInfo used in TopicDescription
> > needs
> > >> to
> > >> > > add
> > >> > > >>>> new
> > >> > > >>>> > fields related to the ELR.
> > >> > > >>>> >       c. The outputs will add the ELR related fields.
> > >> > > >>>> >
> > >> > > >>>> > On Thu, Sep 14, 2023 at 9:19 PM Colin McCabe <
> > >> cmccabe@apache.org>
> > >> > > >>>> wrote:
> > >> > > >>>> >
> > >> > > >>>> >> Hi Calvin,
> > >> > > >>>> >>
> > >> > > >>>> >> Thanks for the changes.
> > >> > > >>>> >>
> > >> > > >>>> >> 1. Earlier I commented that creating
> > >> "unclean.recovery.strategy "
> > >> > > is
> > >> > > >>>> not
> > >> > > >>>> >> necessary, and we can just reuse the existing
> > >> > > >>>> >> "unclean.leader.election.enable" configuration key. Let's
> > >> discuss
> > >> > > >>>> that.
> > >> > > >>>> >>
> > >> > > >>>> >> 2.I also don't understand why you didn't add a
> > configuration to
> > >> > > >>>> enable or
> > >> > > >>>> >> disable the Unclean Recovery Manager. This seems like a
> very
> > >> > simple
> > >> > > >>>> way to
> > >> > > >>>> >> handle the staging issue which we discussed. The URM can
> > just
> > >> be
> > >> > > >>>> turned off
> > >> > > >>>> >> until it is production ready. Let's discuss this.
> > >> > > >>>> >>
> > >> > > >>>> >> 3. You still need to describe the changes to AdminClient
> > that
> > >> are
> > >> > > >>>> needed
> > >> > > >>>> >> to use DescribeTopicRequest.
> > >> > > >>>> >>
> > >> > > >>>> >> Keep at it. It's looking better. :)
> > >> > > >>>> >>
> > >> > > >>>> >> best,
> > >> > > >>>> >> Colin
> > >> > > >>>> >>
> > >> > > >>>> >>
> > >> > > >>>> >> On Thu, Sep 14, 2023, at 11:03, Calvin Liu wrote:
> > >> > > >>>> >> > Hi Colin
> > >> > > >>>> >> > Thanks for the comments!
> > >> > > >>>> >> >
> > >> > > >>>> >> > I did the following changes
> > >> > > >>>> >> >
> > >> > > >>>> >> >    1.
> > >> > > >>>> >> >
> > >> > > >>>> >> >    Simplified the API spec section to only include the
> > diff.
> > >> > > >>>> >> >    2.
> > >> > > >>>> >> >
> > >> > > >>>> >> >    Reordered the HWM requirement section.
> > >> > > >>>> >> >    3.
> > >> > > >>>> >> >
> > >> > > >>>> >> >    Removed the URM implementation details to keep the
> > >> necessary
> > >> > > >>>> >> >    characteristics to perform the unclean recovery.
> > >> > > >>>> >> >    1.
> > >> > > >>>> >> >
> > >> > > >>>> >> >       When to perform the unclean recovery
> > >> > > >>>> >> >       2.
> > >> > > >>>> >> >
> > >> > > >>>> >> >       Under different config, how the unclean recovery
> > finds
> > >> > the
> > >> > > >>>> leader.
> > >> > > >>>> >> >       3.
> > >> > > >>>> >> >
> > >> > > >>>> >> >       How the config unclean.leader.election.enable and
> > >> > > >>>> >> >       unclean.recovery.strategy are converted when users
> > >> > > >>>> enable/disable
> > >> > > >>>> >> the
> > >> > > >>>> >> >       unclean recovery.
> > >> > > >>>> >> >       4.
> > >> > > >>>> >> >
> > >> > > >>>> >> >    More details about how we change admin client.
> > >> > > >>>> >> >    5.
> > >> > > >>>> >> >
> > >> > > >>>> >> >    API limits on the GetReplicaLogInfoRequest and
> > >> > > >>>> DescribeTopicRequest.
> > >> > > >>>> >> >    6.
> > >> > > >>>> >> >
> > >> > > >>>> >> >    Two metrics added
> > >> > > >>>> >> >    1.
> > >> > > >>>> >> >
> > >> > > >>>> >> >
> >  Kafka.controller.global_under_min_isr_partition_count
> > >> > > >>>> >> >       2.
> > >> > > >>>> >> >
> > >> > > >>>> >> >       kafka.controller.unclean_recovery_finished_count
> > >> > > >>>> >> >
> > >> > > >>>> >> >
> > >> > > >>>> >> > On Wed, Sep 13, 2023 at 10:46 AM Colin McCabe <
> > >> > > cmccabe@apache.org>
> > >> > > >>>> >> wrote:
> > >> > > >>>> >> >
> > >> > > >>>> >> >> On Tue, Sep 12, 2023, at 17:21, Calvin Liu wrote:
> > >> > > >>>> >> >> > Hi Colin
> > >> > > >>>> >> >> > Thanks for the comments!
> > >> > > >>>> >> >> >
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> Hi Calvin,
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> Thanks again for the KIP.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> One meta-comment: it's usually better to just do a diff
> > on a
> > >> > > >>>> message
> > >> > > >>>> >> spec
> > >> > > >>>> >> >> file or java file if you're including changes to it in
> > the
> > >> > KIP.
> > >> > > >>>> This is
> > >> > > >>>> >> >> easier to read than looking for "new fields begin" etc.
> > in
> > >> the
> > >> > > >>>> text, and
> > >> > > >>>> >> >> gracefully handles the case where existing fields were
> > >> > changed.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> > Rewrite the Additional High Watermark advancement
> > >> > requirement
> > >> > > >>>> >> >> > There was feedback on this section that some readers
> > may
> > >> not
> > >> > > be
> > >> > > >>>> >> familiar
> > >> > > >>>> >> >> > with HWM and Ack=0,1,all requests. This can help them
> > >> > > understand
> > >> > > >>>> the
> > >> > > >>>> >> >> > proposal. I will rewrite this part for more
> > readability.
> > >> > > >>>> >> >> >
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> To be clear, I wasn't suggesting dropping either
> > section. I
> > >> > > agree
> > >> > > >>>> that
> > >> > > >>>> >> >> they add useful background. I was just suggesting that
> we
> > >> > should
> > >> > > >>>> discuss
> > >> > > >>>> >> >> the "acks" setting AFTER discussing the new high
> > watermark
> > >> > > >>>> advancement
> > >> > > >>>> >> >> conditions. We also should discuss acks=0. While it
> isn't
> > >> > > >>>> conceptually
> > >> > > >>>> >> much
> > >> > > >>>> >> >> different than acks=1 here, its omission from this
> > section
> > >> is
> > >> > > >>>> confusing.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> > Unclean recovery
> > >> > > >>>> >> >> >
> > >> > > >>>> >> >> > The plan is to replace the
> > unclean.leader.election.enable
> > >> > with
> > >> > > >>>> >> >> > unclean.recovery.strategy. If the Unclean Recovery is
> > >> > enabled
> > >> > > >>>> then it
> > >> > > >>>> >> >> deals
> > >> > > >>>> >> >> > with the three options in the
> > unclean.recovery.strategy.
> > >> > > >>>> >> >> >
> > >> > > >>>> >> >> >
> > >> > > >>>> >> >> > Let’s refine the Unclean Recovery. We have already
> > taken a
> > >> > > lot of
> > >> > > >>>> >> >> > suggestions and I hope to enhance the durability of
> > Kafka
> > >> to
> > >> > > the
> > >> > > >>>> next
> > >> > > >>>> >> >> level
> > >> > > >>>> >> >> > with this KIP.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> I am OK with doing the unclean leader recovery
> > improvements
> > >> in
> > >> > > >>>> this KIP.
> > >> > > >>>> >> >> However, I think we need to really work on the
> > configuration
> > >> > > >>>> settings.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> Configuration overrides are often quite messy. For
> > example,
> > >> > the
> > >> > > >>>> cases
> > >> > > >>>> >> >> where we have log.roll.hours and log.roll.segment.ms,
> > the
> > >> > user
> > >> > > >>>> has to
> > >> > > >>>> >> >> remember which one takes precedence, and it is not
> > obvious.
> > >> > So,
> > >> > > >>>> rather
> > >> > > >>>> >> than
> > >> > > >>>> >> >> creating a new configuration, why not add additional
> > values
> > >> to
> > >> > > >>>> >> >> "unclean.leader.election.enable"? I think this will be
> > >> simpler
> > >> > > for
> > >> > > >>>> >> people
> > >> > > >>>> >> >> to understand, and simpler in the code as well.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> What if we continued to use
> > "unclean.leader.election.enable"
> > >> > but
> > >> > > >>>> >> extended
> > >> > > >>>> >> >> it so that it took a string? Then the string could have
> > >> these
> > >> > > >>>> values:
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> never
> > >> > > >>>> >> >>     never automatically do an unclean leader election
> > under
> > >> > any
> > >> > > >>>> >> conditions
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> false / default
> > >> > > >>>> >> >>     only do an unclean leader election if there may be
> > >> > possible
> > >> > > >>>> data
> > >> > > >>>> >> loss
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> true / always
> > >> > > >>>> >> >>     always do an unclean leader election if we can't
> > >> > immediately
> > >> > > >>>> elect a
> > >> > > >>>> >> >> leader
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> It's a bit awkward that false maps to default rather
> > than to
> > >> > > >>>> never. But
> > >> > > >>>> >> >> this awkwardness exists if we use two different
> > >> configuration
> > >> > > keys
> > >> > > >>>> as
> > >> > > >>>> >> well.
> > >> > > >>>> >> >> The reason for the awkwardness is that we simply don't
> > want
> > >> > most
> > >> > > >>>> of the
> > >> > > >>>> >> >> people currently setting
> > >> unclean.leader.election.enable=false
> > >> > to
> > >> > > >>>> get the
> > >> > > >>>> >> >> "never" behavior. We have to bite that bullet. Better
> to
> > be
> > >> > > clear
> > >> > > >>>> and
> > >> > > >>>> >> >> explicit than hide it.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> Another thing that's a bit awkward is having two
> > different
> > >> > ways
> > >> > > to
> > >> > > >>>> do
> > >> > > >>>> >> >> unclean leader election specified in the KIP. You
> > descirbe
> > >> two
> > >> > > >>>> methods:
> > >> > > >>>> >> the
> > >> > > >>>> >> >> simple "choose the last leader" method, and the
> "unclean
> > >> > > recovery
> > >> > > >>>> >> manager"
> > >> > > >>>> >> >> method. I understand why you did it this way -- "choose
> > the
> > >> > last
> > >> > > >>>> >> leader" is
> > >> > > >>>> >> >> simple, and will help us deliver an implementation
> > quickly,
> > >> > > while
> > >> > > >>>> the
> > >> > > >>>> >> URM
> > >> > > >>>> >> >> is preferable in the long term. My suggestion here is
> to
> > >> > > separate
> > >> > > >>>> the
> > >> > > >>>> >> >> decision of HOW to do unclean leader election from the
> > >> > decision
> > >> > > of
> > >> > > >>>> WHEN
> > >> > > >>>> >> to
> > >> > > >>>> >> >> do it.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> So in other words, have
> "unclean.leader.election.enable"
> > >> > specify
> > >> > > >>>> when we
> > >> > > >>>> >> >> do unclean leader election, and have a new
> configuration
> > >> like
> > >> > > >>>> >> >> "unclean.recovery.manager.enable" to determine if we
> use
> > the
> > >> > > URM.
> > >> > > >>>> >> >> Presumably the URM will take some time to get fully
> > stable,
> > >> so
> > >> > > >>>> this can
> > >> > > >>>> >> >> default to false for a while, and we can flip the
> > default to
> > >> > > true
> > >> > > >>>> when
> > >> > > >>>> >> we
> > >> > > >>>> >> >> feel ready.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> The URM is somewhat under-described here. I think we
> > need a
> > >> > few
> > >> > > >>>> >> >> configurations here for it. For example, we need a
> > >> > > configuration to
> > >> > > >>>> >> specify
> > >> > > >>>> >> >> how long it should wait for a broker to respond to its
> > RPCs
> > >> > > before
> > >> > > >>>> >> moving
> > >> > > >>>> >> >> on. We also need to understand how the URM interacts
> with
> > >> > > >>>> >> >> unclean.leader.election.enable=always. I assume that
> with
> > >> > > "always"
> > >> > > >>>> we
> > >> > > >>>> >> will
> > >> > > >>>> >> >> just unconditionally use the URM rather than choosing
> > >> > randomly.
> > >> > > >>>> But this
> > >> > > >>>> >> >> should be spelled out in the KIP.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> >
> > >> > > >>>> >> >> > DescribeTopicRequest
> > >> > > >>>> >> >> >
> > >> > > >>>> >> >> >    1.
> > >> > > >>>> >> >> >    Yes, the plan is to replace the MetadataRequest
> with
> > >> the
> > >> > > >>>> >> >> >    DescribeTopicRequest for the admin clients. Will
> > check
> > >> > the
> > >> > > >>>> details.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> Sounds good. But as I said, you need to specify how
> > >> > AdminClient
> > >> > > >>>> >> interacts
> > >> > > >>>> >> >> with the new request. This will involve adding some
> > fields
> > >> to
> > >> > > >>>> >> >> TopicDescription.java. And you need to specify the
> > changes
> > >> to
> > >> > > the
> > >> > > >>>> >> >> kafka-topics.sh command line tool. Otherwise we cannot
> > use
> > >> the
> > >> > > >>>> tool to
> > >> > > >>>> >> see
> > >> > > >>>> >> >> the new information.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> The new requests, DescribeTopicRequest and
> > >> > > >>>> GetReplicaLogInfoRequest,
> > >> > > >>>> >> need
> > >> > > >>>> >> >> to have limits placed on them so that their size can't
> be
> > >> > > >>>> infinite. We
> > >> > > >>>> >> >> don't want to propagate the current problems of
> > >> > MetadataRequest,
> > >> > > >>>> where
> > >> > > >>>> >> >> clients can request massive responses that can mess up
> > the
> > >> JVM
> > >> > > when
> > >> > > >>>> >> handled.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> Adding limits is simple for GetReplicaLogInfoRequest --
> > we
> > >> can
> > >> > > >>>> just say
> > >> > > >>>> >> >> that only 2000 partitions at a time can be requested.
> For
> > >> > > >>>> >> >> DescribeTopicRequest we can probably just limit to 20
> > topics
> > >> > or
> > >> > > >>>> >> something
> > >> > > >>>> >> >> like that, to avoid the complexity of doing pagination
> in
> > >> this
> > >> > > KIP.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> >    2.
> > >> > > >>>> >> >> >    I can let the broker load the ELR info so that
> they
> > can
> > >> > > serve
> > >> > > >>>> the
> > >> > > >>>> >> >> >    DescribeTopicRequest as well.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> Yes, it's fine to add to MetadataCache. In fact, you'll
> > be
> > >> > > loading
> > >> > > >>>> it
> > >> > > >>>> >> >> anyway once it's added to PartitionImage.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> >    3.
> > >> > > >>>> >> >> >    Yeah, it does not make sense to have the topic id
> if
> > >> > > >>>> >> >> >    DescribeTopicRequest is only used by the admin
> > client.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> OK. That makes things simpler. We can always create a
> new
> > >> API
> > >> > > later
> > >> > > >>>> >> >> (hopefully not in this KIP!) to query by topic ID.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> >
> > >> > > >>>> >> >> >
> > >> > > >>>> >> >> > Metrics
> > >> > > >>>> >> >> >
> > >> > > >>>> >> >> > As for overall cluster health metrics, I think
> > >> under-min-ISR
> > >> > > is
> > >> > > >>>> still
> > >> > > >>>> >> a
> > >> > > >>>> >> >> > useful one. ELR is more like a safety belt. When the
> > ELR
> > >> is
> > >> > > >>>> used, the
> > >> > > >>>> >> >> > cluster availability has already been impacted.
> > >> > > >>>> >> >> >
> > >> > > >>>> >> >> > Maybe we can have a metric to count the partitions
> that
> > >> > > sum(ISR,
> > >> > > >>>> ELR)
> > >> > > >>>> >> <
> > >> > > >>>> >> >> min
> > >> > > >>>> >> >> > ISR. What do you think?
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> How about:
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> A.  a metric for the totoal number of under-min-isr
> > >> > partitions?
> > >> > > We
> > >> > > >>>> don't
> > >> > > >>>> >> >> have that in Apache Kafka at the moment.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> B. a metric for the number of unclean leader elections
> we
> > >> did
> > >> > > (for
> > >> > > >>>> >> >> simplicity, it can reset to 0 on controller restart: we
> > >> expect
> > >> > > >>>> people to
> > >> > > >>>> >> >> monitor the change over time anyway)
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> best,
> > >> > > >>>> >> >> Colin
> > >> > > >>>> >> >>
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> >
> > >> > > >>>> >> >> > Yeah, for the ongoing unclean recoveries, the
> > controller
> > >> can
> > >> > > >>>> keep an
> > >> > > >>>> >> >> > accurate count through failover because partition
> > >> > registration
> > >> > > >>>> can
> > >> > > >>>> >> >> indicate
> > >> > > >>>> >> >> > whether a recovery is needed. However, for the
> happened
> > >> > ones,
> > >> > > >>>> unless
> > >> > > >>>> >> we
> > >> > > >>>> >> >> > want to persist the number somewhere, we can only
> > figure
> > >> it
> > >> > > out
> > >> > > >>>> from
> > >> > > >>>> >> the
> > >> > > >>>> >> >> > log.
> > >> > > >>>> >> >> >
> > >> > > >>>> >> >> > On Tue, Sep 12, 2023 at 3:16 PM Colin McCabe <
> > >> > > cmccabe@apache.org
> > >> > > >>>> >
> > >> > > >>>> >> wrote:
> > >> > > >>>> >> >> >
> > >> > > >>>> >> >> >> Also, we should have metrics that show what is going
> > on
> > >> > with
> > >> > > >>>> regard
> > >> > > >>>> >> to
> > >> > > >>>> >> >> the
> > >> > > >>>> >> >> >> eligible replica set. I'm not sure exactly what to
> > >> suggest,
> > >> > > but
> > >> > > >>>> >> >> something
> > >> > > >>>> >> >> >> that could identify when things are going wrong in
> the
> > >> > > clsuter.
> > >> > > >>>> >> >> >>
> > >> > > >>>> >> >> >> For example, maybe a metric for partitions
> containing
> > >> > > replicas
> > >> > > >>>> that
> > >> > > >>>> >> are
> > >> > > >>>> >> >> >> ineligible to be leader? That would show a spike
> when
> > a
> > >> > > broker
> > >> > > >>>> had an
> > >> > > >>>> >> >> >> unclean restart.
> > >> > > >>>> >> >> >>
> > >> > > >>>> >> >> >> Ideally, we'd also have a metric that indicates when
> > an
> > >> > > unclear
> > >> > > >>>> >> leader
> > >> > > >>>> >> >> >> election or a recovery happened. It's a bit tricky
> > >> because
> > >> > > the
> > >> > > >>>> simple
> > >> > > >>>> >> >> >> thing, of tracking it per controller, may be a bit
> > >> > confusing
> > >> > > >>>> during
> > >> > > >>>> >> >> >> failovers.
> > >> > > >>>> >> >> >>
> > >> > > >>>> >> >> >> best,
> > >> > > >>>> >> >> >> Colin
> > >> > > >>>> >> >> >>
> > >> > > >>>> >> >> >>
> > >> > > >>>> >> >> >> On Tue, Sep 12, 2023, at 14:25, Colin McCabe wrote:
> > >> > > >>>> >> >> >> > Hi Calvin,
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> > Thanks for the KIP. I think this is a great
> > >> improvement.
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> >> Additional High Watermark advance requirement
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> > Typo: change "advance" to "advancement"
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> >> A bit recap of some key concepts.
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> > Typo: change "bit" to "quick"
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> >> Ack=1/all produce request. It defines when the
> > Kafka
> > >> > > server
> > >> > > >>>> should
> > >> > > >>>> >> >> >> respond to the produce request
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> > I think this section would be clearer if we talked
> > >> about
> > >> > > the
> > >> > > >>>> new
> > >> > > >>>> >> high
> > >> > > >>>> >> >> >> > watermark advancement requirement first, and THEN
> > >> talked
> > >> > > >>>> about its
> > >> > > >>>> >> >> >> > impact on acks=0, acks=1, and     acks=all.
> > acks=all
> > >> is
> > >> > of
> > >> > > >>>> course
> > >> > > >>>> >> the
> > >> > > >>>> >> >> >> > main case we care about here, so it would be good
> to
> > >> lead
> > >> > > with
> > >> > > >>>> >> that,
> > >> > > >>>> >> >> >> > rather than delving into the technicalities of
> > acks=0/1
> > >> > > first.
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> >> Unclean recovery
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> > So, here you are introducing a new configuration,
> > >> > > >>>> >> >> >> > unclean.recovery.strategy. The difficult thing
> here
> > is
> > >> > that
> > >> > > >>>> there
> > >> > > >>>> >> is a
> > >> > > >>>> >> >> >> > lot of overlap with
> unclean.leader.election.enable.
> > So
> > >> we
> > >> > > >>>> have 3
> > >> > > >>>> >> >> >> > different settings for unclean.recovery.strategy,
> > plus
> > >> 2
> > >> > > >>>> different
> > >> > > >>>> >> >> >> > settings for unclean.leader.election.enable,
> giving
> > a
> > >> > cross
> > >> > > >>>> >> product of
> > >> > > >>>> >> >> >> > 6 different options. The following "unclean
> recovery
> > >> > > manager"
> > >> > > >>>> >> section
> > >> > > >>>> >> >> >> > only applies to one fo those 6 different
> > possibilities
> > >> (I
> > >> > > >>>> think?)
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> > I simply don't think we need so many different
> > election
> > >> > > types.
> > >> > > >>>> >> Really
> > >> > > >>>> >> >> >> > the use-cases we need are people who want NO
> unclean
> > >> > > >>>> elections,
> > >> > > >>>> >> people
> > >> > > >>>> >> >> >> > who want "the reasonable thing" and people who
> want
> > >> > > >>>> avaialbility at
> > >> > > >>>> >> >> all
> > >> > > >>>> >> >> >> > costs.
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> > Overall, I feel like the first half of the KIP is
> > about
> > >> > the
> > >> > > >>>> ELR,
> > >> > > >>>> >> and
> > >> > > >>>> >> >> >> > the second half is about reworking unclean leader
> > >> > > election. It
> > >> > > >>>> >> might
> > >> > > >>>> >> >> be
> > >> > > >>>> >> >> >> > better to move that second half to a separate KIP
> so
> > >> that
> > >> > > we
> > >> > > >>>> can
> > >> > > >>>> >> >> figure
> > >> > > >>>> >> >> >> > it out fully. It should be fine to punt this until
> > >> later
> > >> > > and
> > >> > > >>>> just
> > >> > > >>>> >> have
> > >> > > >>>> >> >> >> > the current behavior on empty ELR be waiting for
> the
> > >> last
> > >> > > >>>> known
> > >> > > >>>> >> leader
> > >> > > >>>> >> >> >> > to return. After all, that's what we do today.
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> >> DescribeTopicRequest
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> > Is the intention for AdminClient to use this RPC
> for
> > >> > > >>>> >> >> >> > Admin#describeTopics ? If so, we need to describe
> > all
> > >> of
> > >> > > the
> > >> > > >>>> >> changes
> > >> > > >>>> >> >> to
> > >> > > >>>> >> >> >> > the admin client API, as well as changes to
> > >> command-line
> > >> > > >>>> tools like
> > >> > > >>>> >> >> >> > kafka-topics.sh (if there are any). For example,
> you
> > >> will
> > >> > > >>>> probably
> > >> > > >>>> >> >> need
> > >> > > >>>> >> >> >> > changes to TopicDescription.java. You will also
> > need to
> > >> > > >>>> provide
> > >> > > >>>> >> all of
> > >> > > >>>> >> >> >> > the things that admin client needs -- for example,
> > >> > > >>>> >> >> >> > TopicAuthorizedOperations.
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> > I also don't think the controller should serve
> this
> > >> > > request.
> > >> > > >>>> We
> > >> > > >>>> >> want
> > >> > > >>>> >> >> to
> > >> > > >>>> >> >> >> > minimize load on the controller. Just like with
> the
> > >> other
> > >> > > >>>> metadata
> > >> > > >>>> >> >> >> > requests like MetadataRequest, this should be
> > served by
> > >> > > >>>> brokers.
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> > It's a bit confusing why both topic ID and topic
> > name
> > >> are
> > >> > > >>>> provided
> > >> > > >>>> >> to
> > >> > > >>>> >> >> >> > this API. Is the intention that callers should set
> > one
> > >> > but
> > >> > > >>>> not the
> > >> > > >>>> >> >> >> > other? Or both? This needs to be clarified. Also,
> > if we
> > >> > do
> > >> > > >>>> want to
> > >> > > >>>> >> >> >> > support lookups by UUID, that is another thing
> that
> > >> needs
> > >> > > to
> > >> > > >>>> be
> > >> > > >>>> >> added
> > >> > > >>>> >> >> >> > to adminclient.
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> > In general, I feel like this should also probably
> be
> > >> its
> > >> > > own
> > >> > > >>>> KIP
> > >> > > >>>> >> since
> > >> > > >>>> >> >> >> > it's fairly complex
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> > best,
> > >> > > >>>> >> >> >> > Colin
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> > On Thu, Aug 10, 2023, at 15:46, Calvin Liu 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
> > >> > > >>>> >> >> >>
> > >> > > >>>> >> >>
> > >> > > >>>> >>
> > >> > > >>>>
> > >> > > >>>
> > >> > >
> > >> >
> > >>
> >
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Calvin Liu <ca...@confluent.io.INVALID>.
Hi Jun,
Thanks for the comments. And thanks to Colin's explanation.

41. The unclean recovery manager may need time to be ready for production.
So adding a flag to disable the new feature is in case it has any problems.
The config may be deprecated or removed when we are ready.

50. Sorry for the confusion, it should be “Others will be returned with
InvalidRequestError.”

51. Good point, added the error message field. Also if more partitions are
included, it will have the InvalidRequestError

52. Done

53. Good idea, make the DesiredLeaders nullable.

54. The downgrade refers to the Metadata version downgrade, so the brokers
still have the ability to parse the records.

In the ELR case, if the MV is downgraded, the ELR fields will be dropped in
the following partition updates.

55. Good idea, adding a version field to the CleanShutdownFile

56. Will have a section to show all the config changes.

===========

Hi David,

Thanks for the comments.

57. The CleanShutdownFile is removed after the log manager is initialized.
It will be created and written when the log manager is shutting down.

58. Good question, if the broker shuts down before it receives the broker
epoch, it will write -1.

59. The CleanShutdownFile is a part of the log manager shutdown. Its
presence represents the files were safely flushed to the disk before the
shutdown. So if the controlled shutdown timeouts or having any errors, it
is not necessary to affect the CleanShutdownFile.

On Tue, Oct 3, 2023 at 3:40 PM David Arthur
<da...@confluent.io.invalid> wrote:

> Calvin, thanks for the KIP!
>
> I'm getting up to speed on the discussion. I had a few questions
>
> 57. When is the CleanShutdownFile removed? I think it probably happens
> after registering with the controller, but it would be good to clarify
> this.
>
> 58. Since the broker epoch comes from the controller, what would go
> into the CleanShutdownFile in the case of a broker being unable to register
> with the controller? For example:
>
> 1) Broker A registers
>
> 2) Controller sees A, gives epoch 1
>
> 3) Broker A crashes, no CleanShutdownFile
>
> 4) Broker A starts up and shuts down before registering
>
>
> During 4) is a CleanShutdownFile produced? If so, what epoch goes in it?
>
> 59. What is the expected behavior when controlled shutdown times out?
> Looking at BrokerServer, I think the logs have a chance of still being
> closed cleanly, so this could be a regular clean shutdown scenario.
>
>
>
>
> On Tue, Oct 3, 2023 at 6:04 PM Colin McCabe <cm...@apache.org> wrote:
>
> > On Tue, Oct 3, 2023, at 10:49, Jun Rao wrote:
> > > Hi, Calvin,
> > >
> > > Thanks for the update KIP. A few more comments.
> > >
> > > 41. Why would a user choose the option to select a random replica as
> the
> > > leader instead of using unclean.recovery.strateg=Aggressive? It seems
> > that
> > > the latter is strictly better? If that's not the case, could we fold
> this
> > > option under unclean.recovery.strategy instead of introducing a
> separate
> > > config?
> >
> > Hi Jun,
> >
> > I thought the flow of control was:
> >
> > If there is no leader for the partition {
> >   If (there are unfenced ELR members) {
> >     choose_an_unfenced_ELR_member
> >   } else if (there are fenced ELR members AND strategy=Aggressive) {
> >     do_unclean_recovery
> >   } else if (there are no ELR members AND strategy != None) {
> >     do_unclean_recovery
> >   } else {
> >     do nothing about the missing leader
> >   }
> > }
> >
> > do_unclean_recovery() {
> >    if (unclean.recovery.manager.enabled) {
> >     use UncleanRecoveryManager
> >   } else {
> >     choose the last known leader if that is available, or a random leader
> > if not)
> >   }
> > }
> >
> > However, I think this could be clarified, especially the behavior when
> > unclean.recovery.manager.enabled=false. Inuitively the goal for
> > unclean.recovery.manager.enabled=false is to be "the same as now, mostly"
> > but it's very underspecified in the KIP, I agree.
> >
> > >
> > > 50. ElectLeadersRequest: "If more than 20 topics are included, only the
> > > first 20 will be served. Others will be returned with DesiredLeaders."
> > Hmm,
> > > not sure that I understand this. ElectLeadersResponse doesn't have a
> > > DesiredLeaders field.
> > >
> > > 51. GetReplicaLogInfo: "If more than 2000 partitions are included, only
> > the
> > > first 2000 will be served" Do we return an error for the remaining
> > > partitions? Actually, should we include an errorCode field at the
> > partition
> > > level in GetReplicaLogInfoResponse to cover non-existing partitions and
> > no
> > > authorization, etc?
> > >
> > > 52. The entry should matches => The entry should match
> > >
> > > 53. ElectLeadersRequest.DesiredLeaders: Should it be nullable since a
> > user
> > > may not specify DesiredLeaders?
> > >
> > > 54. Downgrade: Is that indeed possible? I thought earlier you said that
> > > once the new version of the records are in the metadata log, one can't
> > > downgrade since the old broker doesn't know how to parse the new
> version
> > of
> > > the metadata records?
> > >
> >
> > MetadataVersion downgrade is currently broken but we have fixing it on
> our
> > plate for Kafka 3.7.
> >
> > The way downgrade works is that "new features" are dropped, leaving only
> > the old ones.
> >
> > > 55. CleanShutdownFile: Should we add a version field for future
> > extension?
> > >
> > > 56. Config changes are public facing. Could we have a separate section
> to
> > > document all the config changes?
> >
> > +1. A separate section for this would be good.
> >
> > best,
> > Colin
> >
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Mon, Sep 25, 2023 at 4:29 PM Calvin Liu <caliu@confluent.io.invalid
> >
> > > wrote:
> > >
> > >> Hi Jun
> > >> Thanks for the comments.
> > >>
> > >> 40. If we change to None, it is not guaranteed for no data loss. For
> > users
> > >> who are not able to validate the data with external resources, manual
> > >> intervention does not give a better result but a loss of availability.
> > So
> > >> practically speaking, the Balance mode would be a better default
> value.
> > >>
> > >> 41. No, it represents how we want to do the unclean leader election.
> If
> > it
> > >> is false, the unclean leader election will be the old random way.
> > >> Otherwise, the unclean recovery will be used.
> > >>
> > >> 42. Good catch. Updated.
> > >>
> > >> 43. Only the first 20 topics will be served. Others will be returned
> > with
> > >> InvalidRequestError
> > >>
> > >> 44. The order matters. The desired leader entries match with the topic
> > >> partition list by the index.
> > >>
> > >> 45. Thanks! Updated.
> > >>
> > >> 46. Good advice! Updated.
> > >>
> > >> 47.1, updated the comment. Basically it will elect the replica in the
> > >> desiredLeader field to be the leader
> > >>
> > >> 47.2 We can let the admin client do the conversion. Using the
> > desiredLeader
> > >> field in the json format seems easier for users.
> > >>
> > >> 48. Once the MV version is downgraded, all the ELR related fields will
> > be
> > >> removed on the next partition change. The controller will also ignore
> > the
> > >> ELR fields. Updated the KIP.
> > >>
> > >> 49. Yes, it would be deprecated/removed.
> > >>
> > >>
> > >> On Mon, Sep 25, 2023 at 3:49 PM Jun Rao <ju...@confluent.io.invalid>
> > wrote:
> > >>
> > >> > Hi, Calvin,
> > >> >
> > >> > Thanks for the updated KIP. Made another pass. A few more comments
> > below.
> > >> >
> > >> > 40. unclean.leader.election.enable.false ->
> > >> > unclean.recovery.strategy.Balanced: The Balanced mode could still
> > lead to
> > >> > data loss. So, I am wondering if
> unclean.leader.election.enable.false
> > >> > should map to None?
> > >> >
> > >> > 41. unclean.recovery.manager.enabled: I am not sure why we introduce
> > this
> > >> > additional config. Is it the same as unclean.recovery.strategy=None?
> > >> >
> > >> > 42. DescribeTopicResponse.TopicAuthorizedOperations: Should this be
> at
> > >> the
> > >> > topic level?
> > >> >
> > >> > 43. "Limit: 20 topics max per request": Could we describe what
> > happens if
> > >> > the request includes more than 20 topics?
> > >> >
> > >> > 44. ElectLeadersRequest.DesiredLeaders: Could we describe whether
> the
> > >> > ordering matters?
> > >> >
> > >> > 45. GetReplicaLogInfo.TopicPartitions: "about": "The topic
> partitions
> > to
> > >> > elect leaders.": The description in "about" is incorrect.
> > >> >
> > >> > 46. GetReplicaLogInfoResponse: Should we nest partitions under
> > topicId to
> > >> > be consistent with other types of responses?
> > >> >
> > >> > 47. kafka-leader-election.sh:
> > >> > 47.1 Could we explain DESIGNATION?
> > >> > 47.2 desiredLeader: Should it be a list to match the field in
> > >> > ElectLeadersRequest?
> > >> >
> > >> > 48. We could add a section on downgrade?
> > >> >
> > >> > 49. LastKnownLeader: This seems only needed in the first phase of
> > >> > delivering ELR. Will it be removed when the complete KIP is
> delivered?
> > >> >
> > >> > Thanks,
> > >> >
> > >> > Jun
> > >> >
> > >> > On Tue, Sep 19, 2023 at 1:30 PM Colin McCabe <cm...@apache.org>
> > wrote:
> > >> >
> > >> > > Hi Calvin,
> > >> > >
> > >> > > Thanks for the explanations. I like the idea of using none,
> > balanced,
> > >> > > aggressive. We also had an offline discussion about why it is good
> > to
> > >> > use a
> > >> > > new config key (basically, so that we can deprecate the old one
> > which
> > >> had
> > >> > > only false/true values in 4.0) With these changes, I am +1.
> > >> > >
> > >> > > best,
> > >> > > Colin
> > >> > >
> > >> > > On Mon, Sep 18, 2023, at 15:54, Calvin Liu wrote:
> > >> > > > Hi Colin,
> > >> > > > Also, can we deprecate unclean.leader.election.enable in 4.0?
> > Before
> > >> > > that,
> > >> > > > we can have both the config unclean.recovery.strategy and
> > >> > > > unclean.leader.election.enable
> > >> > > > and using the unclean.recovery.Enabled to determine which config
> > to
> > >> use
> > >> > > > during the unclean leader election.
> > >> > > >
> > >> > > > On Mon, Sep 18, 2023 at 3:51 PM Calvin Liu <ca...@confluent.io>
> > >> wrote:
> > >> > > >
> > >> > > >> Hi Colin,
> > >> > > >> For the unclean.recovery.strategy config name, how about we use
> > the
> > >> > > >> following
> > >> > > >> None. It basically means no unclean recovery will be performed.
> > >> > > >> Aggressive. It means availability goes first. Whenever the
> > partition
> > >> > > can't
> > >> > > >> elect a durable replica, the controller will try the unclean
> > >> recovery.
> > >> > > >> Balanced. It is the balance point of the availability
> > >> > first(Aggressive)
> > >> > > >> and least availability(None). The controller performs unclean
> > >> recovery
> > >> > > when
> > >> > > >> both ISR and ELR are empty.
> > >> > > >>
> > >> > > >>
> > >> > > >> On Fri, Sep 15, 2023 at 11:42 AM Calvin Liu <
> caliu@confluent.io>
> > >> > wrote:
> > >> > > >>
> > >> > > >>> Hi Colin,
> > >> > > >>>
> > >> > > >>> > So, the proposal is that if someone sets
> > >> > > "unclean.leader.election.enable
> > >> > > >>> = true"...
> > >> > > >>>
> > >> > > >>>
> > >> > > >>> The idea is to use one of the unclean.leader.election.enable
> and
> > >> > > >>> unclean.recovery.strategy based on the
> > unclean.recovery.Enabled. A
> > >> > > possible
> > >> > > >>> version can be
> > >> > > >>>
> > >> > > >>> If unclean.recovery.Enabled:
> > >> > > >>>
> > >> > > >>> {
> > >> > > >>>
> > >> > > >>> Check unclean.recovery.strategy. If set, use it. Otherwise,
> > check
> > >> > > >>> unclean.leader.election.enable and translate it to
> > >> > > >>> unclean.recovery.strategy.
> > >> > > >>>
> > >> > > >>> } else {
> > >> > > >>>
> > >> > > >>> Use unclean.leader.election.enable
> > >> > > >>>
> > >> > > >>> }
> > >> > > >>>
> > >> > > >>>
> > >> > > >>> —--------
> > >> > > >>>
> > >> > > >>> >The configuration key should be
> > >> "unclean.recovery.manager.enabled",
> > >> > > >>> right?
> > >> > > >>>
> > >> > > >>>
> > >> > > >>> I think we have two ways of choosing a leader uncleanly,
> unclean
> > >> > leader
> > >> > > >>> election and unclean recovery(log inspection) and we try to
> > switch
> > >> > > between
> > >> > > >>> them.
> > >> > > >>>
> > >> > > >>> Do you mean we want to develop two ways of performing the
> > unclean
> > >> > > >>> recovery and one of them is using “unclean recovery manager”?
> I
> > >> guess
> > >> > > we
> > >> > > >>> haven’t discussed the second way.
> > >> > > >>>
> > >> > > >>>
> > >> > > >>> —-------
> > >> > > >>>
> > >> > > >>> >How do these 4 levels of overrides interact with your new
> > >> > > >>> configurations?
> > >> > > >>>
> > >> > > >>>
> > >> > > >>> I do notice in the Kraft controller code, the method to check
> > >> whether
> > >> > > >>> perform unclean leader election is hard coded to false since
> > >> > > >>> 2021(uncleanLeaderElectionEnabledForTopic). Isn’t it a good
> > chance
> > >> to
> > >> > > >>> completely deprecate the unclean.leader.election.enable? We
> > don’t
> > >> > even
> > >> > > have
> > >> > > >>> to worry about the config conversion.
> > >> > > >>>
> > >> > > >>> On the other hand, whatever the override is, as long as the
> > >> > controller
> > >> > > >>> can have the final effective unclean.leader.election.enable,
> the
> > >> > topic
> > >> > > >>> level config unclean.recovery.strategy, the cluster level
> config
> > >> > > >>> unclean.recovery.Enabled, the controller can calculate the
> > correct
> > >> > > methods
> > >> > > >>> to use right?
> > >> > > >>>
> > >> > > >>>
> > >> > > >>> On Fri, Sep 15, 2023 at 10:02 AM Colin McCabe <
> > cmccabe@apache.org>
> > >> > > wrote:
> > >> > > >>>
> > >> > > >>>> On Thu, Sep 14, 2023, at 22:23, Calvin Liu wrote:
> > >> > > >>>> > Hi Colin
> > >> > > >>>> > 1. I think using the new config name is more clear.
> > >> > > >>>> >        a. The unclean leader election is actually removed
> if
> > >> > unclean
> > >> > > >>>> > recovery is in use.
> > >> > > >>>> >        b. Using multiple values in
> > >> unclean.leader.election.enable
> > >> > is
> > >> > > >>>> > confusing and it will be more confusing after people forget
> > >> about
> > >> > > this
> > >> > > >>>> > discussion.
> > >> > > >>>>
> > >> > > >>>> Hi Calvin,
> > >> > > >>>>
> > >> > > >>>> So, the proposal is that if someone sets
> > >> > > "unclean.leader.election.enable
> > >> > > >>>> = true" but then sets one of your new configurations, the
> > value of
> > >> > > >>>> unclean.leader.election.enable is ignored? That seems less
> > clear
> > >> to
> > >> > > me, not
> > >> > > >>>> more. Just in general, having multiple configuration keys to
> > >> control
> > >> > > the
> > >> > > >>>> same thing confuses users. Basically, they are sitting at a
> > giant
> > >> > > control
> > >> > > >>>> panel, and some of the levers do nothing.
> > >> > > >>>>
> > >> > > >>>> > 2. Sorry I forgot to mention in the response that I did add
> > the
> > >> > > >>>> > unclean.recovery.Enabled flag.
> > >> > > >>>>
> > >> > > >>>> The configuration key should be
> > >> "unclean.recovery.manager.enabled",
> > >> > > >>>> right? Becuase we can do "unclean recovery" without the
> > manager.
> > >> > > Disabling
> > >> > > >>>> the manager just means we use a different mechanism for
> > recovery.
> > >> > > >>>>
> > >> > > >>>> >        c. Maybe I underestimated the challenge of replacing
> > the
> > >> > > >>>> config. Any
> > >> > > >>>> > implementation problems ahead?
> > >> > > >>>>
> > >> > > >>>> There are four levels of overrides for
> > >> > unclean.leader.election.enable.
> > >> > > >>>>
> > >> > > >>>> 1. static configuration for node.
> > >> > > >>>>     This goes in the configuration file, typically named
> > >> > > >>>> server.properties
> > >> > > >>>>
> > >> > > >>>> 2. dynamic configuration for node default
> > >> > > >>>>   ConfigResource(type=BROKER, name="")
> > >> > > >>>>
> > >> > > >>>> 3. dynamic configuration for node
> > >> > > >>>>   ConfigResource(type=BROKER, name=<controller id>)
> > >> > > >>>>
> > >> > > >>>> 4. dynamic configuration for topic
> > >> > > >>>>   ConfigResource(type=TOPIC, name=<topic-name>)
> > >> > > >>>>
> > >> > > >>>> How do these 4 levels of overrides interact with your new
> > >> > > >>>> configurations? If the new configurations dominate over the
> old
> > >> > ones,
> > >> > > it
> > >> > > >>>> seems like this will get a lot more confusing to implement
> (and
> > >> also
> > >> > > to
> > >> > > >>>> use.)
> > >> > > >>>>
> > >> > > >>>> Again, I'd recommend just adding some new values to
> > >> > > >>>> unclean.leader.election.enable. It's simple and will prevent
> > user
> > >> > > confusion
> > >> > > >>>> (as well as developer confusion.)
> > >> > > >>>>
> > >> > > >>>> best,
> > >> > > >>>> Colin
> > >> > > >>>>
> > >> > > >>>>
> > >> > > >>>> > 3. About the admin client, I mentioned 3 changes in the
> > client.
> > >> > > >>>> Anything
> > >> > > >>>> > else I missed in the KIP?
> > >> > > >>>> >       a. The client will switch to using the new RPC
> instead
> > of
> > >> > > >>>> > MetadataRequest for the topics.
> > >> > > >>>> >       b. The TopicPartitionInfo used in TopicDescription
> > needs
> > >> to
> > >> > > add
> > >> > > >>>> new
> > >> > > >>>> > fields related to the ELR.
> > >> > > >>>> >       c. The outputs will add the ELR related fields.
> > >> > > >>>> >
> > >> > > >>>> > On Thu, Sep 14, 2023 at 9:19 PM Colin McCabe <
> > >> cmccabe@apache.org>
> > >> > > >>>> wrote:
> > >> > > >>>> >
> > >> > > >>>> >> Hi Calvin,
> > >> > > >>>> >>
> > >> > > >>>> >> Thanks for the changes.
> > >> > > >>>> >>
> > >> > > >>>> >> 1. Earlier I commented that creating
> > >> "unclean.recovery.strategy "
> > >> > > is
> > >> > > >>>> not
> > >> > > >>>> >> necessary, and we can just reuse the existing
> > >> > > >>>> >> "unclean.leader.election.enable" configuration key. Let's
> > >> discuss
> > >> > > >>>> that.
> > >> > > >>>> >>
> > >> > > >>>> >> 2.I also don't understand why you didn't add a
> > configuration to
> > >> > > >>>> enable or
> > >> > > >>>> >> disable the Unclean Recovery Manager. This seems like a
> very
> > >> > simple
> > >> > > >>>> way to
> > >> > > >>>> >> handle the staging issue which we discussed. The URM can
> > just
> > >> be
> > >> > > >>>> turned off
> > >> > > >>>> >> until it is production ready. Let's discuss this.
> > >> > > >>>> >>
> > >> > > >>>> >> 3. You still need to describe the changes to AdminClient
> > that
> > >> are
> > >> > > >>>> needed
> > >> > > >>>> >> to use DescribeTopicRequest.
> > >> > > >>>> >>
> > >> > > >>>> >> Keep at it. It's looking better. :)
> > >> > > >>>> >>
> > >> > > >>>> >> best,
> > >> > > >>>> >> Colin
> > >> > > >>>> >>
> > >> > > >>>> >>
> > >> > > >>>> >> On Thu, Sep 14, 2023, at 11:03, Calvin Liu wrote:
> > >> > > >>>> >> > Hi Colin
> > >> > > >>>> >> > Thanks for the comments!
> > >> > > >>>> >> >
> > >> > > >>>> >> > I did the following changes
> > >> > > >>>> >> >
> > >> > > >>>> >> >    1.
> > >> > > >>>> >> >
> > >> > > >>>> >> >    Simplified the API spec section to only include the
> > diff.
> > >> > > >>>> >> >    2.
> > >> > > >>>> >> >
> > >> > > >>>> >> >    Reordered the HWM requirement section.
> > >> > > >>>> >> >    3.
> > >> > > >>>> >> >
> > >> > > >>>> >> >    Removed the URM implementation details to keep the
> > >> necessary
> > >> > > >>>> >> >    characteristics to perform the unclean recovery.
> > >> > > >>>> >> >    1.
> > >> > > >>>> >> >
> > >> > > >>>> >> >       When to perform the unclean recovery
> > >> > > >>>> >> >       2.
> > >> > > >>>> >> >
> > >> > > >>>> >> >       Under different config, how the unclean recovery
> > finds
> > >> > the
> > >> > > >>>> leader.
> > >> > > >>>> >> >       3.
> > >> > > >>>> >> >
> > >> > > >>>> >> >       How the config unclean.leader.election.enable and
> > >> > > >>>> >> >       unclean.recovery.strategy are converted when users
> > >> > > >>>> enable/disable
> > >> > > >>>> >> the
> > >> > > >>>> >> >       unclean recovery.
> > >> > > >>>> >> >       4.
> > >> > > >>>> >> >
> > >> > > >>>> >> >    More details about how we change admin client.
> > >> > > >>>> >> >    5.
> > >> > > >>>> >> >
> > >> > > >>>> >> >    API limits on the GetReplicaLogInfoRequest and
> > >> > > >>>> DescribeTopicRequest.
> > >> > > >>>> >> >    6.
> > >> > > >>>> >> >
> > >> > > >>>> >> >    Two metrics added
> > >> > > >>>> >> >    1.
> > >> > > >>>> >> >
> > >> > > >>>> >> >
> >  Kafka.controller.global_under_min_isr_partition_count
> > >> > > >>>> >> >       2.
> > >> > > >>>> >> >
> > >> > > >>>> >> >       kafka.controller.unclean_recovery_finished_count
> > >> > > >>>> >> >
> > >> > > >>>> >> >
> > >> > > >>>> >> > On Wed, Sep 13, 2023 at 10:46 AM Colin McCabe <
> > >> > > cmccabe@apache.org>
> > >> > > >>>> >> wrote:
> > >> > > >>>> >> >
> > >> > > >>>> >> >> On Tue, Sep 12, 2023, at 17:21, Calvin Liu wrote:
> > >> > > >>>> >> >> > Hi Colin
> > >> > > >>>> >> >> > Thanks for the comments!
> > >> > > >>>> >> >> >
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> Hi Calvin,
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> Thanks again for the KIP.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> One meta-comment: it's usually better to just do a diff
> > on a
> > >> > > >>>> message
> > >> > > >>>> >> spec
> > >> > > >>>> >> >> file or java file if you're including changes to it in
> > the
> > >> > KIP.
> > >> > > >>>> This is
> > >> > > >>>> >> >> easier to read than looking for "new fields begin" etc.
> > in
> > >> the
> > >> > > >>>> text, and
> > >> > > >>>> >> >> gracefully handles the case where existing fields were
> > >> > changed.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> > Rewrite the Additional High Watermark advancement
> > >> > requirement
> > >> > > >>>> >> >> > There was feedback on this section that some readers
> > may
> > >> not
> > >> > > be
> > >> > > >>>> >> familiar
> > >> > > >>>> >> >> > with HWM and Ack=0,1,all requests. This can help them
> > >> > > understand
> > >> > > >>>> the
> > >> > > >>>> >> >> > proposal. I will rewrite this part for more
> > readability.
> > >> > > >>>> >> >> >
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> To be clear, I wasn't suggesting dropping either
> > section. I
> > >> > > agree
> > >> > > >>>> that
> > >> > > >>>> >> >> they add useful background. I was just suggesting that
> we
> > >> > should
> > >> > > >>>> discuss
> > >> > > >>>> >> >> the "acks" setting AFTER discussing the new high
> > watermark
> > >> > > >>>> advancement
> > >> > > >>>> >> >> conditions. We also should discuss acks=0. While it
> isn't
> > >> > > >>>> conceptually
> > >> > > >>>> >> much
> > >> > > >>>> >> >> different than acks=1 here, its omission from this
> > section
> > >> is
> > >> > > >>>> confusing.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> > Unclean recovery
> > >> > > >>>> >> >> >
> > >> > > >>>> >> >> > The plan is to replace the
> > unclean.leader.election.enable
> > >> > with
> > >> > > >>>> >> >> > unclean.recovery.strategy. If the Unclean Recovery is
> > >> > enabled
> > >> > > >>>> then it
> > >> > > >>>> >> >> deals
> > >> > > >>>> >> >> > with the three options in the
> > unclean.recovery.strategy.
> > >> > > >>>> >> >> >
> > >> > > >>>> >> >> >
> > >> > > >>>> >> >> > Let’s refine the Unclean Recovery. We have already
> > taken a
> > >> > > lot of
> > >> > > >>>> >> >> > suggestions and I hope to enhance the durability of
> > Kafka
> > >> to
> > >> > > the
> > >> > > >>>> next
> > >> > > >>>> >> >> level
> > >> > > >>>> >> >> > with this KIP.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> I am OK with doing the unclean leader recovery
> > improvements
> > >> in
> > >> > > >>>> this KIP.
> > >> > > >>>> >> >> However, I think we need to really work on the
> > configuration
> > >> > > >>>> settings.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> Configuration overrides are often quite messy. For
> > example,
> > >> > the
> > >> > > >>>> cases
> > >> > > >>>> >> >> where we have log.roll.hours and log.roll.segment.ms,
> > the
> > >> > user
> > >> > > >>>> has to
> > >> > > >>>> >> >> remember which one takes precedence, and it is not
> > obvious.
> > >> > So,
> > >> > > >>>> rather
> > >> > > >>>> >> than
> > >> > > >>>> >> >> creating a new configuration, why not add additional
> > values
> > >> to
> > >> > > >>>> >> >> "unclean.leader.election.enable"? I think this will be
> > >> simpler
> > >> > > for
> > >> > > >>>> >> people
> > >> > > >>>> >> >> to understand, and simpler in the code as well.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> What if we continued to use
> > "unclean.leader.election.enable"
> > >> > but
> > >> > > >>>> >> extended
> > >> > > >>>> >> >> it so that it took a string? Then the string could have
> > >> these
> > >> > > >>>> values:
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> never
> > >> > > >>>> >> >>     never automatically do an unclean leader election
> > under
> > >> > any
> > >> > > >>>> >> conditions
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> false / default
> > >> > > >>>> >> >>     only do an unclean leader election if there may be
> > >> > possible
> > >> > > >>>> data
> > >> > > >>>> >> loss
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> true / always
> > >> > > >>>> >> >>     always do an unclean leader election if we can't
> > >> > immediately
> > >> > > >>>> elect a
> > >> > > >>>> >> >> leader
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> It's a bit awkward that false maps to default rather
> > than to
> > >> > > >>>> never. But
> > >> > > >>>> >> >> this awkwardness exists if we use two different
> > >> configuration
> > >> > > keys
> > >> > > >>>> as
> > >> > > >>>> >> well.
> > >> > > >>>> >> >> The reason for the awkwardness is that we simply don't
> > want
> > >> > most
> > >> > > >>>> of the
> > >> > > >>>> >> >> people currently setting
> > >> unclean.leader.election.enable=false
> > >> > to
> > >> > > >>>> get the
> > >> > > >>>> >> >> "never" behavior. We have to bite that bullet. Better
> to
> > be
> > >> > > clear
> > >> > > >>>> and
> > >> > > >>>> >> >> explicit than hide it.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> Another thing that's a bit awkward is having two
> > different
> > >> > ways
> > >> > > to
> > >> > > >>>> do
> > >> > > >>>> >> >> unclean leader election specified in the KIP. You
> > descirbe
> > >> two
> > >> > > >>>> methods:
> > >> > > >>>> >> the
> > >> > > >>>> >> >> simple "choose the last leader" method, and the
> "unclean
> > >> > > recovery
> > >> > > >>>> >> manager"
> > >> > > >>>> >> >> method. I understand why you did it this way -- "choose
> > the
> > >> > last
> > >> > > >>>> >> leader" is
> > >> > > >>>> >> >> simple, and will help us deliver an implementation
> > quickly,
> > >> > > while
> > >> > > >>>> the
> > >> > > >>>> >> URM
> > >> > > >>>> >> >> is preferable in the long term. My suggestion here is
> to
> > >> > > separate
> > >> > > >>>> the
> > >> > > >>>> >> >> decision of HOW to do unclean leader election from the
> > >> > decision
> > >> > > of
> > >> > > >>>> WHEN
> > >> > > >>>> >> to
> > >> > > >>>> >> >> do it.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> So in other words, have
> "unclean.leader.election.enable"
> > >> > specify
> > >> > > >>>> when we
> > >> > > >>>> >> >> do unclean leader election, and have a new
> configuration
> > >> like
> > >> > > >>>> >> >> "unclean.recovery.manager.enable" to determine if we
> use
> > the
> > >> > > URM.
> > >> > > >>>> >> >> Presumably the URM will take some time to get fully
> > stable,
> > >> so
> > >> > > >>>> this can
> > >> > > >>>> >> >> default to false for a while, and we can flip the
> > default to
> > >> > > true
> > >> > > >>>> when
> > >> > > >>>> >> we
> > >> > > >>>> >> >> feel ready.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> The URM is somewhat under-described here. I think we
> > need a
> > >> > few
> > >> > > >>>> >> >> configurations here for it. For example, we need a
> > >> > > configuration to
> > >> > > >>>> >> specify
> > >> > > >>>> >> >> how long it should wait for a broker to respond to its
> > RPCs
> > >> > > before
> > >> > > >>>> >> moving
> > >> > > >>>> >> >> on. We also need to understand how the URM interacts
> with
> > >> > > >>>> >> >> unclean.leader.election.enable=always. I assume that
> with
> > >> > > "always"
> > >> > > >>>> we
> > >> > > >>>> >> will
> > >> > > >>>> >> >> just unconditionally use the URM rather than choosing
> > >> > randomly.
> > >> > > >>>> But this
> > >> > > >>>> >> >> should be spelled out in the KIP.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> >
> > >> > > >>>> >> >> > DescribeTopicRequest
> > >> > > >>>> >> >> >
> > >> > > >>>> >> >> >    1.
> > >> > > >>>> >> >> >    Yes, the plan is to replace the MetadataRequest
> with
> > >> the
> > >> > > >>>> >> >> >    DescribeTopicRequest for the admin clients. Will
> > check
> > >> > the
> > >> > > >>>> details.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> Sounds good. But as I said, you need to specify how
> > >> > AdminClient
> > >> > > >>>> >> interacts
> > >> > > >>>> >> >> with the new request. This will involve adding some
> > fields
> > >> to
> > >> > > >>>> >> >> TopicDescription.java. And you need to specify the
> > changes
> > >> to
> > >> > > the
> > >> > > >>>> >> >> kafka-topics.sh command line tool. Otherwise we cannot
> > use
> > >> the
> > >> > > >>>> tool to
> > >> > > >>>> >> see
> > >> > > >>>> >> >> the new information.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> The new requests, DescribeTopicRequest and
> > >> > > >>>> GetReplicaLogInfoRequest,
> > >> > > >>>> >> need
> > >> > > >>>> >> >> to have limits placed on them so that their size can't
> be
> > >> > > >>>> infinite. We
> > >> > > >>>> >> >> don't want to propagate the current problems of
> > >> > MetadataRequest,
> > >> > > >>>> where
> > >> > > >>>> >> >> clients can request massive responses that can mess up
> > the
> > >> JVM
> > >> > > when
> > >> > > >>>> >> handled.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> Adding limits is simple for GetReplicaLogInfoRequest --
> > we
> > >> can
> > >> > > >>>> just say
> > >> > > >>>> >> >> that only 2000 partitions at a time can be requested.
> For
> > >> > > >>>> >> >> DescribeTopicRequest we can probably just limit to 20
> > topics
> > >> > or
> > >> > > >>>> >> something
> > >> > > >>>> >> >> like that, to avoid the complexity of doing pagination
> in
> > >> this
> > >> > > KIP.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> >    2.
> > >> > > >>>> >> >> >    I can let the broker load the ELR info so that
> they
> > can
> > >> > > serve
> > >> > > >>>> the
> > >> > > >>>> >> >> >    DescribeTopicRequest as well.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> Yes, it's fine to add to MetadataCache. In fact, you'll
> > be
> > >> > > loading
> > >> > > >>>> it
> > >> > > >>>> >> >> anyway once it's added to PartitionImage.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> >    3.
> > >> > > >>>> >> >> >    Yeah, it does not make sense to have the topic id
> if
> > >> > > >>>> >> >> >    DescribeTopicRequest is only used by the admin
> > client.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> OK. That makes things simpler. We can always create a
> new
> > >> API
> > >> > > later
> > >> > > >>>> >> >> (hopefully not in this KIP!) to query by topic ID.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> >
> > >> > > >>>> >> >> >
> > >> > > >>>> >> >> > Metrics
> > >> > > >>>> >> >> >
> > >> > > >>>> >> >> > As for overall cluster health metrics, I think
> > >> under-min-ISR
> > >> > > is
> > >> > > >>>> still
> > >> > > >>>> >> a
> > >> > > >>>> >> >> > useful one. ELR is more like a safety belt. When the
> > ELR
> > >> is
> > >> > > >>>> used, the
> > >> > > >>>> >> >> > cluster availability has already been impacted.
> > >> > > >>>> >> >> >
> > >> > > >>>> >> >> > Maybe we can have a metric to count the partitions
> that
> > >> > > sum(ISR,
> > >> > > >>>> ELR)
> > >> > > >>>> >> <
> > >> > > >>>> >> >> min
> > >> > > >>>> >> >> > ISR. What do you think?
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> How about:
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> A.  a metric for the totoal number of under-min-isr
> > >> > partitions?
> > >> > > We
> > >> > > >>>> don't
> > >> > > >>>> >> >> have that in Apache Kafka at the moment.
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> B. a metric for the number of unclean leader elections
> we
> > >> did
> > >> > > (for
> > >> > > >>>> >> >> simplicity, it can reset to 0 on controller restart: we
> > >> expect
> > >> > > >>>> people to
> > >> > > >>>> >> >> monitor the change over time anyway)
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> best,
> > >> > > >>>> >> >> Colin
> > >> > > >>>> >> >>
> > >> > > >>>> >> >>
> > >> > > >>>> >> >> >
> > >> > > >>>> >> >> > Yeah, for the ongoing unclean recoveries, the
> > controller
> > >> can
> > >> > > >>>> keep an
> > >> > > >>>> >> >> > accurate count through failover because partition
> > >> > registration
> > >> > > >>>> can
> > >> > > >>>> >> >> indicate
> > >> > > >>>> >> >> > whether a recovery is needed. However, for the
> happened
> > >> > ones,
> > >> > > >>>> unless
> > >> > > >>>> >> we
> > >> > > >>>> >> >> > want to persist the number somewhere, we can only
> > figure
> > >> it
> > >> > > out
> > >> > > >>>> from
> > >> > > >>>> >> the
> > >> > > >>>> >> >> > log.
> > >> > > >>>> >> >> >
> > >> > > >>>> >> >> > On Tue, Sep 12, 2023 at 3:16 PM Colin McCabe <
> > >> > > cmccabe@apache.org
> > >> > > >>>> >
> > >> > > >>>> >> wrote:
> > >> > > >>>> >> >> >
> > >> > > >>>> >> >> >> Also, we should have metrics that show what is going
> > on
> > >> > with
> > >> > > >>>> regard
> > >> > > >>>> >> to
> > >> > > >>>> >> >> the
> > >> > > >>>> >> >> >> eligible replica set. I'm not sure exactly what to
> > >> suggest,
> > >> > > but
> > >> > > >>>> >> >> something
> > >> > > >>>> >> >> >> that could identify when things are going wrong in
> the
> > >> > > clsuter.
> > >> > > >>>> >> >> >>
> > >> > > >>>> >> >> >> For example, maybe a metric for partitions
> containing
> > >> > > replicas
> > >> > > >>>> that
> > >> > > >>>> >> are
> > >> > > >>>> >> >> >> ineligible to be leader? That would show a spike
> when
> > a
> > >> > > broker
> > >> > > >>>> had an
> > >> > > >>>> >> >> >> unclean restart.
> > >> > > >>>> >> >> >>
> > >> > > >>>> >> >> >> Ideally, we'd also have a metric that indicates when
> > an
> > >> > > unclear
> > >> > > >>>> >> leader
> > >> > > >>>> >> >> >> election or a recovery happened. It's a bit tricky
> > >> because
> > >> > > the
> > >> > > >>>> simple
> > >> > > >>>> >> >> >> thing, of tracking it per controller, may be a bit
> > >> > confusing
> > >> > > >>>> during
> > >> > > >>>> >> >> >> failovers.
> > >> > > >>>> >> >> >>
> > >> > > >>>> >> >> >> best,
> > >> > > >>>> >> >> >> Colin
> > >> > > >>>> >> >> >>
> > >> > > >>>> >> >> >>
> > >> > > >>>> >> >> >> On Tue, Sep 12, 2023, at 14:25, Colin McCabe wrote:
> > >> > > >>>> >> >> >> > Hi Calvin,
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> > Thanks for the KIP. I think this is a great
> > >> improvement.
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> >> Additional High Watermark advance requirement
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> > Typo: change "advance" to "advancement"
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> >> A bit recap of some key concepts.
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> > Typo: change "bit" to "quick"
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> >> Ack=1/all produce request. It defines when the
> > Kafka
> > >> > > server
> > >> > > >>>> should
> > >> > > >>>> >> >> >> respond to the produce request
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> > I think this section would be clearer if we talked
> > >> about
> > >> > > the
> > >> > > >>>> new
> > >> > > >>>> >> high
> > >> > > >>>> >> >> >> > watermark advancement requirement first, and THEN
> > >> talked
> > >> > > >>>> about its
> > >> > > >>>> >> >> >> > impact on acks=0, acks=1, and     acks=all.
> > acks=all
> > >> is
> > >> > of
> > >> > > >>>> course
> > >> > > >>>> >> the
> > >> > > >>>> >> >> >> > main case we care about here, so it would be good
> to
> > >> lead
> > >> > > with
> > >> > > >>>> >> that,
> > >> > > >>>> >> >> >> > rather than delving into the technicalities of
> > acks=0/1
> > >> > > first.
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> >> Unclean recovery
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> > So, here you are introducing a new configuration,
> > >> > > >>>> >> >> >> > unclean.recovery.strategy. The difficult thing
> here
> > is
> > >> > that
> > >> > > >>>> there
> > >> > > >>>> >> is a
> > >> > > >>>> >> >> >> > lot of overlap with
> unclean.leader.election.enable.
> > So
> > >> we
> > >> > > >>>> have 3
> > >> > > >>>> >> >> >> > different settings for unclean.recovery.strategy,
> > plus
> > >> 2
> > >> > > >>>> different
> > >> > > >>>> >> >> >> > settings for unclean.leader.election.enable,
> giving
> > a
> > >> > cross
> > >> > > >>>> >> product of
> > >> > > >>>> >> >> >> > 6 different options. The following "unclean
> recovery
> > >> > > manager"
> > >> > > >>>> >> section
> > >> > > >>>> >> >> >> > only applies to one fo those 6 different
> > possibilities
> > >> (I
> > >> > > >>>> think?)
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> > I simply don't think we need so many different
> > election
> > >> > > types.
> > >> > > >>>> >> Really
> > >> > > >>>> >> >> >> > the use-cases we need are people who want NO
> unclean
> > >> > > >>>> elections,
> > >> > > >>>> >> people
> > >> > > >>>> >> >> >> > who want "the reasonable thing" and people who
> want
> > >> > > >>>> avaialbility at
> > >> > > >>>> >> >> all
> > >> > > >>>> >> >> >> > costs.
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> > Overall, I feel like the first half of the KIP is
> > about
> > >> > the
> > >> > > >>>> ELR,
> > >> > > >>>> >> and
> > >> > > >>>> >> >> >> > the second half is about reworking unclean leader
> > >> > > election. It
> > >> > > >>>> >> might
> > >> > > >>>> >> >> be
> > >> > > >>>> >> >> >> > better to move that second half to a separate KIP
> so
> > >> that
> > >> > > we
> > >> > > >>>> can
> > >> > > >>>> >> >> figure
> > >> > > >>>> >> >> >> > it out fully. It should be fine to punt this until
> > >> later
> > >> > > and
> > >> > > >>>> just
> > >> > > >>>> >> have
> > >> > > >>>> >> >> >> > the current behavior on empty ELR be waiting for
> the
> > >> last
> > >> > > >>>> known
> > >> > > >>>> >> leader
> > >> > > >>>> >> >> >> > to return. After all, that's what we do today.
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> >> DescribeTopicRequest
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> > Is the intention for AdminClient to use this RPC
> for
> > >> > > >>>> >> >> >> > Admin#describeTopics ? If so, we need to describe
> > all
> > >> of
> > >> > > the
> > >> > > >>>> >> changes
> > >> > > >>>> >> >> to
> > >> > > >>>> >> >> >> > the admin client API, as well as changes to
> > >> command-line
> > >> > > >>>> tools like
> > >> > > >>>> >> >> >> > kafka-topics.sh (if there are any). For example,
> you
> > >> will
> > >> > > >>>> probably
> > >> > > >>>> >> >> need
> > >> > > >>>> >> >> >> > changes to TopicDescription.java. You will also
> > need to
> > >> > > >>>> provide
> > >> > > >>>> >> all of
> > >> > > >>>> >> >> >> > the things that admin client needs -- for example,
> > >> > > >>>> >> >> >> > TopicAuthorizedOperations.
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> > I also don't think the controller should serve
> this
> > >> > > request.
> > >> > > >>>> We
> > >> > > >>>> >> want
> > >> > > >>>> >> >> to
> > >> > > >>>> >> >> >> > minimize load on the controller. Just like with
> the
> > >> other
> > >> > > >>>> metadata
> > >> > > >>>> >> >> >> > requests like MetadataRequest, this should be
> > served by
> > >> > > >>>> brokers.
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> > It's a bit confusing why both topic ID and topic
> > name
> > >> are
> > >> > > >>>> provided
> > >> > > >>>> >> to
> > >> > > >>>> >> >> >> > this API. Is the intention that callers should set
> > one
> > >> > but
> > >> > > >>>> not the
> > >> > > >>>> >> >> >> > other? Or both? This needs to be clarified. Also,
> > if we
> > >> > do
> > >> > > >>>> want to
> > >> > > >>>> >> >> >> > support lookups by UUID, that is another thing
> that
> > >> needs
> > >> > > to
> > >> > > >>>> be
> > >> > > >>>> >> added
> > >> > > >>>> >> >> >> > to adminclient.
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> > In general, I feel like this should also probably
> be
> > >> its
> > >> > > own
> > >> > > >>>> KIP
> > >> > > >>>> >> since
> > >> > > >>>> >> >> >> > it's fairly complex
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> > best,
> > >> > > >>>> >> >> >> > Colin
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> >
> > >> > > >>>> >> >> >> > On Thu, Aug 10, 2023, at 15:46, Calvin Liu 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
> > >> > > >>>> >> >> >>
> > >> > > >>>> >> >>
> > >> > > >>>> >>
> > >> > > >>>>
> > >> > > >>>
> > >> > >
> > >> >
> > >>
> >
>
>
> --
> -David
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by David Arthur <da...@confluent.io.INVALID>.
Calvin, thanks for the KIP!

I'm getting up to speed on the discussion. I had a few questions

57. When is the CleanShutdownFile removed? I think it probably happens
after registering with the controller, but it would be good to clarify this.

58. Since the broker epoch comes from the controller, what would go
into the CleanShutdownFile in the case of a broker being unable to register
with the controller? For example:

1) Broker A registers

2) Controller sees A, gives epoch 1

3) Broker A crashes, no CleanShutdownFile

4) Broker A starts up and shuts down before registering


During 4) is a CleanShutdownFile produced? If so, what epoch goes in it?

59. What is the expected behavior when controlled shutdown times out?
Looking at BrokerServer, I think the logs have a chance of still being
closed cleanly, so this could be a regular clean shutdown scenario.




On Tue, Oct 3, 2023 at 6:04 PM Colin McCabe <cm...@apache.org> wrote:

> On Tue, Oct 3, 2023, at 10:49, Jun Rao wrote:
> > Hi, Calvin,
> >
> > Thanks for the update KIP. A few more comments.
> >
> > 41. Why would a user choose the option to select a random replica as the
> > leader instead of using unclean.recovery.strateg=Aggressive? It seems
> that
> > the latter is strictly better? If that's not the case, could we fold this
> > option under unclean.recovery.strategy instead of introducing a separate
> > config?
>
> Hi Jun,
>
> I thought the flow of control was:
>
> If there is no leader for the partition {
>   If (there are unfenced ELR members) {
>     choose_an_unfenced_ELR_member
>   } else if (there are fenced ELR members AND strategy=Aggressive) {
>     do_unclean_recovery
>   } else if (there are no ELR members AND strategy != None) {
>     do_unclean_recovery
>   } else {
>     do nothing about the missing leader
>   }
> }
>
> do_unclean_recovery() {
>    if (unclean.recovery.manager.enabled) {
>     use UncleanRecoveryManager
>   } else {
>     choose the last known leader if that is available, or a random leader
> if not)
>   }
> }
>
> However, I think this could be clarified, especially the behavior when
> unclean.recovery.manager.enabled=false. Inuitively the goal for
> unclean.recovery.manager.enabled=false is to be "the same as now, mostly"
> but it's very underspecified in the KIP, I agree.
>
> >
> > 50. ElectLeadersRequest: "If more than 20 topics are included, only the
> > first 20 will be served. Others will be returned with DesiredLeaders."
> Hmm,
> > not sure that I understand this. ElectLeadersResponse doesn't have a
> > DesiredLeaders field.
> >
> > 51. GetReplicaLogInfo: "If more than 2000 partitions are included, only
> the
> > first 2000 will be served" Do we return an error for the remaining
> > partitions? Actually, should we include an errorCode field at the
> partition
> > level in GetReplicaLogInfoResponse to cover non-existing partitions and
> no
> > authorization, etc?
> >
> > 52. The entry should matches => The entry should match
> >
> > 53. ElectLeadersRequest.DesiredLeaders: Should it be nullable since a
> user
> > may not specify DesiredLeaders?
> >
> > 54. Downgrade: Is that indeed possible? I thought earlier you said that
> > once the new version of the records are in the metadata log, one can't
> > downgrade since the old broker doesn't know how to parse the new version
> of
> > the metadata records?
> >
>
> MetadataVersion downgrade is currently broken but we have fixing it on our
> plate for Kafka 3.7.
>
> The way downgrade works is that "new features" are dropped, leaving only
> the old ones.
>
> > 55. CleanShutdownFile: Should we add a version field for future
> extension?
> >
> > 56. Config changes are public facing. Could we have a separate section to
> > document all the config changes?
>
> +1. A separate section for this would be good.
>
> best,
> Colin
>
> >
> > Thanks,
> >
> > Jun
> >
> > On Mon, Sep 25, 2023 at 4:29 PM Calvin Liu <ca...@confluent.io.invalid>
> > wrote:
> >
> >> Hi Jun
> >> Thanks for the comments.
> >>
> >> 40. If we change to None, it is not guaranteed for no data loss. For
> users
> >> who are not able to validate the data with external resources, manual
> >> intervention does not give a better result but a loss of availability.
> So
> >> practically speaking, the Balance mode would be a better default value.
> >>
> >> 41. No, it represents how we want to do the unclean leader election. If
> it
> >> is false, the unclean leader election will be the old random way.
> >> Otherwise, the unclean recovery will be used.
> >>
> >> 42. Good catch. Updated.
> >>
> >> 43. Only the first 20 topics will be served. Others will be returned
> with
> >> InvalidRequestError
> >>
> >> 44. The order matters. The desired leader entries match with the topic
> >> partition list by the index.
> >>
> >> 45. Thanks! Updated.
> >>
> >> 46. Good advice! Updated.
> >>
> >> 47.1, updated the comment. Basically it will elect the replica in the
> >> desiredLeader field to be the leader
> >>
> >> 47.2 We can let the admin client do the conversion. Using the
> desiredLeader
> >> field in the json format seems easier for users.
> >>
> >> 48. Once the MV version is downgraded, all the ELR related fields will
> be
> >> removed on the next partition change. The controller will also ignore
> the
> >> ELR fields. Updated the KIP.
> >>
> >> 49. Yes, it would be deprecated/removed.
> >>
> >>
> >> On Mon, Sep 25, 2023 at 3:49 PM Jun Rao <ju...@confluent.io.invalid>
> wrote:
> >>
> >> > Hi, Calvin,
> >> >
> >> > Thanks for the updated KIP. Made another pass. A few more comments
> below.
> >> >
> >> > 40. unclean.leader.election.enable.false ->
> >> > unclean.recovery.strategy.Balanced: The Balanced mode could still
> lead to
> >> > data loss. So, I am wondering if unclean.leader.election.enable.false
> >> > should map to None?
> >> >
> >> > 41. unclean.recovery.manager.enabled: I am not sure why we introduce
> this
> >> > additional config. Is it the same as unclean.recovery.strategy=None?
> >> >
> >> > 42. DescribeTopicResponse.TopicAuthorizedOperations: Should this be at
> >> the
> >> > topic level?
> >> >
> >> > 43. "Limit: 20 topics max per request": Could we describe what
> happens if
> >> > the request includes more than 20 topics?
> >> >
> >> > 44. ElectLeadersRequest.DesiredLeaders: Could we describe whether the
> >> > ordering matters?
> >> >
> >> > 45. GetReplicaLogInfo.TopicPartitions: "about": "The topic partitions
> to
> >> > elect leaders.": The description in "about" is incorrect.
> >> >
> >> > 46. GetReplicaLogInfoResponse: Should we nest partitions under
> topicId to
> >> > be consistent with other types of responses?
> >> >
> >> > 47. kafka-leader-election.sh:
> >> > 47.1 Could we explain DESIGNATION?
> >> > 47.2 desiredLeader: Should it be a list to match the field in
> >> > ElectLeadersRequest?
> >> >
> >> > 48. We could add a section on downgrade?
> >> >
> >> > 49. LastKnownLeader: This seems only needed in the first phase of
> >> > delivering ELR. Will it be removed when the complete KIP is delivered?
> >> >
> >> > Thanks,
> >> >
> >> > Jun
> >> >
> >> > On Tue, Sep 19, 2023 at 1:30 PM Colin McCabe <cm...@apache.org>
> wrote:
> >> >
> >> > > Hi Calvin,
> >> > >
> >> > > Thanks for the explanations. I like the idea of using none,
> balanced,
> >> > > aggressive. We also had an offline discussion about why it is good
> to
> >> > use a
> >> > > new config key (basically, so that we can deprecate the old one
> which
> >> had
> >> > > only false/true values in 4.0) With these changes, I am +1.
> >> > >
> >> > > best,
> >> > > Colin
> >> > >
> >> > > On Mon, Sep 18, 2023, at 15:54, Calvin Liu wrote:
> >> > > > Hi Colin,
> >> > > > Also, can we deprecate unclean.leader.election.enable in 4.0?
> Before
> >> > > that,
> >> > > > we can have both the config unclean.recovery.strategy and
> >> > > > unclean.leader.election.enable
> >> > > > and using the unclean.recovery.Enabled to determine which config
> to
> >> use
> >> > > > during the unclean leader election.
> >> > > >
> >> > > > On Mon, Sep 18, 2023 at 3:51 PM Calvin Liu <ca...@confluent.io>
> >> wrote:
> >> > > >
> >> > > >> Hi Colin,
> >> > > >> For the unclean.recovery.strategy config name, how about we use
> the
> >> > > >> following
> >> > > >> None. It basically means no unclean recovery will be performed.
> >> > > >> Aggressive. It means availability goes first. Whenever the
> partition
> >> > > can't
> >> > > >> elect a durable replica, the controller will try the unclean
> >> recovery.
> >> > > >> Balanced. It is the balance point of the availability
> >> > first(Aggressive)
> >> > > >> and least availability(None). The controller performs unclean
> >> recovery
> >> > > when
> >> > > >> both ISR and ELR are empty.
> >> > > >>
> >> > > >>
> >> > > >> On Fri, Sep 15, 2023 at 11:42 AM Calvin Liu <ca...@confluent.io>
> >> > wrote:
> >> > > >>
> >> > > >>> Hi Colin,
> >> > > >>>
> >> > > >>> > So, the proposal is that if someone sets
> >> > > "unclean.leader.election.enable
> >> > > >>> = true"...
> >> > > >>>
> >> > > >>>
> >> > > >>> The idea is to use one of the unclean.leader.election.enable and
> >> > > >>> unclean.recovery.strategy based on the
> unclean.recovery.Enabled. A
> >> > > possible
> >> > > >>> version can be
> >> > > >>>
> >> > > >>> If unclean.recovery.Enabled:
> >> > > >>>
> >> > > >>> {
> >> > > >>>
> >> > > >>> Check unclean.recovery.strategy. If set, use it. Otherwise,
> check
> >> > > >>> unclean.leader.election.enable and translate it to
> >> > > >>> unclean.recovery.strategy.
> >> > > >>>
> >> > > >>> } else {
> >> > > >>>
> >> > > >>> Use unclean.leader.election.enable
> >> > > >>>
> >> > > >>> }
> >> > > >>>
> >> > > >>>
> >> > > >>> —--------
> >> > > >>>
> >> > > >>> >The configuration key should be
> >> "unclean.recovery.manager.enabled",
> >> > > >>> right?
> >> > > >>>
> >> > > >>>
> >> > > >>> I think we have two ways of choosing a leader uncleanly, unclean
> >> > leader
> >> > > >>> election and unclean recovery(log inspection) and we try to
> switch
> >> > > between
> >> > > >>> them.
> >> > > >>>
> >> > > >>> Do you mean we want to develop two ways of performing the
> unclean
> >> > > >>> recovery and one of them is using “unclean recovery manager”? I
> >> guess
> >> > > we
> >> > > >>> haven’t discussed the second way.
> >> > > >>>
> >> > > >>>
> >> > > >>> —-------
> >> > > >>>
> >> > > >>> >How do these 4 levels of overrides interact with your new
> >> > > >>> configurations?
> >> > > >>>
> >> > > >>>
> >> > > >>> I do notice in the Kraft controller code, the method to check
> >> whether
> >> > > >>> perform unclean leader election is hard coded to false since
> >> > > >>> 2021(uncleanLeaderElectionEnabledForTopic). Isn’t it a good
> chance
> >> to
> >> > > >>> completely deprecate the unclean.leader.election.enable? We
> don’t
> >> > even
> >> > > have
> >> > > >>> to worry about the config conversion.
> >> > > >>>
> >> > > >>> On the other hand, whatever the override is, as long as the
> >> > controller
> >> > > >>> can have the final effective unclean.leader.election.enable, the
> >> > topic
> >> > > >>> level config unclean.recovery.strategy, the cluster level config
> >> > > >>> unclean.recovery.Enabled, the controller can calculate the
> correct
> >> > > methods
> >> > > >>> to use right?
> >> > > >>>
> >> > > >>>
> >> > > >>> On Fri, Sep 15, 2023 at 10:02 AM Colin McCabe <
> cmccabe@apache.org>
> >> > > wrote:
> >> > > >>>
> >> > > >>>> On Thu, Sep 14, 2023, at 22:23, Calvin Liu wrote:
> >> > > >>>> > Hi Colin
> >> > > >>>> > 1. I think using the new config name is more clear.
> >> > > >>>> >        a. The unclean leader election is actually removed if
> >> > unclean
> >> > > >>>> > recovery is in use.
> >> > > >>>> >        b. Using multiple values in
> >> unclean.leader.election.enable
> >> > is
> >> > > >>>> > confusing and it will be more confusing after people forget
> >> about
> >> > > this
> >> > > >>>> > discussion.
> >> > > >>>>
> >> > > >>>> Hi Calvin,
> >> > > >>>>
> >> > > >>>> So, the proposal is that if someone sets
> >> > > "unclean.leader.election.enable
> >> > > >>>> = true" but then sets one of your new configurations, the
> value of
> >> > > >>>> unclean.leader.election.enable is ignored? That seems less
> clear
> >> to
> >> > > me, not
> >> > > >>>> more. Just in general, having multiple configuration keys to
> >> control
> >> > > the
> >> > > >>>> same thing confuses users. Basically, they are sitting at a
> giant
> >> > > control
> >> > > >>>> panel, and some of the levers do nothing.
> >> > > >>>>
> >> > > >>>> > 2. Sorry I forgot to mention in the response that I did add
> the
> >> > > >>>> > unclean.recovery.Enabled flag.
> >> > > >>>>
> >> > > >>>> The configuration key should be
> >> "unclean.recovery.manager.enabled",
> >> > > >>>> right? Becuase we can do "unclean recovery" without the
> manager.
> >> > > Disabling
> >> > > >>>> the manager just means we use a different mechanism for
> recovery.
> >> > > >>>>
> >> > > >>>> >        c. Maybe I underestimated the challenge of replacing
> the
> >> > > >>>> config. Any
> >> > > >>>> > implementation problems ahead?
> >> > > >>>>
> >> > > >>>> There are four levels of overrides for
> >> > unclean.leader.election.enable.
> >> > > >>>>
> >> > > >>>> 1. static configuration for node.
> >> > > >>>>     This goes in the configuration file, typically named
> >> > > >>>> server.properties
> >> > > >>>>
> >> > > >>>> 2. dynamic configuration for node default
> >> > > >>>>   ConfigResource(type=BROKER, name="")
> >> > > >>>>
> >> > > >>>> 3. dynamic configuration for node
> >> > > >>>>   ConfigResource(type=BROKER, name=<controller id>)
> >> > > >>>>
> >> > > >>>> 4. dynamic configuration for topic
> >> > > >>>>   ConfigResource(type=TOPIC, name=<topic-name>)
> >> > > >>>>
> >> > > >>>> How do these 4 levels of overrides interact with your new
> >> > > >>>> configurations? If the new configurations dominate over the old
> >> > ones,
> >> > > it
> >> > > >>>> seems like this will get a lot more confusing to implement (and
> >> also
> >> > > to
> >> > > >>>> use.)
> >> > > >>>>
> >> > > >>>> Again, I'd recommend just adding some new values to
> >> > > >>>> unclean.leader.election.enable. It's simple and will prevent
> user
> >> > > confusion
> >> > > >>>> (as well as developer confusion.)
> >> > > >>>>
> >> > > >>>> best,
> >> > > >>>> Colin
> >> > > >>>>
> >> > > >>>>
> >> > > >>>> > 3. About the admin client, I mentioned 3 changes in the
> client.
> >> > > >>>> Anything
> >> > > >>>> > else I missed in the KIP?
> >> > > >>>> >       a. The client will switch to using the new RPC instead
> of
> >> > > >>>> > MetadataRequest for the topics.
> >> > > >>>> >       b. The TopicPartitionInfo used in TopicDescription
> needs
> >> to
> >> > > add
> >> > > >>>> new
> >> > > >>>> > fields related to the ELR.
> >> > > >>>> >       c. The outputs will add the ELR related fields.
> >> > > >>>> >
> >> > > >>>> > On Thu, Sep 14, 2023 at 9:19 PM Colin McCabe <
> >> cmccabe@apache.org>
> >> > > >>>> wrote:
> >> > > >>>> >
> >> > > >>>> >> Hi Calvin,
> >> > > >>>> >>
> >> > > >>>> >> Thanks for the changes.
> >> > > >>>> >>
> >> > > >>>> >> 1. Earlier I commented that creating
> >> "unclean.recovery.strategy "
> >> > > is
> >> > > >>>> not
> >> > > >>>> >> necessary, and we can just reuse the existing
> >> > > >>>> >> "unclean.leader.election.enable" configuration key. Let's
> >> discuss
> >> > > >>>> that.
> >> > > >>>> >>
> >> > > >>>> >> 2.I also don't understand why you didn't add a
> configuration to
> >> > > >>>> enable or
> >> > > >>>> >> disable the Unclean Recovery Manager. This seems like a very
> >> > simple
> >> > > >>>> way to
> >> > > >>>> >> handle the staging issue which we discussed. The URM can
> just
> >> be
> >> > > >>>> turned off
> >> > > >>>> >> until it is production ready. Let's discuss this.
> >> > > >>>> >>
> >> > > >>>> >> 3. You still need to describe the changes to AdminClient
> that
> >> are
> >> > > >>>> needed
> >> > > >>>> >> to use DescribeTopicRequest.
> >> > > >>>> >>
> >> > > >>>> >> Keep at it. It's looking better. :)
> >> > > >>>> >>
> >> > > >>>> >> best,
> >> > > >>>> >> Colin
> >> > > >>>> >>
> >> > > >>>> >>
> >> > > >>>> >> On Thu, Sep 14, 2023, at 11:03, Calvin Liu wrote:
> >> > > >>>> >> > Hi Colin
> >> > > >>>> >> > Thanks for the comments!
> >> > > >>>> >> >
> >> > > >>>> >> > I did the following changes
> >> > > >>>> >> >
> >> > > >>>> >> >    1.
> >> > > >>>> >> >
> >> > > >>>> >> >    Simplified the API spec section to only include the
> diff.
> >> > > >>>> >> >    2.
> >> > > >>>> >> >
> >> > > >>>> >> >    Reordered the HWM requirement section.
> >> > > >>>> >> >    3.
> >> > > >>>> >> >
> >> > > >>>> >> >    Removed the URM implementation details to keep the
> >> necessary
> >> > > >>>> >> >    characteristics to perform the unclean recovery.
> >> > > >>>> >> >    1.
> >> > > >>>> >> >
> >> > > >>>> >> >       When to perform the unclean recovery
> >> > > >>>> >> >       2.
> >> > > >>>> >> >
> >> > > >>>> >> >       Under different config, how the unclean recovery
> finds
> >> > the
> >> > > >>>> leader.
> >> > > >>>> >> >       3.
> >> > > >>>> >> >
> >> > > >>>> >> >       How the config unclean.leader.election.enable and
> >> > > >>>> >> >       unclean.recovery.strategy are converted when users
> >> > > >>>> enable/disable
> >> > > >>>> >> the
> >> > > >>>> >> >       unclean recovery.
> >> > > >>>> >> >       4.
> >> > > >>>> >> >
> >> > > >>>> >> >    More details about how we change admin client.
> >> > > >>>> >> >    5.
> >> > > >>>> >> >
> >> > > >>>> >> >    API limits on the GetReplicaLogInfoRequest and
> >> > > >>>> DescribeTopicRequest.
> >> > > >>>> >> >    6.
> >> > > >>>> >> >
> >> > > >>>> >> >    Two metrics added
> >> > > >>>> >> >    1.
> >> > > >>>> >> >
> >> > > >>>> >> >
>  Kafka.controller.global_under_min_isr_partition_count
> >> > > >>>> >> >       2.
> >> > > >>>> >> >
> >> > > >>>> >> >       kafka.controller.unclean_recovery_finished_count
> >> > > >>>> >> >
> >> > > >>>> >> >
> >> > > >>>> >> > On Wed, Sep 13, 2023 at 10:46 AM Colin McCabe <
> >> > > cmccabe@apache.org>
> >> > > >>>> >> wrote:
> >> > > >>>> >> >
> >> > > >>>> >> >> On Tue, Sep 12, 2023, at 17:21, Calvin Liu wrote:
> >> > > >>>> >> >> > Hi Colin
> >> > > >>>> >> >> > Thanks for the comments!
> >> > > >>>> >> >> >
> >> > > >>>> >> >>
> >> > > >>>> >> >> Hi Calvin,
> >> > > >>>> >> >>
> >> > > >>>> >> >> Thanks again for the KIP.
> >> > > >>>> >> >>
> >> > > >>>> >> >> One meta-comment: it's usually better to just do a diff
> on a
> >> > > >>>> message
> >> > > >>>> >> spec
> >> > > >>>> >> >> file or java file if you're including changes to it in
> the
> >> > KIP.
> >> > > >>>> This is
> >> > > >>>> >> >> easier to read than looking for "new fields begin" etc.
> in
> >> the
> >> > > >>>> text, and
> >> > > >>>> >> >> gracefully handles the case where existing fields were
> >> > changed.
> >> > > >>>> >> >>
> >> > > >>>> >> >> > Rewrite the Additional High Watermark advancement
> >> > requirement
> >> > > >>>> >> >> > There was feedback on this section that some readers
> may
> >> not
> >> > > be
> >> > > >>>> >> familiar
> >> > > >>>> >> >> > with HWM and Ack=0,1,all requests. This can help them
> >> > > understand
> >> > > >>>> the
> >> > > >>>> >> >> > proposal. I will rewrite this part for more
> readability.
> >> > > >>>> >> >> >
> >> > > >>>> >> >>
> >> > > >>>> >> >> To be clear, I wasn't suggesting dropping either
> section. I
> >> > > agree
> >> > > >>>> that
> >> > > >>>> >> >> they add useful background. I was just suggesting that we
> >> > should
> >> > > >>>> discuss
> >> > > >>>> >> >> the "acks" setting AFTER discussing the new high
> watermark
> >> > > >>>> advancement
> >> > > >>>> >> >> conditions. We also should discuss acks=0. While it isn't
> >> > > >>>> conceptually
> >> > > >>>> >> much
> >> > > >>>> >> >> different than acks=1 here, its omission from this
> section
> >> is
> >> > > >>>> confusing.
> >> > > >>>> >> >>
> >> > > >>>> >> >> > Unclean recovery
> >> > > >>>> >> >> >
> >> > > >>>> >> >> > The plan is to replace the
> unclean.leader.election.enable
> >> > with
> >> > > >>>> >> >> > unclean.recovery.strategy. If the Unclean Recovery is
> >> > enabled
> >> > > >>>> then it
> >> > > >>>> >> >> deals
> >> > > >>>> >> >> > with the three options in the
> unclean.recovery.strategy.
> >> > > >>>> >> >> >
> >> > > >>>> >> >> >
> >> > > >>>> >> >> > Let’s refine the Unclean Recovery. We have already
> taken a
> >> > > lot of
> >> > > >>>> >> >> > suggestions and I hope to enhance the durability of
> Kafka
> >> to
> >> > > the
> >> > > >>>> next
> >> > > >>>> >> >> level
> >> > > >>>> >> >> > with this KIP.
> >> > > >>>> >> >>
> >> > > >>>> >> >> I am OK with doing the unclean leader recovery
> improvements
> >> in
> >> > > >>>> this KIP.
> >> > > >>>> >> >> However, I think we need to really work on the
> configuration
> >> > > >>>> settings.
> >> > > >>>> >> >>
> >> > > >>>> >> >> Configuration overrides are often quite messy. For
> example,
> >> > the
> >> > > >>>> cases
> >> > > >>>> >> >> where we have log.roll.hours and log.roll.segment.ms,
> the
> >> > user
> >> > > >>>> has to
> >> > > >>>> >> >> remember which one takes precedence, and it is not
> obvious.
> >> > So,
> >> > > >>>> rather
> >> > > >>>> >> than
> >> > > >>>> >> >> creating a new configuration, why not add additional
> values
> >> to
> >> > > >>>> >> >> "unclean.leader.election.enable"? I think this will be
> >> simpler
> >> > > for
> >> > > >>>> >> people
> >> > > >>>> >> >> to understand, and simpler in the code as well.
> >> > > >>>> >> >>
> >> > > >>>> >> >> What if we continued to use
> "unclean.leader.election.enable"
> >> > but
> >> > > >>>> >> extended
> >> > > >>>> >> >> it so that it took a string? Then the string could have
> >> these
> >> > > >>>> values:
> >> > > >>>> >> >>
> >> > > >>>> >> >> never
> >> > > >>>> >> >>     never automatically do an unclean leader election
> under
> >> > any
> >> > > >>>> >> conditions
> >> > > >>>> >> >>
> >> > > >>>> >> >> false / default
> >> > > >>>> >> >>     only do an unclean leader election if there may be
> >> > possible
> >> > > >>>> data
> >> > > >>>> >> loss
> >> > > >>>> >> >>
> >> > > >>>> >> >> true / always
> >> > > >>>> >> >>     always do an unclean leader election if we can't
> >> > immediately
> >> > > >>>> elect a
> >> > > >>>> >> >> leader
> >> > > >>>> >> >>
> >> > > >>>> >> >> It's a bit awkward that false maps to default rather
> than to
> >> > > >>>> never. But
> >> > > >>>> >> >> this awkwardness exists if we use two different
> >> configuration
> >> > > keys
> >> > > >>>> as
> >> > > >>>> >> well.
> >> > > >>>> >> >> The reason for the awkwardness is that we simply don't
> want
> >> > most
> >> > > >>>> of the
> >> > > >>>> >> >> people currently setting
> >> unclean.leader.election.enable=false
> >> > to
> >> > > >>>> get the
> >> > > >>>> >> >> "never" behavior. We have to bite that bullet. Better to
> be
> >> > > clear
> >> > > >>>> and
> >> > > >>>> >> >> explicit than hide it.
> >> > > >>>> >> >>
> >> > > >>>> >> >> Another thing that's a bit awkward is having two
> different
> >> > ways
> >> > > to
> >> > > >>>> do
> >> > > >>>> >> >> unclean leader election specified in the KIP. You
> descirbe
> >> two
> >> > > >>>> methods:
> >> > > >>>> >> the
> >> > > >>>> >> >> simple "choose the last leader" method, and the "unclean
> >> > > recovery
> >> > > >>>> >> manager"
> >> > > >>>> >> >> method. I understand why you did it this way -- "choose
> the
> >> > last
> >> > > >>>> >> leader" is
> >> > > >>>> >> >> simple, and will help us deliver an implementation
> quickly,
> >> > > while
> >> > > >>>> the
> >> > > >>>> >> URM
> >> > > >>>> >> >> is preferable in the long term. My suggestion here is to
> >> > > separate
> >> > > >>>> the
> >> > > >>>> >> >> decision of HOW to do unclean leader election from the
> >> > decision
> >> > > of
> >> > > >>>> WHEN
> >> > > >>>> >> to
> >> > > >>>> >> >> do it.
> >> > > >>>> >> >>
> >> > > >>>> >> >> So in other words, have "unclean.leader.election.enable"
> >> > specify
> >> > > >>>> when we
> >> > > >>>> >> >> do unclean leader election, and have a new configuration
> >> like
> >> > > >>>> >> >> "unclean.recovery.manager.enable" to determine if we use
> the
> >> > > URM.
> >> > > >>>> >> >> Presumably the URM will take some time to get fully
> stable,
> >> so
> >> > > >>>> this can
> >> > > >>>> >> >> default to false for a while, and we can flip the
> default to
> >> > > true
> >> > > >>>> when
> >> > > >>>> >> we
> >> > > >>>> >> >> feel ready.
> >> > > >>>> >> >>
> >> > > >>>> >> >> The URM is somewhat under-described here. I think we
> need a
> >> > few
> >> > > >>>> >> >> configurations here for it. For example, we need a
> >> > > configuration to
> >> > > >>>> >> specify
> >> > > >>>> >> >> how long it should wait for a broker to respond to its
> RPCs
> >> > > before
> >> > > >>>> >> moving
> >> > > >>>> >> >> on. We also need to understand how the URM interacts with
> >> > > >>>> >> >> unclean.leader.election.enable=always. I assume that with
> >> > > "always"
> >> > > >>>> we
> >> > > >>>> >> will
> >> > > >>>> >> >> just unconditionally use the URM rather than choosing
> >> > randomly.
> >> > > >>>> But this
> >> > > >>>> >> >> should be spelled out in the KIP.
> >> > > >>>> >> >>
> >> > > >>>> >> >> >
> >> > > >>>> >> >> > DescribeTopicRequest
> >> > > >>>> >> >> >
> >> > > >>>> >> >> >    1.
> >> > > >>>> >> >> >    Yes, the plan is to replace the MetadataRequest with
> >> the
> >> > > >>>> >> >> >    DescribeTopicRequest for the admin clients. Will
> check
> >> > the
> >> > > >>>> details.
> >> > > >>>> >> >>
> >> > > >>>> >> >> Sounds good. But as I said, you need to specify how
> >> > AdminClient
> >> > > >>>> >> interacts
> >> > > >>>> >> >> with the new request. This will involve adding some
> fields
> >> to
> >> > > >>>> >> >> TopicDescription.java. And you need to specify the
> changes
> >> to
> >> > > the
> >> > > >>>> >> >> kafka-topics.sh command line tool. Otherwise we cannot
> use
> >> the
> >> > > >>>> tool to
> >> > > >>>> >> see
> >> > > >>>> >> >> the new information.
> >> > > >>>> >> >>
> >> > > >>>> >> >> The new requests, DescribeTopicRequest and
> >> > > >>>> GetReplicaLogInfoRequest,
> >> > > >>>> >> need
> >> > > >>>> >> >> to have limits placed on them so that their size can't be
> >> > > >>>> infinite. We
> >> > > >>>> >> >> don't want to propagate the current problems of
> >> > MetadataRequest,
> >> > > >>>> where
> >> > > >>>> >> >> clients can request massive responses that can mess up
> the
> >> JVM
> >> > > when
> >> > > >>>> >> handled.
> >> > > >>>> >> >>
> >> > > >>>> >> >> Adding limits is simple for GetReplicaLogInfoRequest --
> we
> >> can
> >> > > >>>> just say
> >> > > >>>> >> >> that only 2000 partitions at a time can be requested. For
> >> > > >>>> >> >> DescribeTopicRequest we can probably just limit to 20
> topics
> >> > or
> >> > > >>>> >> something
> >> > > >>>> >> >> like that, to avoid the complexity of doing pagination in
> >> this
> >> > > KIP.
> >> > > >>>> >> >>
> >> > > >>>> >> >> >    2.
> >> > > >>>> >> >> >    I can let the broker load the ELR info so that they
> can
> >> > > serve
> >> > > >>>> the
> >> > > >>>> >> >> >    DescribeTopicRequest as well.
> >> > > >>>> >> >>
> >> > > >>>> >> >> Yes, it's fine to add to MetadataCache. In fact, you'll
> be
> >> > > loading
> >> > > >>>> it
> >> > > >>>> >> >> anyway once it's added to PartitionImage.
> >> > > >>>> >> >>
> >> > > >>>> >> >> >    3.
> >> > > >>>> >> >> >    Yeah, it does not make sense to have the topic id if
> >> > > >>>> >> >> >    DescribeTopicRequest is only used by the admin
> client.
> >> > > >>>> >> >>
> >> > > >>>> >> >> OK. That makes things simpler. We can always create a new
> >> API
> >> > > later
> >> > > >>>> >> >> (hopefully not in this KIP!) to query by topic ID.
> >> > > >>>> >> >>
> >> > > >>>> >> >> >
> >> > > >>>> >> >> >
> >> > > >>>> >> >> > Metrics
> >> > > >>>> >> >> >
> >> > > >>>> >> >> > As for overall cluster health metrics, I think
> >> under-min-ISR
> >> > > is
> >> > > >>>> still
> >> > > >>>> >> a
> >> > > >>>> >> >> > useful one. ELR is more like a safety belt. When the
> ELR
> >> is
> >> > > >>>> used, the
> >> > > >>>> >> >> > cluster availability has already been impacted.
> >> > > >>>> >> >> >
> >> > > >>>> >> >> > Maybe we can have a metric to count the partitions that
> >> > > sum(ISR,
> >> > > >>>> ELR)
> >> > > >>>> >> <
> >> > > >>>> >> >> min
> >> > > >>>> >> >> > ISR. What do you think?
> >> > > >>>> >> >>
> >> > > >>>> >> >> How about:
> >> > > >>>> >> >>
> >> > > >>>> >> >> A.  a metric for the totoal number of under-min-isr
> >> > partitions?
> >> > > We
> >> > > >>>> don't
> >> > > >>>> >> >> have that in Apache Kafka at the moment.
> >> > > >>>> >> >>
> >> > > >>>> >> >> B. a metric for the number of unclean leader elections we
> >> did
> >> > > (for
> >> > > >>>> >> >> simplicity, it can reset to 0 on controller restart: we
> >> expect
> >> > > >>>> people to
> >> > > >>>> >> >> monitor the change over time anyway)
> >> > > >>>> >> >>
> >> > > >>>> >> >> best,
> >> > > >>>> >> >> Colin
> >> > > >>>> >> >>
> >> > > >>>> >> >>
> >> > > >>>> >> >> >
> >> > > >>>> >> >> > Yeah, for the ongoing unclean recoveries, the
> controller
> >> can
> >> > > >>>> keep an
> >> > > >>>> >> >> > accurate count through failover because partition
> >> > registration
> >> > > >>>> can
> >> > > >>>> >> >> indicate
> >> > > >>>> >> >> > whether a recovery is needed. However, for the happened
> >> > ones,
> >> > > >>>> unless
> >> > > >>>> >> we
> >> > > >>>> >> >> > want to persist the number somewhere, we can only
> figure
> >> it
> >> > > out
> >> > > >>>> from
> >> > > >>>> >> the
> >> > > >>>> >> >> > log.
> >> > > >>>> >> >> >
> >> > > >>>> >> >> > On Tue, Sep 12, 2023 at 3:16 PM Colin McCabe <
> >> > > cmccabe@apache.org
> >> > > >>>> >
> >> > > >>>> >> wrote:
> >> > > >>>> >> >> >
> >> > > >>>> >> >> >> Also, we should have metrics that show what is going
> on
> >> > with
> >> > > >>>> regard
> >> > > >>>> >> to
> >> > > >>>> >> >> the
> >> > > >>>> >> >> >> eligible replica set. I'm not sure exactly what to
> >> suggest,
> >> > > but
> >> > > >>>> >> >> something
> >> > > >>>> >> >> >> that could identify when things are going wrong in the
> >> > > clsuter.
> >> > > >>>> >> >> >>
> >> > > >>>> >> >> >> For example, maybe a metric for partitions containing
> >> > > replicas
> >> > > >>>> that
> >> > > >>>> >> are
> >> > > >>>> >> >> >> ineligible to be leader? That would show a spike when
> a
> >> > > broker
> >> > > >>>> had an
> >> > > >>>> >> >> >> unclean restart.
> >> > > >>>> >> >> >>
> >> > > >>>> >> >> >> Ideally, we'd also have a metric that indicates when
> an
> >> > > unclear
> >> > > >>>> >> leader
> >> > > >>>> >> >> >> election or a recovery happened. It's a bit tricky
> >> because
> >> > > the
> >> > > >>>> simple
> >> > > >>>> >> >> >> thing, of tracking it per controller, may be a bit
> >> > confusing
> >> > > >>>> during
> >> > > >>>> >> >> >> failovers.
> >> > > >>>> >> >> >>
> >> > > >>>> >> >> >> best,
> >> > > >>>> >> >> >> Colin
> >> > > >>>> >> >> >>
> >> > > >>>> >> >> >>
> >> > > >>>> >> >> >> On Tue, Sep 12, 2023, at 14:25, Colin McCabe wrote:
> >> > > >>>> >> >> >> > Hi Calvin,
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> > Thanks for the KIP. I think this is a great
> >> improvement.
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> >> Additional High Watermark advance requirement
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> > Typo: change "advance" to "advancement"
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> >> A bit recap of some key concepts.
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> > Typo: change "bit" to "quick"
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> >> Ack=1/all produce request. It defines when the
> Kafka
> >> > > server
> >> > > >>>> should
> >> > > >>>> >> >> >> respond to the produce request
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> > I think this section would be clearer if we talked
> >> about
> >> > > the
> >> > > >>>> new
> >> > > >>>> >> high
> >> > > >>>> >> >> >> > watermark advancement requirement first, and THEN
> >> talked
> >> > > >>>> about its
> >> > > >>>> >> >> >> > impact on acks=0, acks=1, and     acks=all.
> acks=all
> >> is
> >> > of
> >> > > >>>> course
> >> > > >>>> >> the
> >> > > >>>> >> >> >> > main case we care about here, so it would be good to
> >> lead
> >> > > with
> >> > > >>>> >> that,
> >> > > >>>> >> >> >> > rather than delving into the technicalities of
> acks=0/1
> >> > > first.
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> >> Unclean recovery
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> > So, here you are introducing a new configuration,
> >> > > >>>> >> >> >> > unclean.recovery.strategy. The difficult thing here
> is
> >> > that
> >> > > >>>> there
> >> > > >>>> >> is a
> >> > > >>>> >> >> >> > lot of overlap with unclean.leader.election.enable.
> So
> >> we
> >> > > >>>> have 3
> >> > > >>>> >> >> >> > different settings for unclean.recovery.strategy,
> plus
> >> 2
> >> > > >>>> different
> >> > > >>>> >> >> >> > settings for unclean.leader.election.enable, giving
> a
> >> > cross
> >> > > >>>> >> product of
> >> > > >>>> >> >> >> > 6 different options. The following "unclean recovery
> >> > > manager"
> >> > > >>>> >> section
> >> > > >>>> >> >> >> > only applies to one fo those 6 different
> possibilities
> >> (I
> >> > > >>>> think?)
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> > I simply don't think we need so many different
> election
> >> > > types.
> >> > > >>>> >> Really
> >> > > >>>> >> >> >> > the use-cases we need are people who want NO unclean
> >> > > >>>> elections,
> >> > > >>>> >> people
> >> > > >>>> >> >> >> > who want "the reasonable thing" and people who want
> >> > > >>>> avaialbility at
> >> > > >>>> >> >> all
> >> > > >>>> >> >> >> > costs.
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> > Overall, I feel like the first half of the KIP is
> about
> >> > the
> >> > > >>>> ELR,
> >> > > >>>> >> and
> >> > > >>>> >> >> >> > the second half is about reworking unclean leader
> >> > > election. It
> >> > > >>>> >> might
> >> > > >>>> >> >> be
> >> > > >>>> >> >> >> > better to move that second half to a separate KIP so
> >> that
> >> > > we
> >> > > >>>> can
> >> > > >>>> >> >> figure
> >> > > >>>> >> >> >> > it out fully. It should be fine to punt this until
> >> later
> >> > > and
> >> > > >>>> just
> >> > > >>>> >> have
> >> > > >>>> >> >> >> > the current behavior on empty ELR be waiting for the
> >> last
> >> > > >>>> known
> >> > > >>>> >> leader
> >> > > >>>> >> >> >> > to return. After all, that's what we do today.
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> >> DescribeTopicRequest
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> > Is the intention for AdminClient to use this RPC for
> >> > > >>>> >> >> >> > Admin#describeTopics ? If so, we need to describe
> all
> >> of
> >> > > the
> >> > > >>>> >> changes
> >> > > >>>> >> >> to
> >> > > >>>> >> >> >> > the admin client API, as well as changes to
> >> command-line
> >> > > >>>> tools like
> >> > > >>>> >> >> >> > kafka-topics.sh (if there are any). For example, you
> >> will
> >> > > >>>> probably
> >> > > >>>> >> >> need
> >> > > >>>> >> >> >> > changes to TopicDescription.java. You will also
> need to
> >> > > >>>> provide
> >> > > >>>> >> all of
> >> > > >>>> >> >> >> > the things that admin client needs -- for example,
> >> > > >>>> >> >> >> > TopicAuthorizedOperations.
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> > I also don't think the controller should serve this
> >> > > request.
> >> > > >>>> We
> >> > > >>>> >> want
> >> > > >>>> >> >> to
> >> > > >>>> >> >> >> > minimize load on the controller. Just like with the
> >> other
> >> > > >>>> metadata
> >> > > >>>> >> >> >> > requests like MetadataRequest, this should be
> served by
> >> > > >>>> brokers.
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> > It's a bit confusing why both topic ID and topic
> name
> >> are
> >> > > >>>> provided
> >> > > >>>> >> to
> >> > > >>>> >> >> >> > this API. Is the intention that callers should set
> one
> >> > but
> >> > > >>>> not the
> >> > > >>>> >> >> >> > other? Or both? This needs to be clarified. Also,
> if we
> >> > do
> >> > > >>>> want to
> >> > > >>>> >> >> >> > support lookups by UUID, that is another thing that
> >> needs
> >> > > to
> >> > > >>>> be
> >> > > >>>> >> added
> >> > > >>>> >> >> >> > to adminclient.
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> > In general, I feel like this should also probably be
> >> its
> >> > > own
> >> > > >>>> KIP
> >> > > >>>> >> since
> >> > > >>>> >> >> >> > it's fairly complex
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> > best,
> >> > > >>>> >> >> >> > Colin
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> > On Thu, Aug 10, 2023, at 15:46, Calvin Liu 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
> >> > > >>>> >> >> >>
> >> > > >>>> >> >>
> >> > > >>>> >>
> >> > > >>>>
> >> > > >>>
> >> > >
> >> >
> >>
>


-- 
-David

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Artem Livshits <al...@confluent.io.INVALID>.
Hi Colin,

I think in your example "do_unclean_recovery" would need to do different
things depending on the strategy.

do_unclean_recovery() {
   if (unclean.recovery.manager.enabled) {
    if (strategy == Aggressive)
      use UncleanRecoveryManager(waitLastKnownERL=false)  // just inspect
logs from whoever is available
    else
      use  UncleanRecoveryManager(waitLastKnownERL=true)  // must wait for
at least last known ELR
  } else {
    if (strategy == Aggressive)
      choose the last known leader if that is available, or a random leader
if not)
    else
      wait for last known leader to get back
  }
}

The idea is that the Aggressive strategy would kick in as soon as we lost
the leader and would pick a leader from whoever is available; but the
Balanced will only kick in when ELR is empty and will wait for the brokers
that likely have most data to be available.

On Tue, Oct 3, 2023 at 3:04 PM Colin McCabe <cm...@apache.org> wrote:

> On Tue, Oct 3, 2023, at 10:49, Jun Rao wrote:
> > Hi, Calvin,
> >
> > Thanks for the update KIP. A few more comments.
> >
> > 41. Why would a user choose the option to select a random replica as the
> > leader instead of using unclean.recovery.strateg=Aggressive? It seems
> that
> > the latter is strictly better? If that's not the case, could we fold this
> > option under unclean.recovery.strategy instead of introducing a separate
> > config?
>
> Hi Jun,
>
> I thought the flow of control was:
>
> If there is no leader for the partition {
>   If (there are unfenced ELR members) {
>     choose_an_unfenced_ELR_member
>   } else if (there are fenced ELR members AND strategy=Aggressive) {
>     do_unclean_recovery
>   } else if (there are no ELR members AND strategy != None) {
>     do_unclean_recovery
>   } else {
>     do nothing about the missing leader
>   }
> }
>
> do_unclean_recovery() {
>    if (unclean.recovery.manager.enabled) {
>     use UncleanRecoveryManager
>   } else {
>     choose the last known leader if that is available, or a random leader
> if not)
>   }
> }
>
> However, I think this could be clarified, especially the behavior when
> unclean.recovery.manager.enabled=false. Inuitively the goal for
> unclean.recovery.manager.enabled=false is to be "the same as now, mostly"
> but it's very underspecified in the KIP, I agree.
>
> >
> > 50. ElectLeadersRequest: "If more than 20 topics are included, only the
> > first 20 will be served. Others will be returned with DesiredLeaders."
> Hmm,
> > not sure that I understand this. ElectLeadersResponse doesn't have a
> > DesiredLeaders field.
> >
> > 51. GetReplicaLogInfo: "If more than 2000 partitions are included, only
> the
> > first 2000 will be served" Do we return an error for the remaining
> > partitions? Actually, should we include an errorCode field at the
> partition
> > level in GetReplicaLogInfoResponse to cover non-existing partitions and
> no
> > authorization, etc?
> >
> > 52. The entry should matches => The entry should match
> >
> > 53. ElectLeadersRequest.DesiredLeaders: Should it be nullable since a
> user
> > may not specify DesiredLeaders?
> >
> > 54. Downgrade: Is that indeed possible? I thought earlier you said that
> > once the new version of the records are in the metadata log, one can't
> > downgrade since the old broker doesn't know how to parse the new version
> of
> > the metadata records?
> >
>
> MetadataVersion downgrade is currently broken but we have fixing it on our
> plate for Kafka 3.7.
>
> The way downgrade works is that "new features" are dropped, leaving only
> the old ones.
>
> > 55. CleanShutdownFile: Should we add a version field for future
> extension?
> >
> > 56. Config changes are public facing. Could we have a separate section to
> > document all the config changes?
>
> +1. A separate section for this would be good.
>
> best,
> Colin
>
> >
> > Thanks,
> >
> > Jun
> >
> > On Mon, Sep 25, 2023 at 4:29 PM Calvin Liu <ca...@confluent.io.invalid>
> > wrote:
> >
> >> Hi Jun
> >> Thanks for the comments.
> >>
> >> 40. If we change to None, it is not guaranteed for no data loss. For
> users
> >> who are not able to validate the data with external resources, manual
> >> intervention does not give a better result but a loss of availability.
> So
> >> practically speaking, the Balance mode would be a better default value.
> >>
> >> 41. No, it represents how we want to do the unclean leader election. If
> it
> >> is false, the unclean leader election will be the old random way.
> >> Otherwise, the unclean recovery will be used.
> >>
> >> 42. Good catch. Updated.
> >>
> >> 43. Only the first 20 topics will be served. Others will be returned
> with
> >> InvalidRequestError
> >>
> >> 44. The order matters. The desired leader entries match with the topic
> >> partition list by the index.
> >>
> >> 45. Thanks! Updated.
> >>
> >> 46. Good advice! Updated.
> >>
> >> 47.1, updated the comment. Basically it will elect the replica in the
> >> desiredLeader field to be the leader
> >>
> >> 47.2 We can let the admin client do the conversion. Using the
> desiredLeader
> >> field in the json format seems easier for users.
> >>
> >> 48. Once the MV version is downgraded, all the ELR related fields will
> be
> >> removed on the next partition change. The controller will also ignore
> the
> >> ELR fields. Updated the KIP.
> >>
> >> 49. Yes, it would be deprecated/removed.
> >>
> >>
> >> On Mon, Sep 25, 2023 at 3:49 PM Jun Rao <ju...@confluent.io.invalid>
> wrote:
> >>
> >> > Hi, Calvin,
> >> >
> >> > Thanks for the updated KIP. Made another pass. A few more comments
> below.
> >> >
> >> > 40. unclean.leader.election.enable.false ->
> >> > unclean.recovery.strategy.Balanced: The Balanced mode could still
> lead to
> >> > data loss. So, I am wondering if unclean.leader.election.enable.false
> >> > should map to None?
> >> >
> >> > 41. unclean.recovery.manager.enabled: I am not sure why we introduce
> this
> >> > additional config. Is it the same as unclean.recovery.strategy=None?
> >> >
> >> > 42. DescribeTopicResponse.TopicAuthorizedOperations: Should this be at
> >> the
> >> > topic level?
> >> >
> >> > 43. "Limit: 20 topics max per request": Could we describe what
> happens if
> >> > the request includes more than 20 topics?
> >> >
> >> > 44. ElectLeadersRequest.DesiredLeaders: Could we describe whether the
> >> > ordering matters?
> >> >
> >> > 45. GetReplicaLogInfo.TopicPartitions: "about": "The topic partitions
> to
> >> > elect leaders.": The description in "about" is incorrect.
> >> >
> >> > 46. GetReplicaLogInfoResponse: Should we nest partitions under
> topicId to
> >> > be consistent with other types of responses?
> >> >
> >> > 47. kafka-leader-election.sh:
> >> > 47.1 Could we explain DESIGNATION?
> >> > 47.2 desiredLeader: Should it be a list to match the field in
> >> > ElectLeadersRequest?
> >> >
> >> > 48. We could add a section on downgrade?
> >> >
> >> > 49. LastKnownLeader: This seems only needed in the first phase of
> >> > delivering ELR. Will it be removed when the complete KIP is delivered?
> >> >
> >> > Thanks,
> >> >
> >> > Jun
> >> >
> >> > On Tue, Sep 19, 2023 at 1:30 PM Colin McCabe <cm...@apache.org>
> wrote:
> >> >
> >> > > Hi Calvin,
> >> > >
> >> > > Thanks for the explanations. I like the idea of using none,
> balanced,
> >> > > aggressive. We also had an offline discussion about why it is good
> to
> >> > use a
> >> > > new config key (basically, so that we can deprecate the old one
> which
> >> had
> >> > > only false/true values in 4.0) With these changes, I am +1.
> >> > >
> >> > > best,
> >> > > Colin
> >> > >
> >> > > On Mon, Sep 18, 2023, at 15:54, Calvin Liu wrote:
> >> > > > Hi Colin,
> >> > > > Also, can we deprecate unclean.leader.election.enable in 4.0?
> Before
> >> > > that,
> >> > > > we can have both the config unclean.recovery.strategy and
> >> > > > unclean.leader.election.enable
> >> > > > and using the unclean.recovery.Enabled to determine which config
> to
> >> use
> >> > > > during the unclean leader election.
> >> > > >
> >> > > > On Mon, Sep 18, 2023 at 3:51 PM Calvin Liu <ca...@confluent.io>
> >> wrote:
> >> > > >
> >> > > >> Hi Colin,
> >> > > >> For the unclean.recovery.strategy config name, how about we use
> the
> >> > > >> following
> >> > > >> None. It basically means no unclean recovery will be performed.
> >> > > >> Aggressive. It means availability goes first. Whenever the
> partition
> >> > > can't
> >> > > >> elect a durable replica, the controller will try the unclean
> >> recovery.
> >> > > >> Balanced. It is the balance point of the availability
> >> > first(Aggressive)
> >> > > >> and least availability(None). The controller performs unclean
> >> recovery
> >> > > when
> >> > > >> both ISR and ELR are empty.
> >> > > >>
> >> > > >>
> >> > > >> On Fri, Sep 15, 2023 at 11:42 AM Calvin Liu <ca...@confluent.io>
> >> > wrote:
> >> > > >>
> >> > > >>> Hi Colin,
> >> > > >>>
> >> > > >>> > So, the proposal is that if someone sets
> >> > > "unclean.leader.election.enable
> >> > > >>> = true"...
> >> > > >>>
> >> > > >>>
> >> > > >>> The idea is to use one of the unclean.leader.election.enable and
> >> > > >>> unclean.recovery.strategy based on the
> unclean.recovery.Enabled. A
> >> > > possible
> >> > > >>> version can be
> >> > > >>>
> >> > > >>> If unclean.recovery.Enabled:
> >> > > >>>
> >> > > >>> {
> >> > > >>>
> >> > > >>> Check unclean.recovery.strategy. If set, use it. Otherwise,
> check
> >> > > >>> unclean.leader.election.enable and translate it to
> >> > > >>> unclean.recovery.strategy.
> >> > > >>>
> >> > > >>> } else {
> >> > > >>>
> >> > > >>> Use unclean.leader.election.enable
> >> > > >>>
> >> > > >>> }
> >> > > >>>
> >> > > >>>
> >> > > >>> —--------
> >> > > >>>
> >> > > >>> >The configuration key should be
> >> "unclean.recovery.manager.enabled",
> >> > > >>> right?
> >> > > >>>
> >> > > >>>
> >> > > >>> I think we have two ways of choosing a leader uncleanly, unclean
> >> > leader
> >> > > >>> election and unclean recovery(log inspection) and we try to
> switch
> >> > > between
> >> > > >>> them.
> >> > > >>>
> >> > > >>> Do you mean we want to develop two ways of performing the
> unclean
> >> > > >>> recovery and one of them is using “unclean recovery manager”? I
> >> guess
> >> > > we
> >> > > >>> haven’t discussed the second way.
> >> > > >>>
> >> > > >>>
> >> > > >>> —-------
> >> > > >>>
> >> > > >>> >How do these 4 levels of overrides interact with your new
> >> > > >>> configurations?
> >> > > >>>
> >> > > >>>
> >> > > >>> I do notice in the Kraft controller code, the method to check
> >> whether
> >> > > >>> perform unclean leader election is hard coded to false since
> >> > > >>> 2021(uncleanLeaderElectionEnabledForTopic). Isn’t it a good
> chance
> >> to
> >> > > >>> completely deprecate the unclean.leader.election.enable? We
> don’t
> >> > even
> >> > > have
> >> > > >>> to worry about the config conversion.
> >> > > >>>
> >> > > >>> On the other hand, whatever the override is, as long as the
> >> > controller
> >> > > >>> can have the final effective unclean.leader.election.enable, the
> >> > topic
> >> > > >>> level config unclean.recovery.strategy, the cluster level config
> >> > > >>> unclean.recovery.Enabled, the controller can calculate the
> correct
> >> > > methods
> >> > > >>> to use right?
> >> > > >>>
> >> > > >>>
> >> > > >>> On Fri, Sep 15, 2023 at 10:02 AM Colin McCabe <
> cmccabe@apache.org>
> >> > > wrote:
> >> > > >>>
> >> > > >>>> On Thu, Sep 14, 2023, at 22:23, Calvin Liu wrote:
> >> > > >>>> > Hi Colin
> >> > > >>>> > 1. I think using the new config name is more clear.
> >> > > >>>> >        a. The unclean leader election is actually removed if
> >> > unclean
> >> > > >>>> > recovery is in use.
> >> > > >>>> >        b. Using multiple values in
> >> unclean.leader.election.enable
> >> > is
> >> > > >>>> > confusing and it will be more confusing after people forget
> >> about
> >> > > this
> >> > > >>>> > discussion.
> >> > > >>>>
> >> > > >>>> Hi Calvin,
> >> > > >>>>
> >> > > >>>> So, the proposal is that if someone sets
> >> > > "unclean.leader.election.enable
> >> > > >>>> = true" but then sets one of your new configurations, the
> value of
> >> > > >>>> unclean.leader.election.enable is ignored? That seems less
> clear
> >> to
> >> > > me, not
> >> > > >>>> more. Just in general, having multiple configuration keys to
> >> control
> >> > > the
> >> > > >>>> same thing confuses users. Basically, they are sitting at a
> giant
> >> > > control
> >> > > >>>> panel, and some of the levers do nothing.
> >> > > >>>>
> >> > > >>>> > 2. Sorry I forgot to mention in the response that I did add
> the
> >> > > >>>> > unclean.recovery.Enabled flag.
> >> > > >>>>
> >> > > >>>> The configuration key should be
> >> "unclean.recovery.manager.enabled",
> >> > > >>>> right? Becuase we can do "unclean recovery" without the
> manager.
> >> > > Disabling
> >> > > >>>> the manager just means we use a different mechanism for
> recovery.
> >> > > >>>>
> >> > > >>>> >        c. Maybe I underestimated the challenge of replacing
> the
> >> > > >>>> config. Any
> >> > > >>>> > implementation problems ahead?
> >> > > >>>>
> >> > > >>>> There are four levels of overrides for
> >> > unclean.leader.election.enable.
> >> > > >>>>
> >> > > >>>> 1. static configuration for node.
> >> > > >>>>     This goes in the configuration file, typically named
> >> > > >>>> server.properties
> >> > > >>>>
> >> > > >>>> 2. dynamic configuration for node default
> >> > > >>>>   ConfigResource(type=BROKER, name="")
> >> > > >>>>
> >> > > >>>> 3. dynamic configuration for node
> >> > > >>>>   ConfigResource(type=BROKER, name=<controller id>)
> >> > > >>>>
> >> > > >>>> 4. dynamic configuration for topic
> >> > > >>>>   ConfigResource(type=TOPIC, name=<topic-name>)
> >> > > >>>>
> >> > > >>>> How do these 4 levels of overrides interact with your new
> >> > > >>>> configurations? If the new configurations dominate over the old
> >> > ones,
> >> > > it
> >> > > >>>> seems like this will get a lot more confusing to implement (and
> >> also
> >> > > to
> >> > > >>>> use.)
> >> > > >>>>
> >> > > >>>> Again, I'd recommend just adding some new values to
> >> > > >>>> unclean.leader.election.enable. It's simple and will prevent
> user
> >> > > confusion
> >> > > >>>> (as well as developer confusion.)
> >> > > >>>>
> >> > > >>>> best,
> >> > > >>>> Colin
> >> > > >>>>
> >> > > >>>>
> >> > > >>>> > 3. About the admin client, I mentioned 3 changes in the
> client.
> >> > > >>>> Anything
> >> > > >>>> > else I missed in the KIP?
> >> > > >>>> >       a. The client will switch to using the new RPC instead
> of
> >> > > >>>> > MetadataRequest for the topics.
> >> > > >>>> >       b. The TopicPartitionInfo used in TopicDescription
> needs
> >> to
> >> > > add
> >> > > >>>> new
> >> > > >>>> > fields related to the ELR.
> >> > > >>>> >       c. The outputs will add the ELR related fields.
> >> > > >>>> >
> >> > > >>>> > On Thu, Sep 14, 2023 at 9:19 PM Colin McCabe <
> >> cmccabe@apache.org>
> >> > > >>>> wrote:
> >> > > >>>> >
> >> > > >>>> >> Hi Calvin,
> >> > > >>>> >>
> >> > > >>>> >> Thanks for the changes.
> >> > > >>>> >>
> >> > > >>>> >> 1. Earlier I commented that creating
> >> "unclean.recovery.strategy "
> >> > > is
> >> > > >>>> not
> >> > > >>>> >> necessary, and we can just reuse the existing
> >> > > >>>> >> "unclean.leader.election.enable" configuration key. Let's
> >> discuss
> >> > > >>>> that.
> >> > > >>>> >>
> >> > > >>>> >> 2.I also don't understand why you didn't add a
> configuration to
> >> > > >>>> enable or
> >> > > >>>> >> disable the Unclean Recovery Manager. This seems like a very
> >> > simple
> >> > > >>>> way to
> >> > > >>>> >> handle the staging issue which we discussed. The URM can
> just
> >> be
> >> > > >>>> turned off
> >> > > >>>> >> until it is production ready. Let's discuss this.
> >> > > >>>> >>
> >> > > >>>> >> 3. You still need to describe the changes to AdminClient
> that
> >> are
> >> > > >>>> needed
> >> > > >>>> >> to use DescribeTopicRequest.
> >> > > >>>> >>
> >> > > >>>> >> Keep at it. It's looking better. :)
> >> > > >>>> >>
> >> > > >>>> >> best,
> >> > > >>>> >> Colin
> >> > > >>>> >>
> >> > > >>>> >>
> >> > > >>>> >> On Thu, Sep 14, 2023, at 11:03, Calvin Liu wrote:
> >> > > >>>> >> > Hi Colin
> >> > > >>>> >> > Thanks for the comments!
> >> > > >>>> >> >
> >> > > >>>> >> > I did the following changes
> >> > > >>>> >> >
> >> > > >>>> >> >    1.
> >> > > >>>> >> >
> >> > > >>>> >> >    Simplified the API spec section to only include the
> diff.
> >> > > >>>> >> >    2.
> >> > > >>>> >> >
> >> > > >>>> >> >    Reordered the HWM requirement section.
> >> > > >>>> >> >    3.
> >> > > >>>> >> >
> >> > > >>>> >> >    Removed the URM implementation details to keep the
> >> necessary
> >> > > >>>> >> >    characteristics to perform the unclean recovery.
> >> > > >>>> >> >    1.
> >> > > >>>> >> >
> >> > > >>>> >> >       When to perform the unclean recovery
> >> > > >>>> >> >       2.
> >> > > >>>> >> >
> >> > > >>>> >> >       Under different config, how the unclean recovery
> finds
> >> > the
> >> > > >>>> leader.
> >> > > >>>> >> >       3.
> >> > > >>>> >> >
> >> > > >>>> >> >       How the config unclean.leader.election.enable and
> >> > > >>>> >> >       unclean.recovery.strategy are converted when users
> >> > > >>>> enable/disable
> >> > > >>>> >> the
> >> > > >>>> >> >       unclean recovery.
> >> > > >>>> >> >       4.
> >> > > >>>> >> >
> >> > > >>>> >> >    More details about how we change admin client.
> >> > > >>>> >> >    5.
> >> > > >>>> >> >
> >> > > >>>> >> >    API limits on the GetReplicaLogInfoRequest and
> >> > > >>>> DescribeTopicRequest.
> >> > > >>>> >> >    6.
> >> > > >>>> >> >
> >> > > >>>> >> >    Two metrics added
> >> > > >>>> >> >    1.
> >> > > >>>> >> >
> >> > > >>>> >> >
>  Kafka.controller.global_under_min_isr_partition_count
> >> > > >>>> >> >       2.
> >> > > >>>> >> >
> >> > > >>>> >> >       kafka.controller.unclean_recovery_finished_count
> >> > > >>>> >> >
> >> > > >>>> >> >
> >> > > >>>> >> > On Wed, Sep 13, 2023 at 10:46 AM Colin McCabe <
> >> > > cmccabe@apache.org>
> >> > > >>>> >> wrote:
> >> > > >>>> >> >
> >> > > >>>> >> >> On Tue, Sep 12, 2023, at 17:21, Calvin Liu wrote:
> >> > > >>>> >> >> > Hi Colin
> >> > > >>>> >> >> > Thanks for the comments!
> >> > > >>>> >> >> >
> >> > > >>>> >> >>
> >> > > >>>> >> >> Hi Calvin,
> >> > > >>>> >> >>
> >> > > >>>> >> >> Thanks again for the KIP.
> >> > > >>>> >> >>
> >> > > >>>> >> >> One meta-comment: it's usually better to just do a diff
> on a
> >> > > >>>> message
> >> > > >>>> >> spec
> >> > > >>>> >> >> file or java file if you're including changes to it in
> the
> >> > KIP.
> >> > > >>>> This is
> >> > > >>>> >> >> easier to read than looking for "new fields begin" etc.
> in
> >> the
> >> > > >>>> text, and
> >> > > >>>> >> >> gracefully handles the case where existing fields were
> >> > changed.
> >> > > >>>> >> >>
> >> > > >>>> >> >> > Rewrite the Additional High Watermark advancement
> >> > requirement
> >> > > >>>> >> >> > There was feedback on this section that some readers
> may
> >> not
> >> > > be
> >> > > >>>> >> familiar
> >> > > >>>> >> >> > with HWM and Ack=0,1,all requests. This can help them
> >> > > understand
> >> > > >>>> the
> >> > > >>>> >> >> > proposal. I will rewrite this part for more
> readability.
> >> > > >>>> >> >> >
> >> > > >>>> >> >>
> >> > > >>>> >> >> To be clear, I wasn't suggesting dropping either
> section. I
> >> > > agree
> >> > > >>>> that
> >> > > >>>> >> >> they add useful background. I was just suggesting that we
> >> > should
> >> > > >>>> discuss
> >> > > >>>> >> >> the "acks" setting AFTER discussing the new high
> watermark
> >> > > >>>> advancement
> >> > > >>>> >> >> conditions. We also should discuss acks=0. While it isn't
> >> > > >>>> conceptually
> >> > > >>>> >> much
> >> > > >>>> >> >> different than acks=1 here, its omission from this
> section
> >> is
> >> > > >>>> confusing.
> >> > > >>>> >> >>
> >> > > >>>> >> >> > Unclean recovery
> >> > > >>>> >> >> >
> >> > > >>>> >> >> > The plan is to replace the
> unclean.leader.election.enable
> >> > with
> >> > > >>>> >> >> > unclean.recovery.strategy. If the Unclean Recovery is
> >> > enabled
> >> > > >>>> then it
> >> > > >>>> >> >> deals
> >> > > >>>> >> >> > with the three options in the
> unclean.recovery.strategy.
> >> > > >>>> >> >> >
> >> > > >>>> >> >> >
> >> > > >>>> >> >> > Let’s refine the Unclean Recovery. We have already
> taken a
> >> > > lot of
> >> > > >>>> >> >> > suggestions and I hope to enhance the durability of
> Kafka
> >> to
> >> > > the
> >> > > >>>> next
> >> > > >>>> >> >> level
> >> > > >>>> >> >> > with this KIP.
> >> > > >>>> >> >>
> >> > > >>>> >> >> I am OK with doing the unclean leader recovery
> improvements
> >> in
> >> > > >>>> this KIP.
> >> > > >>>> >> >> However, I think we need to really work on the
> configuration
> >> > > >>>> settings.
> >> > > >>>> >> >>
> >> > > >>>> >> >> Configuration overrides are often quite messy. For
> example,
> >> > the
> >> > > >>>> cases
> >> > > >>>> >> >> where we have log.roll.hours and log.roll.segment.ms,
> the
> >> > user
> >> > > >>>> has to
> >> > > >>>> >> >> remember which one takes precedence, and it is not
> obvious.
> >> > So,
> >> > > >>>> rather
> >> > > >>>> >> than
> >> > > >>>> >> >> creating a new configuration, why not add additional
> values
> >> to
> >> > > >>>> >> >> "unclean.leader.election.enable"? I think this will be
> >> simpler
> >> > > for
> >> > > >>>> >> people
> >> > > >>>> >> >> to understand, and simpler in the code as well.
> >> > > >>>> >> >>
> >> > > >>>> >> >> What if we continued to use
> "unclean.leader.election.enable"
> >> > but
> >> > > >>>> >> extended
> >> > > >>>> >> >> it so that it took a string? Then the string could have
> >> these
> >> > > >>>> values:
> >> > > >>>> >> >>
> >> > > >>>> >> >> never
> >> > > >>>> >> >>     never automatically do an unclean leader election
> under
> >> > any
> >> > > >>>> >> conditions
> >> > > >>>> >> >>
> >> > > >>>> >> >> false / default
> >> > > >>>> >> >>     only do an unclean leader election if there may be
> >> > possible
> >> > > >>>> data
> >> > > >>>> >> loss
> >> > > >>>> >> >>
> >> > > >>>> >> >> true / always
> >> > > >>>> >> >>     always do an unclean leader election if we can't
> >> > immediately
> >> > > >>>> elect a
> >> > > >>>> >> >> leader
> >> > > >>>> >> >>
> >> > > >>>> >> >> It's a bit awkward that false maps to default rather
> than to
> >> > > >>>> never. But
> >> > > >>>> >> >> this awkwardness exists if we use two different
> >> configuration
> >> > > keys
> >> > > >>>> as
> >> > > >>>> >> well.
> >> > > >>>> >> >> The reason for the awkwardness is that we simply don't
> want
> >> > most
> >> > > >>>> of the
> >> > > >>>> >> >> people currently setting
> >> unclean.leader.election.enable=false
> >> > to
> >> > > >>>> get the
> >> > > >>>> >> >> "never" behavior. We have to bite that bullet. Better to
> be
> >> > > clear
> >> > > >>>> and
> >> > > >>>> >> >> explicit than hide it.
> >> > > >>>> >> >>
> >> > > >>>> >> >> Another thing that's a bit awkward is having two
> different
> >> > ways
> >> > > to
> >> > > >>>> do
> >> > > >>>> >> >> unclean leader election specified in the KIP. You
> descirbe
> >> two
> >> > > >>>> methods:
> >> > > >>>> >> the
> >> > > >>>> >> >> simple "choose the last leader" method, and the "unclean
> >> > > recovery
> >> > > >>>> >> manager"
> >> > > >>>> >> >> method. I understand why you did it this way -- "choose
> the
> >> > last
> >> > > >>>> >> leader" is
> >> > > >>>> >> >> simple, and will help us deliver an implementation
> quickly,
> >> > > while
> >> > > >>>> the
> >> > > >>>> >> URM
> >> > > >>>> >> >> is preferable in the long term. My suggestion here is to
> >> > > separate
> >> > > >>>> the
> >> > > >>>> >> >> decision of HOW to do unclean leader election from the
> >> > decision
> >> > > of
> >> > > >>>> WHEN
> >> > > >>>> >> to
> >> > > >>>> >> >> do it.
> >> > > >>>> >> >>
> >> > > >>>> >> >> So in other words, have "unclean.leader.election.enable"
> >> > specify
> >> > > >>>> when we
> >> > > >>>> >> >> do unclean leader election, and have a new configuration
> >> like
> >> > > >>>> >> >> "unclean.recovery.manager.enable" to determine if we use
> the
> >> > > URM.
> >> > > >>>> >> >> Presumably the URM will take some time to get fully
> stable,
> >> so
> >> > > >>>> this can
> >> > > >>>> >> >> default to false for a while, and we can flip the
> default to
> >> > > true
> >> > > >>>> when
> >> > > >>>> >> we
> >> > > >>>> >> >> feel ready.
> >> > > >>>> >> >>
> >> > > >>>> >> >> The URM is somewhat under-described here. I think we
> need a
> >> > few
> >> > > >>>> >> >> configurations here for it. For example, we need a
> >> > > configuration to
> >> > > >>>> >> specify
> >> > > >>>> >> >> how long it should wait for a broker to respond to its
> RPCs
> >> > > before
> >> > > >>>> >> moving
> >> > > >>>> >> >> on. We also need to understand how the URM interacts with
> >> > > >>>> >> >> unclean.leader.election.enable=always. I assume that with
> >> > > "always"
> >> > > >>>> we
> >> > > >>>> >> will
> >> > > >>>> >> >> just unconditionally use the URM rather than choosing
> >> > randomly.
> >> > > >>>> But this
> >> > > >>>> >> >> should be spelled out in the KIP.
> >> > > >>>> >> >>
> >> > > >>>> >> >> >
> >> > > >>>> >> >> > DescribeTopicRequest
> >> > > >>>> >> >> >
> >> > > >>>> >> >> >    1.
> >> > > >>>> >> >> >    Yes, the plan is to replace the MetadataRequest with
> >> the
> >> > > >>>> >> >> >    DescribeTopicRequest for the admin clients. Will
> check
> >> > the
> >> > > >>>> details.
> >> > > >>>> >> >>
> >> > > >>>> >> >> Sounds good. But as I said, you need to specify how
> >> > AdminClient
> >> > > >>>> >> interacts
> >> > > >>>> >> >> with the new request. This will involve adding some
> fields
> >> to
> >> > > >>>> >> >> TopicDescription.java. And you need to specify the
> changes
> >> to
> >> > > the
> >> > > >>>> >> >> kafka-topics.sh command line tool. Otherwise we cannot
> use
> >> the
> >> > > >>>> tool to
> >> > > >>>> >> see
> >> > > >>>> >> >> the new information.
> >> > > >>>> >> >>
> >> > > >>>> >> >> The new requests, DescribeTopicRequest and
> >> > > >>>> GetReplicaLogInfoRequest,
> >> > > >>>> >> need
> >> > > >>>> >> >> to have limits placed on them so that their size can't be
> >> > > >>>> infinite. We
> >> > > >>>> >> >> don't want to propagate the current problems of
> >> > MetadataRequest,
> >> > > >>>> where
> >> > > >>>> >> >> clients can request massive responses that can mess up
> the
> >> JVM
> >> > > when
> >> > > >>>> >> handled.
> >> > > >>>> >> >>
> >> > > >>>> >> >> Adding limits is simple for GetReplicaLogInfoRequest --
> we
> >> can
> >> > > >>>> just say
> >> > > >>>> >> >> that only 2000 partitions at a time can be requested. For
> >> > > >>>> >> >> DescribeTopicRequest we can probably just limit to 20
> topics
> >> > or
> >> > > >>>> >> something
> >> > > >>>> >> >> like that, to avoid the complexity of doing pagination in
> >> this
> >> > > KIP.
> >> > > >>>> >> >>
> >> > > >>>> >> >> >    2.
> >> > > >>>> >> >> >    I can let the broker load the ELR info so that they
> can
> >> > > serve
> >> > > >>>> the
> >> > > >>>> >> >> >    DescribeTopicRequest as well.
> >> > > >>>> >> >>
> >> > > >>>> >> >> Yes, it's fine to add to MetadataCache. In fact, you'll
> be
> >> > > loading
> >> > > >>>> it
> >> > > >>>> >> >> anyway once it's added to PartitionImage.
> >> > > >>>> >> >>
> >> > > >>>> >> >> >    3.
> >> > > >>>> >> >> >    Yeah, it does not make sense to have the topic id if
> >> > > >>>> >> >> >    DescribeTopicRequest is only used by the admin
> client.
> >> > > >>>> >> >>
> >> > > >>>> >> >> OK. That makes things simpler. We can always create a new
> >> API
> >> > > later
> >> > > >>>> >> >> (hopefully not in this KIP!) to query by topic ID.
> >> > > >>>> >> >>
> >> > > >>>> >> >> >
> >> > > >>>> >> >> >
> >> > > >>>> >> >> > Metrics
> >> > > >>>> >> >> >
> >> > > >>>> >> >> > As for overall cluster health metrics, I think
> >> under-min-ISR
> >> > > is
> >> > > >>>> still
> >> > > >>>> >> a
> >> > > >>>> >> >> > useful one. ELR is more like a safety belt. When the
> ELR
> >> is
> >> > > >>>> used, the
> >> > > >>>> >> >> > cluster availability has already been impacted.
> >> > > >>>> >> >> >
> >> > > >>>> >> >> > Maybe we can have a metric to count the partitions that
> >> > > sum(ISR,
> >> > > >>>> ELR)
> >> > > >>>> >> <
> >> > > >>>> >> >> min
> >> > > >>>> >> >> > ISR. What do you think?
> >> > > >>>> >> >>
> >> > > >>>> >> >> How about:
> >> > > >>>> >> >>
> >> > > >>>> >> >> A.  a metric for the totoal number of under-min-isr
> >> > partitions?
> >> > > We
> >> > > >>>> don't
> >> > > >>>> >> >> have that in Apache Kafka at the moment.
> >> > > >>>> >> >>
> >> > > >>>> >> >> B. a metric for the number of unclean leader elections we
> >> did
> >> > > (for
> >> > > >>>> >> >> simplicity, it can reset to 0 on controller restart: we
> >> expect
> >> > > >>>> people to
> >> > > >>>> >> >> monitor the change over time anyway)
> >> > > >>>> >> >>
> >> > > >>>> >> >> best,
> >> > > >>>> >> >> Colin
> >> > > >>>> >> >>
> >> > > >>>> >> >>
> >> > > >>>> >> >> >
> >> > > >>>> >> >> > Yeah, for the ongoing unclean recoveries, the
> controller
> >> can
> >> > > >>>> keep an
> >> > > >>>> >> >> > accurate count through failover because partition
> >> > registration
> >> > > >>>> can
> >> > > >>>> >> >> indicate
> >> > > >>>> >> >> > whether a recovery is needed. However, for the happened
> >> > ones,
> >> > > >>>> unless
> >> > > >>>> >> we
> >> > > >>>> >> >> > want to persist the number somewhere, we can only
> figure
> >> it
> >> > > out
> >> > > >>>> from
> >> > > >>>> >> the
> >> > > >>>> >> >> > log.
> >> > > >>>> >> >> >
> >> > > >>>> >> >> > On Tue, Sep 12, 2023 at 3:16 PM Colin McCabe <
> >> > > cmccabe@apache.org
> >> > > >>>> >
> >> > > >>>> >> wrote:
> >> > > >>>> >> >> >
> >> > > >>>> >> >> >> Also, we should have metrics that show what is going
> on
> >> > with
> >> > > >>>> regard
> >> > > >>>> >> to
> >> > > >>>> >> >> the
> >> > > >>>> >> >> >> eligible replica set. I'm not sure exactly what to
> >> suggest,
> >> > > but
> >> > > >>>> >> >> something
> >> > > >>>> >> >> >> that could identify when things are going wrong in the
> >> > > clsuter.
> >> > > >>>> >> >> >>
> >> > > >>>> >> >> >> For example, maybe a metric for partitions containing
> >> > > replicas
> >> > > >>>> that
> >> > > >>>> >> are
> >> > > >>>> >> >> >> ineligible to be leader? That would show a spike when
> a
> >> > > broker
> >> > > >>>> had an
> >> > > >>>> >> >> >> unclean restart.
> >> > > >>>> >> >> >>
> >> > > >>>> >> >> >> Ideally, we'd also have a metric that indicates when
> an
> >> > > unclear
> >> > > >>>> >> leader
> >> > > >>>> >> >> >> election or a recovery happened. It's a bit tricky
> >> because
> >> > > the
> >> > > >>>> simple
> >> > > >>>> >> >> >> thing, of tracking it per controller, may be a bit
> >> > confusing
> >> > > >>>> during
> >> > > >>>> >> >> >> failovers.
> >> > > >>>> >> >> >>
> >> > > >>>> >> >> >> best,
> >> > > >>>> >> >> >> Colin
> >> > > >>>> >> >> >>
> >> > > >>>> >> >> >>
> >> > > >>>> >> >> >> On Tue, Sep 12, 2023, at 14:25, Colin McCabe wrote:
> >> > > >>>> >> >> >> > Hi Calvin,
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> > Thanks for the KIP. I think this is a great
> >> improvement.
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> >> Additional High Watermark advance requirement
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> > Typo: change "advance" to "advancement"
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> >> A bit recap of some key concepts.
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> > Typo: change "bit" to "quick"
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> >> Ack=1/all produce request. It defines when the
> Kafka
> >> > > server
> >> > > >>>> should
> >> > > >>>> >> >> >> respond to the produce request
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> > I think this section would be clearer if we talked
> >> about
> >> > > the
> >> > > >>>> new
> >> > > >>>> >> high
> >> > > >>>> >> >> >> > watermark advancement requirement first, and THEN
> >> talked
> >> > > >>>> about its
> >> > > >>>> >> >> >> > impact on acks=0, acks=1, and     acks=all.
> acks=all
> >> is
> >> > of
> >> > > >>>> course
> >> > > >>>> >> the
> >> > > >>>> >> >> >> > main case we care about here, so it would be good to
> >> lead
> >> > > with
> >> > > >>>> >> that,
> >> > > >>>> >> >> >> > rather than delving into the technicalities of
> acks=0/1
> >> > > first.
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> >> Unclean recovery
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> > So, here you are introducing a new configuration,
> >> > > >>>> >> >> >> > unclean.recovery.strategy. The difficult thing here
> is
> >> > that
> >> > > >>>> there
> >> > > >>>> >> is a
> >> > > >>>> >> >> >> > lot of overlap with unclean.leader.election.enable.
> So
> >> we
> >> > > >>>> have 3
> >> > > >>>> >> >> >> > different settings for unclean.recovery.strategy,
> plus
> >> 2
> >> > > >>>> different
> >> > > >>>> >> >> >> > settings for unclean.leader.election.enable, giving
> a
> >> > cross
> >> > > >>>> >> product of
> >> > > >>>> >> >> >> > 6 different options. The following "unclean recovery
> >> > > manager"
> >> > > >>>> >> section
> >> > > >>>> >> >> >> > only applies to one fo those 6 different
> possibilities
> >> (I
> >> > > >>>> think?)
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> > I simply don't think we need so many different
> election
> >> > > types.
> >> > > >>>> >> Really
> >> > > >>>> >> >> >> > the use-cases we need are people who want NO unclean
> >> > > >>>> elections,
> >> > > >>>> >> people
> >> > > >>>> >> >> >> > who want "the reasonable thing" and people who want
> >> > > >>>> avaialbility at
> >> > > >>>> >> >> all
> >> > > >>>> >> >> >> > costs.
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> > Overall, I feel like the first half of the KIP is
> about
> >> > the
> >> > > >>>> ELR,
> >> > > >>>> >> and
> >> > > >>>> >> >> >> > the second half is about reworking unclean leader
> >> > > election. It
> >> > > >>>> >> might
> >> > > >>>> >> >> be
> >> > > >>>> >> >> >> > better to move that second half to a separate KIP so
> >> that
> >> > > we
> >> > > >>>> can
> >> > > >>>> >> >> figure
> >> > > >>>> >> >> >> > it out fully. It should be fine to punt this until
> >> later
> >> > > and
> >> > > >>>> just
> >> > > >>>> >> have
> >> > > >>>> >> >> >> > the current behavior on empty ELR be waiting for the
> >> last
> >> > > >>>> known
> >> > > >>>> >> leader
> >> > > >>>> >> >> >> > to return. After all, that's what we do today.
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> >> DescribeTopicRequest
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> > Is the intention for AdminClient to use this RPC for
> >> > > >>>> >> >> >> > Admin#describeTopics ? If so, we need to describe
> all
> >> of
> >> > > the
> >> > > >>>> >> changes
> >> > > >>>> >> >> to
> >> > > >>>> >> >> >> > the admin client API, as well as changes to
> >> command-line
> >> > > >>>> tools like
> >> > > >>>> >> >> >> > kafka-topics.sh (if there are any). For example, you
> >> will
> >> > > >>>> probably
> >> > > >>>> >> >> need
> >> > > >>>> >> >> >> > changes to TopicDescription.java. You will also
> need to
> >> > > >>>> provide
> >> > > >>>> >> all of
> >> > > >>>> >> >> >> > the things that admin client needs -- for example,
> >> > > >>>> >> >> >> > TopicAuthorizedOperations.
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> > I also don't think the controller should serve this
> >> > > request.
> >> > > >>>> We
> >> > > >>>> >> want
> >> > > >>>> >> >> to
> >> > > >>>> >> >> >> > minimize load on the controller. Just like with the
> >> other
> >> > > >>>> metadata
> >> > > >>>> >> >> >> > requests like MetadataRequest, this should be
> served by
> >> > > >>>> brokers.
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> > It's a bit confusing why both topic ID and topic
> name
> >> are
> >> > > >>>> provided
> >> > > >>>> >> to
> >> > > >>>> >> >> >> > this API. Is the intention that callers should set
> one
> >> > but
> >> > > >>>> not the
> >> > > >>>> >> >> >> > other? Or both? This needs to be clarified. Also,
> if we
> >> > do
> >> > > >>>> want to
> >> > > >>>> >> >> >> > support lookups by UUID, that is another thing that
> >> needs
> >> > > to
> >> > > >>>> be
> >> > > >>>> >> added
> >> > > >>>> >> >> >> > to adminclient.
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> > In general, I feel like this should also probably be
> >> its
> >> > > own
> >> > > >>>> KIP
> >> > > >>>> >> since
> >> > > >>>> >> >> >> > it's fairly complex
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> > best,
> >> > > >>>> >> >> >> > Colin
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> >
> >> > > >>>> >> >> >> > On Thu, Aug 10, 2023, at 15:46, Calvin Liu 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
> >> > > >>>> >> >> >>
> >> > > >>>> >> >>
> >> > > >>>> >>
> >> > > >>>>
> >> > > >>>
> >> > >
> >> >
> >>
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Colin McCabe <cm...@apache.org>.
On Tue, Oct 3, 2023, at 10:49, Jun Rao wrote:
> Hi, Calvin,
>
> Thanks for the update KIP. A few more comments.
>
> 41. Why would a user choose the option to select a random replica as the
> leader instead of using unclean.recovery.strateg=Aggressive? It seems that
> the latter is strictly better? If that's not the case, could we fold this
> option under unclean.recovery.strategy instead of introducing a separate
> config?

Hi Jun,

I thought the flow of control was:

If there is no leader for the partition {
  If (there are unfenced ELR members) {
    choose_an_unfenced_ELR_member
  } else if (there are fenced ELR members AND strategy=Aggressive) {
    do_unclean_recovery
  } else if (there are no ELR members AND strategy != None) {
    do_unclean_recovery
  } else {
    do nothing about the missing leader
  }
}

do_unclean_recovery() {
   if (unclean.recovery.manager.enabled) {
    use UncleanRecoveryManager
  } else {
    choose the last known leader if that is available, or a random leader if not)
  }
}

However, I think this could be clarified, especially the behavior when
unclean.recovery.manager.enabled=false. Inuitively the goal for 
unclean.recovery.manager.enabled=false is to be "the same as now, mostly"
but it's very underspecified in the KIP, I agree.

>
> 50. ElectLeadersRequest: "If more than 20 topics are included, only the
> first 20 will be served. Others will be returned with DesiredLeaders." Hmm,
> not sure that I understand this. ElectLeadersResponse doesn't have a
> DesiredLeaders field.
>
> 51. GetReplicaLogInfo: "If more than 2000 partitions are included, only the
> first 2000 will be served" Do we return an error for the remaining
> partitions? Actually, should we include an errorCode field at the partition
> level in GetReplicaLogInfoResponse to cover non-existing partitions and no
> authorization, etc?
>
> 52. The entry should matches => The entry should match
>
> 53. ElectLeadersRequest.DesiredLeaders: Should it be nullable since a user
> may not specify DesiredLeaders?
>
> 54. Downgrade: Is that indeed possible? I thought earlier you said that
> once the new version of the records are in the metadata log, one can't
> downgrade since the old broker doesn't know how to parse the new version of
> the metadata records?
>

MetadataVersion downgrade is currently broken but we have fixing it on our plate for Kafka 3.7.

The way downgrade works is that "new features" are dropped, leaving only the old ones.

> 55. CleanShutdownFile: Should we add a version field for future extension?
>
> 56. Config changes are public facing. Could we have a separate section to
> document all the config changes?

+1. A separate section for this would be good.

best,
Colin

>
> Thanks,
>
> Jun
>
> On Mon, Sep 25, 2023 at 4:29 PM Calvin Liu <ca...@confluent.io.invalid>
> wrote:
>
>> Hi Jun
>> Thanks for the comments.
>>
>> 40. If we change to None, it is not guaranteed for no data loss. For users
>> who are not able to validate the data with external resources, manual
>> intervention does not give a better result but a loss of availability. So
>> practically speaking, the Balance mode would be a better default value.
>>
>> 41. No, it represents how we want to do the unclean leader election. If it
>> is false, the unclean leader election will be the old random way.
>> Otherwise, the unclean recovery will be used.
>>
>> 42. Good catch. Updated.
>>
>> 43. Only the first 20 topics will be served. Others will be returned with
>> InvalidRequestError
>>
>> 44. The order matters. The desired leader entries match with the topic
>> partition list by the index.
>>
>> 45. Thanks! Updated.
>>
>> 46. Good advice! Updated.
>>
>> 47.1, updated the comment. Basically it will elect the replica in the
>> desiredLeader field to be the leader
>>
>> 47.2 We can let the admin client do the conversion. Using the desiredLeader
>> field in the json format seems easier for users.
>>
>> 48. Once the MV version is downgraded, all the ELR related fields will be
>> removed on the next partition change. The controller will also ignore the
>> ELR fields. Updated the KIP.
>>
>> 49. Yes, it would be deprecated/removed.
>>
>>
>> On Mon, Sep 25, 2023 at 3:49 PM Jun Rao <ju...@confluent.io.invalid> wrote:
>>
>> > Hi, Calvin,
>> >
>> > Thanks for the updated KIP. Made another pass. A few more comments below.
>> >
>> > 40. unclean.leader.election.enable.false ->
>> > unclean.recovery.strategy.Balanced: The Balanced mode could still lead to
>> > data loss. So, I am wondering if unclean.leader.election.enable.false
>> > should map to None?
>> >
>> > 41. unclean.recovery.manager.enabled: I am not sure why we introduce this
>> > additional config. Is it the same as unclean.recovery.strategy=None?
>> >
>> > 42. DescribeTopicResponse.TopicAuthorizedOperations: Should this be at
>> the
>> > topic level?
>> >
>> > 43. "Limit: 20 topics max per request": Could we describe what happens if
>> > the request includes more than 20 topics?
>> >
>> > 44. ElectLeadersRequest.DesiredLeaders: Could we describe whether the
>> > ordering matters?
>> >
>> > 45. GetReplicaLogInfo.TopicPartitions: "about": "The topic partitions to
>> > elect leaders.": The description in "about" is incorrect.
>> >
>> > 46. GetReplicaLogInfoResponse: Should we nest partitions under topicId to
>> > be consistent with other types of responses?
>> >
>> > 47. kafka-leader-election.sh:
>> > 47.1 Could we explain DESIGNATION?
>> > 47.2 desiredLeader: Should it be a list to match the field in
>> > ElectLeadersRequest?
>> >
>> > 48. We could add a section on downgrade?
>> >
>> > 49. LastKnownLeader: This seems only needed in the first phase of
>> > delivering ELR. Will it be removed when the complete KIP is delivered?
>> >
>> > Thanks,
>> >
>> > Jun
>> >
>> > On Tue, Sep 19, 2023 at 1:30 PM Colin McCabe <cm...@apache.org> wrote:
>> >
>> > > Hi Calvin,
>> > >
>> > > Thanks for the explanations. I like the idea of using none, balanced,
>> > > aggressive. We also had an offline discussion about why it is good to
>> > use a
>> > > new config key (basically, so that we can deprecate the old one which
>> had
>> > > only false/true values in 4.0) With these changes, I am +1.
>> > >
>> > > best,
>> > > Colin
>> > >
>> > > On Mon, Sep 18, 2023, at 15:54, Calvin Liu wrote:
>> > > > Hi Colin,
>> > > > Also, can we deprecate unclean.leader.election.enable in 4.0? Before
>> > > that,
>> > > > we can have both the config unclean.recovery.strategy and
>> > > > unclean.leader.election.enable
>> > > > and using the unclean.recovery.Enabled to determine which config to
>> use
>> > > > during the unclean leader election.
>> > > >
>> > > > On Mon, Sep 18, 2023 at 3:51 PM Calvin Liu <ca...@confluent.io>
>> wrote:
>> > > >
>> > > >> Hi Colin,
>> > > >> For the unclean.recovery.strategy config name, how about we use the
>> > > >> following
>> > > >> None. It basically means no unclean recovery will be performed.
>> > > >> Aggressive. It means availability goes first. Whenever the partition
>> > > can't
>> > > >> elect a durable replica, the controller will try the unclean
>> recovery.
>> > > >> Balanced. It is the balance point of the availability
>> > first(Aggressive)
>> > > >> and least availability(None). The controller performs unclean
>> recovery
>> > > when
>> > > >> both ISR and ELR are empty.
>> > > >>
>> > > >>
>> > > >> On Fri, Sep 15, 2023 at 11:42 AM Calvin Liu <ca...@confluent.io>
>> > wrote:
>> > > >>
>> > > >>> Hi Colin,
>> > > >>>
>> > > >>> > So, the proposal is that if someone sets
>> > > "unclean.leader.election.enable
>> > > >>> = true"...
>> > > >>>
>> > > >>>
>> > > >>> The idea is to use one of the unclean.leader.election.enable and
>> > > >>> unclean.recovery.strategy based on the unclean.recovery.Enabled. A
>> > > possible
>> > > >>> version can be
>> > > >>>
>> > > >>> If unclean.recovery.Enabled:
>> > > >>>
>> > > >>> {
>> > > >>>
>> > > >>> Check unclean.recovery.strategy. If set, use it. Otherwise, check
>> > > >>> unclean.leader.election.enable and translate it to
>> > > >>> unclean.recovery.strategy.
>> > > >>>
>> > > >>> } else {
>> > > >>>
>> > > >>> Use unclean.leader.election.enable
>> > > >>>
>> > > >>> }
>> > > >>>
>> > > >>>
>> > > >>> —--------
>> > > >>>
>> > > >>> >The configuration key should be
>> "unclean.recovery.manager.enabled",
>> > > >>> right?
>> > > >>>
>> > > >>>
>> > > >>> I think we have two ways of choosing a leader uncleanly, unclean
>> > leader
>> > > >>> election and unclean recovery(log inspection) and we try to switch
>> > > between
>> > > >>> them.
>> > > >>>
>> > > >>> Do you mean we want to develop two ways of performing the unclean
>> > > >>> recovery and one of them is using “unclean recovery manager”? I
>> guess
>> > > we
>> > > >>> haven’t discussed the second way.
>> > > >>>
>> > > >>>
>> > > >>> —-------
>> > > >>>
>> > > >>> >How do these 4 levels of overrides interact with your new
>> > > >>> configurations?
>> > > >>>
>> > > >>>
>> > > >>> I do notice in the Kraft controller code, the method to check
>> whether
>> > > >>> perform unclean leader election is hard coded to false since
>> > > >>> 2021(uncleanLeaderElectionEnabledForTopic). Isn’t it a good chance
>> to
>> > > >>> completely deprecate the unclean.leader.election.enable? We don’t
>> > even
>> > > have
>> > > >>> to worry about the config conversion.
>> > > >>>
>> > > >>> On the other hand, whatever the override is, as long as the
>> > controller
>> > > >>> can have the final effective unclean.leader.election.enable, the
>> > topic
>> > > >>> level config unclean.recovery.strategy, the cluster level config
>> > > >>> unclean.recovery.Enabled, the controller can calculate the correct
>> > > methods
>> > > >>> to use right?
>> > > >>>
>> > > >>>
>> > > >>> On Fri, Sep 15, 2023 at 10:02 AM Colin McCabe <cm...@apache.org>
>> > > wrote:
>> > > >>>
>> > > >>>> On Thu, Sep 14, 2023, at 22:23, Calvin Liu wrote:
>> > > >>>> > Hi Colin
>> > > >>>> > 1. I think using the new config name is more clear.
>> > > >>>> >        a. The unclean leader election is actually removed if
>> > unclean
>> > > >>>> > recovery is in use.
>> > > >>>> >        b. Using multiple values in
>> unclean.leader.election.enable
>> > is
>> > > >>>> > confusing and it will be more confusing after people forget
>> about
>> > > this
>> > > >>>> > discussion.
>> > > >>>>
>> > > >>>> Hi Calvin,
>> > > >>>>
>> > > >>>> So, the proposal is that if someone sets
>> > > "unclean.leader.election.enable
>> > > >>>> = true" but then sets one of your new configurations, the value of
>> > > >>>> unclean.leader.election.enable is ignored? That seems less clear
>> to
>> > > me, not
>> > > >>>> more. Just in general, having multiple configuration keys to
>> control
>> > > the
>> > > >>>> same thing confuses users. Basically, they are sitting at a giant
>> > > control
>> > > >>>> panel, and some of the levers do nothing.
>> > > >>>>
>> > > >>>> > 2. Sorry I forgot to mention in the response that I did add the
>> > > >>>> > unclean.recovery.Enabled flag.
>> > > >>>>
>> > > >>>> The configuration key should be
>> "unclean.recovery.manager.enabled",
>> > > >>>> right? Becuase we can do "unclean recovery" without the manager.
>> > > Disabling
>> > > >>>> the manager just means we use a different mechanism for recovery.
>> > > >>>>
>> > > >>>> >        c. Maybe I underestimated the challenge of replacing the
>> > > >>>> config. Any
>> > > >>>> > implementation problems ahead?
>> > > >>>>
>> > > >>>> There are four levels of overrides for
>> > unclean.leader.election.enable.
>> > > >>>>
>> > > >>>> 1. static configuration for node.
>> > > >>>>     This goes in the configuration file, typically named
>> > > >>>> server.properties
>> > > >>>>
>> > > >>>> 2. dynamic configuration for node default
>> > > >>>>   ConfigResource(type=BROKER, name="")
>> > > >>>>
>> > > >>>> 3. dynamic configuration for node
>> > > >>>>   ConfigResource(type=BROKER, name=<controller id>)
>> > > >>>>
>> > > >>>> 4. dynamic configuration for topic
>> > > >>>>   ConfigResource(type=TOPIC, name=<topic-name>)
>> > > >>>>
>> > > >>>> How do these 4 levels of overrides interact with your new
>> > > >>>> configurations? If the new configurations dominate over the old
>> > ones,
>> > > it
>> > > >>>> seems like this will get a lot more confusing to implement (and
>> also
>> > > to
>> > > >>>> use.)
>> > > >>>>
>> > > >>>> Again, I'd recommend just adding some new values to
>> > > >>>> unclean.leader.election.enable. It's simple and will prevent user
>> > > confusion
>> > > >>>> (as well as developer confusion.)
>> > > >>>>
>> > > >>>> best,
>> > > >>>> Colin
>> > > >>>>
>> > > >>>>
>> > > >>>> > 3. About the admin client, I mentioned 3 changes in the client.
>> > > >>>> Anything
>> > > >>>> > else I missed in the KIP?
>> > > >>>> >       a. The client will switch to using the new RPC instead of
>> > > >>>> > MetadataRequest for the topics.
>> > > >>>> >       b. The TopicPartitionInfo used in TopicDescription needs
>> to
>> > > add
>> > > >>>> new
>> > > >>>> > fields related to the ELR.
>> > > >>>> >       c. The outputs will add the ELR related fields.
>> > > >>>> >
>> > > >>>> > On Thu, Sep 14, 2023 at 9:19 PM Colin McCabe <
>> cmccabe@apache.org>
>> > > >>>> wrote:
>> > > >>>> >
>> > > >>>> >> Hi Calvin,
>> > > >>>> >>
>> > > >>>> >> Thanks for the changes.
>> > > >>>> >>
>> > > >>>> >> 1. Earlier I commented that creating
>> "unclean.recovery.strategy "
>> > > is
>> > > >>>> not
>> > > >>>> >> necessary, and we can just reuse the existing
>> > > >>>> >> "unclean.leader.election.enable" configuration key. Let's
>> discuss
>> > > >>>> that.
>> > > >>>> >>
>> > > >>>> >> 2.I also don't understand why you didn't add a configuration to
>> > > >>>> enable or
>> > > >>>> >> disable the Unclean Recovery Manager. This seems like a very
>> > simple
>> > > >>>> way to
>> > > >>>> >> handle the staging issue which we discussed. The URM can just
>> be
>> > > >>>> turned off
>> > > >>>> >> until it is production ready. Let's discuss this.
>> > > >>>> >>
>> > > >>>> >> 3. You still need to describe the changes to AdminClient that
>> are
>> > > >>>> needed
>> > > >>>> >> to use DescribeTopicRequest.
>> > > >>>> >>
>> > > >>>> >> Keep at it. It's looking better. :)
>> > > >>>> >>
>> > > >>>> >> best,
>> > > >>>> >> Colin
>> > > >>>> >>
>> > > >>>> >>
>> > > >>>> >> On Thu, Sep 14, 2023, at 11:03, Calvin Liu wrote:
>> > > >>>> >> > Hi Colin
>> > > >>>> >> > Thanks for the comments!
>> > > >>>> >> >
>> > > >>>> >> > I did the following changes
>> > > >>>> >> >
>> > > >>>> >> >    1.
>> > > >>>> >> >
>> > > >>>> >> >    Simplified the API spec section to only include the diff.
>> > > >>>> >> >    2.
>> > > >>>> >> >
>> > > >>>> >> >    Reordered the HWM requirement section.
>> > > >>>> >> >    3.
>> > > >>>> >> >
>> > > >>>> >> >    Removed the URM implementation details to keep the
>> necessary
>> > > >>>> >> >    characteristics to perform the unclean recovery.
>> > > >>>> >> >    1.
>> > > >>>> >> >
>> > > >>>> >> >       When to perform the unclean recovery
>> > > >>>> >> >       2.
>> > > >>>> >> >
>> > > >>>> >> >       Under different config, how the unclean recovery finds
>> > the
>> > > >>>> leader.
>> > > >>>> >> >       3.
>> > > >>>> >> >
>> > > >>>> >> >       How the config unclean.leader.election.enable and
>> > > >>>> >> >       unclean.recovery.strategy are converted when users
>> > > >>>> enable/disable
>> > > >>>> >> the
>> > > >>>> >> >       unclean recovery.
>> > > >>>> >> >       4.
>> > > >>>> >> >
>> > > >>>> >> >    More details about how we change admin client.
>> > > >>>> >> >    5.
>> > > >>>> >> >
>> > > >>>> >> >    API limits on the GetReplicaLogInfoRequest and
>> > > >>>> DescribeTopicRequest.
>> > > >>>> >> >    6.
>> > > >>>> >> >
>> > > >>>> >> >    Two metrics added
>> > > >>>> >> >    1.
>> > > >>>> >> >
>> > > >>>> >> >       Kafka.controller.global_under_min_isr_partition_count
>> > > >>>> >> >       2.
>> > > >>>> >> >
>> > > >>>> >> >       kafka.controller.unclean_recovery_finished_count
>> > > >>>> >> >
>> > > >>>> >> >
>> > > >>>> >> > On Wed, Sep 13, 2023 at 10:46 AM Colin McCabe <
>> > > cmccabe@apache.org>
>> > > >>>> >> wrote:
>> > > >>>> >> >
>> > > >>>> >> >> On Tue, Sep 12, 2023, at 17:21, Calvin Liu wrote:
>> > > >>>> >> >> > Hi Colin
>> > > >>>> >> >> > Thanks for the comments!
>> > > >>>> >> >> >
>> > > >>>> >> >>
>> > > >>>> >> >> Hi Calvin,
>> > > >>>> >> >>
>> > > >>>> >> >> Thanks again for the KIP.
>> > > >>>> >> >>
>> > > >>>> >> >> One meta-comment: it's usually better to just do a diff on a
>> > > >>>> message
>> > > >>>> >> spec
>> > > >>>> >> >> file or java file if you're including changes to it in the
>> > KIP.
>> > > >>>> This is
>> > > >>>> >> >> easier to read than looking for "new fields begin" etc. in
>> the
>> > > >>>> text, and
>> > > >>>> >> >> gracefully handles the case where existing fields were
>> > changed.
>> > > >>>> >> >>
>> > > >>>> >> >> > Rewrite the Additional High Watermark advancement
>> > requirement
>> > > >>>> >> >> > There was feedback on this section that some readers may
>> not
>> > > be
>> > > >>>> >> familiar
>> > > >>>> >> >> > with HWM and Ack=0,1,all requests. This can help them
>> > > understand
>> > > >>>> the
>> > > >>>> >> >> > proposal. I will rewrite this part for more readability.
>> > > >>>> >> >> >
>> > > >>>> >> >>
>> > > >>>> >> >> To be clear, I wasn't suggesting dropping either section. I
>> > > agree
>> > > >>>> that
>> > > >>>> >> >> they add useful background. I was just suggesting that we
>> > should
>> > > >>>> discuss
>> > > >>>> >> >> the "acks" setting AFTER discussing the new high watermark
>> > > >>>> advancement
>> > > >>>> >> >> conditions. We also should discuss acks=0. While it isn't
>> > > >>>> conceptually
>> > > >>>> >> much
>> > > >>>> >> >> different than acks=1 here, its omission from this section
>> is
>> > > >>>> confusing.
>> > > >>>> >> >>
>> > > >>>> >> >> > Unclean recovery
>> > > >>>> >> >> >
>> > > >>>> >> >> > The plan is to replace the unclean.leader.election.enable
>> > with
>> > > >>>> >> >> > unclean.recovery.strategy. If the Unclean Recovery is
>> > enabled
>> > > >>>> then it
>> > > >>>> >> >> deals
>> > > >>>> >> >> > with the three options in the unclean.recovery.strategy.
>> > > >>>> >> >> >
>> > > >>>> >> >> >
>> > > >>>> >> >> > Let’s refine the Unclean Recovery. We have already taken a
>> > > lot of
>> > > >>>> >> >> > suggestions and I hope to enhance the durability of Kafka
>> to
>> > > the
>> > > >>>> next
>> > > >>>> >> >> level
>> > > >>>> >> >> > with this KIP.
>> > > >>>> >> >>
>> > > >>>> >> >> I am OK with doing the unclean leader recovery improvements
>> in
>> > > >>>> this KIP.
>> > > >>>> >> >> However, I think we need to really work on the configuration
>> > > >>>> settings.
>> > > >>>> >> >>
>> > > >>>> >> >> Configuration overrides are often quite messy. For example,
>> > the
>> > > >>>> cases
>> > > >>>> >> >> where we have log.roll.hours and log.roll.segment.ms, the
>> > user
>> > > >>>> has to
>> > > >>>> >> >> remember which one takes precedence, and it is not obvious.
>> > So,
>> > > >>>> rather
>> > > >>>> >> than
>> > > >>>> >> >> creating a new configuration, why not add additional values
>> to
>> > > >>>> >> >> "unclean.leader.election.enable"? I think this will be
>> simpler
>> > > for
>> > > >>>> >> people
>> > > >>>> >> >> to understand, and simpler in the code as well.
>> > > >>>> >> >>
>> > > >>>> >> >> What if we continued to use "unclean.leader.election.enable"
>> > but
>> > > >>>> >> extended
>> > > >>>> >> >> it so that it took a string? Then the string could have
>> these
>> > > >>>> values:
>> > > >>>> >> >>
>> > > >>>> >> >> never
>> > > >>>> >> >>     never automatically do an unclean leader election under
>> > any
>> > > >>>> >> conditions
>> > > >>>> >> >>
>> > > >>>> >> >> false / default
>> > > >>>> >> >>     only do an unclean leader election if there may be
>> > possible
>> > > >>>> data
>> > > >>>> >> loss
>> > > >>>> >> >>
>> > > >>>> >> >> true / always
>> > > >>>> >> >>     always do an unclean leader election if we can't
>> > immediately
>> > > >>>> elect a
>> > > >>>> >> >> leader
>> > > >>>> >> >>
>> > > >>>> >> >> It's a bit awkward that false maps to default rather than to
>> > > >>>> never. But
>> > > >>>> >> >> this awkwardness exists if we use two different
>> configuration
>> > > keys
>> > > >>>> as
>> > > >>>> >> well.
>> > > >>>> >> >> The reason for the awkwardness is that we simply don't want
>> > most
>> > > >>>> of the
>> > > >>>> >> >> people currently setting
>> unclean.leader.election.enable=false
>> > to
>> > > >>>> get the
>> > > >>>> >> >> "never" behavior. We have to bite that bullet. Better to be
>> > > clear
>> > > >>>> and
>> > > >>>> >> >> explicit than hide it.
>> > > >>>> >> >>
>> > > >>>> >> >> Another thing that's a bit awkward is having two different
>> > ways
>> > > to
>> > > >>>> do
>> > > >>>> >> >> unclean leader election specified in the KIP. You descirbe
>> two
>> > > >>>> methods:
>> > > >>>> >> the
>> > > >>>> >> >> simple "choose the last leader" method, and the "unclean
>> > > recovery
>> > > >>>> >> manager"
>> > > >>>> >> >> method. I understand why you did it this way -- "choose the
>> > last
>> > > >>>> >> leader" is
>> > > >>>> >> >> simple, and will help us deliver an implementation quickly,
>> > > while
>> > > >>>> the
>> > > >>>> >> URM
>> > > >>>> >> >> is preferable in the long term. My suggestion here is to
>> > > separate
>> > > >>>> the
>> > > >>>> >> >> decision of HOW to do unclean leader election from the
>> > decision
>> > > of
>> > > >>>> WHEN
>> > > >>>> >> to
>> > > >>>> >> >> do it.
>> > > >>>> >> >>
>> > > >>>> >> >> So in other words, have "unclean.leader.election.enable"
>> > specify
>> > > >>>> when we
>> > > >>>> >> >> do unclean leader election, and have a new configuration
>> like
>> > > >>>> >> >> "unclean.recovery.manager.enable" to determine if we use the
>> > > URM.
>> > > >>>> >> >> Presumably the URM will take some time to get fully stable,
>> so
>> > > >>>> this can
>> > > >>>> >> >> default to false for a while, and we can flip the default to
>> > > true
>> > > >>>> when
>> > > >>>> >> we
>> > > >>>> >> >> feel ready.
>> > > >>>> >> >>
>> > > >>>> >> >> The URM is somewhat under-described here. I think we need a
>> > few
>> > > >>>> >> >> configurations here for it. For example, we need a
>> > > configuration to
>> > > >>>> >> specify
>> > > >>>> >> >> how long it should wait for a broker to respond to its RPCs
>> > > before
>> > > >>>> >> moving
>> > > >>>> >> >> on. We also need to understand how the URM interacts with
>> > > >>>> >> >> unclean.leader.election.enable=always. I assume that with
>> > > "always"
>> > > >>>> we
>> > > >>>> >> will
>> > > >>>> >> >> just unconditionally use the URM rather than choosing
>> > randomly.
>> > > >>>> But this
>> > > >>>> >> >> should be spelled out in the KIP.
>> > > >>>> >> >>
>> > > >>>> >> >> >
>> > > >>>> >> >> > DescribeTopicRequest
>> > > >>>> >> >> >
>> > > >>>> >> >> >    1.
>> > > >>>> >> >> >    Yes, the plan is to replace the MetadataRequest with
>> the
>> > > >>>> >> >> >    DescribeTopicRequest for the admin clients. Will check
>> > the
>> > > >>>> details.
>> > > >>>> >> >>
>> > > >>>> >> >> Sounds good. But as I said, you need to specify how
>> > AdminClient
>> > > >>>> >> interacts
>> > > >>>> >> >> with the new request. This will involve adding some fields
>> to
>> > > >>>> >> >> TopicDescription.java. And you need to specify the changes
>> to
>> > > the
>> > > >>>> >> >> kafka-topics.sh command line tool. Otherwise we cannot use
>> the
>> > > >>>> tool to
>> > > >>>> >> see
>> > > >>>> >> >> the new information.
>> > > >>>> >> >>
>> > > >>>> >> >> The new requests, DescribeTopicRequest and
>> > > >>>> GetReplicaLogInfoRequest,
>> > > >>>> >> need
>> > > >>>> >> >> to have limits placed on them so that their size can't be
>> > > >>>> infinite. We
>> > > >>>> >> >> don't want to propagate the current problems of
>> > MetadataRequest,
>> > > >>>> where
>> > > >>>> >> >> clients can request massive responses that can mess up the
>> JVM
>> > > when
>> > > >>>> >> handled.
>> > > >>>> >> >>
>> > > >>>> >> >> Adding limits is simple for GetReplicaLogInfoRequest -- we
>> can
>> > > >>>> just say
>> > > >>>> >> >> that only 2000 partitions at a time can be requested. For
>> > > >>>> >> >> DescribeTopicRequest we can probably just limit to 20 topics
>> > or
>> > > >>>> >> something
>> > > >>>> >> >> like that, to avoid the complexity of doing pagination in
>> this
>> > > KIP.
>> > > >>>> >> >>
>> > > >>>> >> >> >    2.
>> > > >>>> >> >> >    I can let the broker load the ELR info so that they can
>> > > serve
>> > > >>>> the
>> > > >>>> >> >> >    DescribeTopicRequest as well.
>> > > >>>> >> >>
>> > > >>>> >> >> Yes, it's fine to add to MetadataCache. In fact, you'll be
>> > > loading
>> > > >>>> it
>> > > >>>> >> >> anyway once it's added to PartitionImage.
>> > > >>>> >> >>
>> > > >>>> >> >> >    3.
>> > > >>>> >> >> >    Yeah, it does not make sense to have the topic id if
>> > > >>>> >> >> >    DescribeTopicRequest is only used by the admin client.
>> > > >>>> >> >>
>> > > >>>> >> >> OK. That makes things simpler. We can always create a new
>> API
>> > > later
>> > > >>>> >> >> (hopefully not in this KIP!) to query by topic ID.
>> > > >>>> >> >>
>> > > >>>> >> >> >
>> > > >>>> >> >> >
>> > > >>>> >> >> > Metrics
>> > > >>>> >> >> >
>> > > >>>> >> >> > As for overall cluster health metrics, I think
>> under-min-ISR
>> > > is
>> > > >>>> still
>> > > >>>> >> a
>> > > >>>> >> >> > useful one. ELR is more like a safety belt. When the ELR
>> is
>> > > >>>> used, the
>> > > >>>> >> >> > cluster availability has already been impacted.
>> > > >>>> >> >> >
>> > > >>>> >> >> > Maybe we can have a metric to count the partitions that
>> > > sum(ISR,
>> > > >>>> ELR)
>> > > >>>> >> <
>> > > >>>> >> >> min
>> > > >>>> >> >> > ISR. What do you think?
>> > > >>>> >> >>
>> > > >>>> >> >> How about:
>> > > >>>> >> >>
>> > > >>>> >> >> A.  a metric for the totoal number of under-min-isr
>> > partitions?
>> > > We
>> > > >>>> don't
>> > > >>>> >> >> have that in Apache Kafka at the moment.
>> > > >>>> >> >>
>> > > >>>> >> >> B. a metric for the number of unclean leader elections we
>> did
>> > > (for
>> > > >>>> >> >> simplicity, it can reset to 0 on controller restart: we
>> expect
>> > > >>>> people to
>> > > >>>> >> >> monitor the change over time anyway)
>> > > >>>> >> >>
>> > > >>>> >> >> best,
>> > > >>>> >> >> Colin
>> > > >>>> >> >>
>> > > >>>> >> >>
>> > > >>>> >> >> >
>> > > >>>> >> >> > Yeah, for the ongoing unclean recoveries, the controller
>> can
>> > > >>>> keep an
>> > > >>>> >> >> > accurate count through failover because partition
>> > registration
>> > > >>>> can
>> > > >>>> >> >> indicate
>> > > >>>> >> >> > whether a recovery is needed. However, for the happened
>> > ones,
>> > > >>>> unless
>> > > >>>> >> we
>> > > >>>> >> >> > want to persist the number somewhere, we can only figure
>> it
>> > > out
>> > > >>>> from
>> > > >>>> >> the
>> > > >>>> >> >> > log.
>> > > >>>> >> >> >
>> > > >>>> >> >> > On Tue, Sep 12, 2023 at 3:16 PM Colin McCabe <
>> > > cmccabe@apache.org
>> > > >>>> >
>> > > >>>> >> wrote:
>> > > >>>> >> >> >
>> > > >>>> >> >> >> Also, we should have metrics that show what is going on
>> > with
>> > > >>>> regard
>> > > >>>> >> to
>> > > >>>> >> >> the
>> > > >>>> >> >> >> eligible replica set. I'm not sure exactly what to
>> suggest,
>> > > but
>> > > >>>> >> >> something
>> > > >>>> >> >> >> that could identify when things are going wrong in the
>> > > clsuter.
>> > > >>>> >> >> >>
>> > > >>>> >> >> >> For example, maybe a metric for partitions containing
>> > > replicas
>> > > >>>> that
>> > > >>>> >> are
>> > > >>>> >> >> >> ineligible to be leader? That would show a spike when a
>> > > broker
>> > > >>>> had an
>> > > >>>> >> >> >> unclean restart.
>> > > >>>> >> >> >>
>> > > >>>> >> >> >> Ideally, we'd also have a metric that indicates when an
>> > > unclear
>> > > >>>> >> leader
>> > > >>>> >> >> >> election or a recovery happened. It's a bit tricky
>> because
>> > > the
>> > > >>>> simple
>> > > >>>> >> >> >> thing, of tracking it per controller, may be a bit
>> > confusing
>> > > >>>> during
>> > > >>>> >> >> >> failovers.
>> > > >>>> >> >> >>
>> > > >>>> >> >> >> best,
>> > > >>>> >> >> >> Colin
>> > > >>>> >> >> >>
>> > > >>>> >> >> >>
>> > > >>>> >> >> >> On Tue, Sep 12, 2023, at 14:25, Colin McCabe wrote:
>> > > >>>> >> >> >> > Hi Calvin,
>> > > >>>> >> >> >> >
>> > > >>>> >> >> >> > Thanks for the KIP. I think this is a great
>> improvement.
>> > > >>>> >> >> >> >
>> > > >>>> >> >> >> >> Additional High Watermark advance requirement
>> > > >>>> >> >> >> >
>> > > >>>> >> >> >> > Typo: change "advance" to "advancement"
>> > > >>>> >> >> >> >
>> > > >>>> >> >> >> >> A bit recap of some key concepts.
>> > > >>>> >> >> >> >
>> > > >>>> >> >> >> > Typo: change "bit" to "quick"
>> > > >>>> >> >> >> >
>> > > >>>> >> >> >> >> Ack=1/all produce request. It defines when the Kafka
>> > > server
>> > > >>>> should
>> > > >>>> >> >> >> respond to the produce request
>> > > >>>> >> >> >> >
>> > > >>>> >> >> >> > I think this section would be clearer if we talked
>> about
>> > > the
>> > > >>>> new
>> > > >>>> >> high
>> > > >>>> >> >> >> > watermark advancement requirement first, and THEN
>> talked
>> > > >>>> about its
>> > > >>>> >> >> >> > impact on acks=0, acks=1, and     acks=all.  acks=all
>> is
>> > of
>> > > >>>> course
>> > > >>>> >> the
>> > > >>>> >> >> >> > main case we care about here, so it would be good to
>> lead
>> > > with
>> > > >>>> >> that,
>> > > >>>> >> >> >> > rather than delving into the technicalities of acks=0/1
>> > > first.
>> > > >>>> >> >> >> >
>> > > >>>> >> >> >> >> Unclean recovery
>> > > >>>> >> >> >> >
>> > > >>>> >> >> >> > So, here you are introducing a new configuration,
>> > > >>>> >> >> >> > unclean.recovery.strategy. The difficult thing here is
>> > that
>> > > >>>> there
>> > > >>>> >> is a
>> > > >>>> >> >> >> > lot of overlap with unclean.leader.election.enable. So
>> we
>> > > >>>> have 3
>> > > >>>> >> >> >> > different settings for unclean.recovery.strategy, plus
>> 2
>> > > >>>> different
>> > > >>>> >> >> >> > settings for unclean.leader.election.enable, giving a
>> > cross
>> > > >>>> >> product of
>> > > >>>> >> >> >> > 6 different options. The following "unclean recovery
>> > > manager"
>> > > >>>> >> section
>> > > >>>> >> >> >> > only applies to one fo those 6 different possibilities
>> (I
>> > > >>>> think?)
>> > > >>>> >> >> >> >
>> > > >>>> >> >> >> > I simply don't think we need so many different election
>> > > types.
>> > > >>>> >> Really
>> > > >>>> >> >> >> > the use-cases we need are people who want NO unclean
>> > > >>>> elections,
>> > > >>>> >> people
>> > > >>>> >> >> >> > who want "the reasonable thing" and people who want
>> > > >>>> avaialbility at
>> > > >>>> >> >> all
>> > > >>>> >> >> >> > costs.
>> > > >>>> >> >> >> >
>> > > >>>> >> >> >> > Overall, I feel like the first half of the KIP is about
>> > the
>> > > >>>> ELR,
>> > > >>>> >> and
>> > > >>>> >> >> >> > the second half is about reworking unclean leader
>> > > election. It
>> > > >>>> >> might
>> > > >>>> >> >> be
>> > > >>>> >> >> >> > better to move that second half to a separate KIP so
>> that
>> > > we
>> > > >>>> can
>> > > >>>> >> >> figure
>> > > >>>> >> >> >> > it out fully. It should be fine to punt this until
>> later
>> > > and
>> > > >>>> just
>> > > >>>> >> have
>> > > >>>> >> >> >> > the current behavior on empty ELR be waiting for the
>> last
>> > > >>>> known
>> > > >>>> >> leader
>> > > >>>> >> >> >> > to return. After all, that's what we do today.
>> > > >>>> >> >> >> >
>> > > >>>> >> >> >> >> DescribeTopicRequest
>> > > >>>> >> >> >> >
>> > > >>>> >> >> >> > Is the intention for AdminClient to use this RPC for
>> > > >>>> >> >> >> > Admin#describeTopics ? If so, we need to describe all
>> of
>> > > the
>> > > >>>> >> changes
>> > > >>>> >> >> to
>> > > >>>> >> >> >> > the admin client API, as well as changes to
>> command-line
>> > > >>>> tools like
>> > > >>>> >> >> >> > kafka-topics.sh (if there are any). For example, you
>> will
>> > > >>>> probably
>> > > >>>> >> >> need
>> > > >>>> >> >> >> > changes to TopicDescription.java. You will also need to
>> > > >>>> provide
>> > > >>>> >> all of
>> > > >>>> >> >> >> > the things that admin client needs -- for example,
>> > > >>>> >> >> >> > TopicAuthorizedOperations.
>> > > >>>> >> >> >> >
>> > > >>>> >> >> >> > I also don't think the controller should serve this
>> > > request.
>> > > >>>> We
>> > > >>>> >> want
>> > > >>>> >> >> to
>> > > >>>> >> >> >> > minimize load on the controller. Just like with the
>> other
>> > > >>>> metadata
>> > > >>>> >> >> >> > requests like MetadataRequest, this should be served by
>> > > >>>> brokers.
>> > > >>>> >> >> >> >
>> > > >>>> >> >> >> > It's a bit confusing why both topic ID and topic name
>> are
>> > > >>>> provided
>> > > >>>> >> to
>> > > >>>> >> >> >> > this API. Is the intention that callers should set one
>> > but
>> > > >>>> not the
>> > > >>>> >> >> >> > other? Or both? This needs to be clarified. Also, if we
>> > do
>> > > >>>> want to
>> > > >>>> >> >> >> > support lookups by UUID, that is another thing that
>> needs
>> > > to
>> > > >>>> be
>> > > >>>> >> added
>> > > >>>> >> >> >> > to adminclient.
>> > > >>>> >> >> >> >
>> > > >>>> >> >> >> > In general, I feel like this should also probably be
>> its
>> > > own
>> > > >>>> KIP
>> > > >>>> >> since
>> > > >>>> >> >> >> > it's fairly complex
>> > > >>>> >> >> >> >
>> > > >>>> >> >> >> > best,
>> > > >>>> >> >> >> > Colin
>> > > >>>> >> >> >> >
>> > > >>>> >> >> >> >
>> > > >>>> >> >> >> > On Thu, Aug 10, 2023, at 15:46, Calvin Liu 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
>> > > >>>> >> >> >>
>> > > >>>> >> >>
>> > > >>>> >>
>> > > >>>>
>> > > >>>
>> > >
>> >
>>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

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

Thanks for the update KIP. A few more comments.

41. Why would a user choose the option to select a random replica as the
leader instead of using unclean.recovery.strateg=Aggressive? It seems that
the latter is strictly better? If that's not the case, could we fold this
option under unclean.recovery.strategy instead of introducing a separate
config?

50. ElectLeadersRequest: "If more than 20 topics are included, only the
first 20 will be served. Others will be returned with DesiredLeaders." Hmm,
not sure that I understand this. ElectLeadersResponse doesn't have a
DesiredLeaders field.

51. GetReplicaLogInfo: "If more than 2000 partitions are included, only the
first 2000 will be served" Do we return an error for the remaining
partitions? Actually, should we include an errorCode field at the partition
level in GetReplicaLogInfoResponse to cover non-existing partitions and no
authorization, etc?

52. The entry should matches => The entry should match

53. ElectLeadersRequest.DesiredLeaders: Should it be nullable since a user
may not specify DesiredLeaders?

54. Downgrade: Is that indeed possible? I thought earlier you said that
once the new version of the records are in the metadata log, one can't
downgrade since the old broker doesn't know how to parse the new version of
the metadata records?

55. CleanShutdownFile: Should we add a version field for future extension?

56. Config changes are public facing. Could we have a separate section to
document all the config changes?

Thanks,

Jun

On Mon, Sep 25, 2023 at 4:29 PM Calvin Liu <ca...@confluent.io.invalid>
wrote:

> Hi Jun
> Thanks for the comments.
>
> 40. If we change to None, it is not guaranteed for no data loss. For users
> who are not able to validate the data with external resources, manual
> intervention does not give a better result but a loss of availability. So
> practically speaking, the Balance mode would be a better default value.
>
> 41. No, it represents how we want to do the unclean leader election. If it
> is false, the unclean leader election will be the old random way.
> Otherwise, the unclean recovery will be used.
>
> 42. Good catch. Updated.
>
> 43. Only the first 20 topics will be served. Others will be returned with
> InvalidRequestError
>
> 44. The order matters. The desired leader entries match with the topic
> partition list by the index.
>
> 45. Thanks! Updated.
>
> 46. Good advice! Updated.
>
> 47.1, updated the comment. Basically it will elect the replica in the
> desiredLeader field to be the leader
>
> 47.2 We can let the admin client do the conversion. Using the desiredLeader
> field in the json format seems easier for users.
>
> 48. Once the MV version is downgraded, all the ELR related fields will be
> removed on the next partition change. The controller will also ignore the
> ELR fields. Updated the KIP.
>
> 49. Yes, it would be deprecated/removed.
>
>
> On Mon, Sep 25, 2023 at 3:49 PM Jun Rao <ju...@confluent.io.invalid> wrote:
>
> > Hi, Calvin,
> >
> > Thanks for the updated KIP. Made another pass. A few more comments below.
> >
> > 40. unclean.leader.election.enable.false ->
> > unclean.recovery.strategy.Balanced: The Balanced mode could still lead to
> > data loss. So, I am wondering if unclean.leader.election.enable.false
> > should map to None?
> >
> > 41. unclean.recovery.manager.enabled: I am not sure why we introduce this
> > additional config. Is it the same as unclean.recovery.strategy=None?
> >
> > 42. DescribeTopicResponse.TopicAuthorizedOperations: Should this be at
> the
> > topic level?
> >
> > 43. "Limit: 20 topics max per request": Could we describe what happens if
> > the request includes more than 20 topics?
> >
> > 44. ElectLeadersRequest.DesiredLeaders: Could we describe whether the
> > ordering matters?
> >
> > 45. GetReplicaLogInfo.TopicPartitions: "about": "The topic partitions to
> > elect leaders.": The description in "about" is incorrect.
> >
> > 46. GetReplicaLogInfoResponse: Should we nest partitions under topicId to
> > be consistent with other types of responses?
> >
> > 47. kafka-leader-election.sh:
> > 47.1 Could we explain DESIGNATION?
> > 47.2 desiredLeader: Should it be a list to match the field in
> > ElectLeadersRequest?
> >
> > 48. We could add a section on downgrade?
> >
> > 49. LastKnownLeader: This seems only needed in the first phase of
> > delivering ELR. Will it be removed when the complete KIP is delivered?
> >
> > Thanks,
> >
> > Jun
> >
> > On Tue, Sep 19, 2023 at 1:30 PM Colin McCabe <cm...@apache.org> wrote:
> >
> > > Hi Calvin,
> > >
> > > Thanks for the explanations. I like the idea of using none, balanced,
> > > aggressive. We also had an offline discussion about why it is good to
> > use a
> > > new config key (basically, so that we can deprecate the old one which
> had
> > > only false/true values in 4.0) With these changes, I am +1.
> > >
> > > best,
> > > Colin
> > >
> > > On Mon, Sep 18, 2023, at 15:54, Calvin Liu wrote:
> > > > Hi Colin,
> > > > Also, can we deprecate unclean.leader.election.enable in 4.0? Before
> > > that,
> > > > we can have both the config unclean.recovery.strategy and
> > > > unclean.leader.election.enable
> > > > and using the unclean.recovery.Enabled to determine which config to
> use
> > > > during the unclean leader election.
> > > >
> > > > On Mon, Sep 18, 2023 at 3:51 PM Calvin Liu <ca...@confluent.io>
> wrote:
> > > >
> > > >> Hi Colin,
> > > >> For the unclean.recovery.strategy config name, how about we use the
> > > >> following
> > > >> None. It basically means no unclean recovery will be performed.
> > > >> Aggressive. It means availability goes first. Whenever the partition
> > > can't
> > > >> elect a durable replica, the controller will try the unclean
> recovery.
> > > >> Balanced. It is the balance point of the availability
> > first(Aggressive)
> > > >> and least availability(None). The controller performs unclean
> recovery
> > > when
> > > >> both ISR and ELR are empty.
> > > >>
> > > >>
> > > >> On Fri, Sep 15, 2023 at 11:42 AM Calvin Liu <ca...@confluent.io>
> > wrote:
> > > >>
> > > >>> Hi Colin,
> > > >>>
> > > >>> > So, the proposal is that if someone sets
> > > "unclean.leader.election.enable
> > > >>> = true"...
> > > >>>
> > > >>>
> > > >>> The idea is to use one of the unclean.leader.election.enable and
> > > >>> unclean.recovery.strategy based on the unclean.recovery.Enabled. A
> > > possible
> > > >>> version can be
> > > >>>
> > > >>> If unclean.recovery.Enabled:
> > > >>>
> > > >>> {
> > > >>>
> > > >>> Check unclean.recovery.strategy. If set, use it. Otherwise, check
> > > >>> unclean.leader.election.enable and translate it to
> > > >>> unclean.recovery.strategy.
> > > >>>
> > > >>> } else {
> > > >>>
> > > >>> Use unclean.leader.election.enable
> > > >>>
> > > >>> }
> > > >>>
> > > >>>
> > > >>> —--------
> > > >>>
> > > >>> >The configuration key should be
> "unclean.recovery.manager.enabled",
> > > >>> right?
> > > >>>
> > > >>>
> > > >>> I think we have two ways of choosing a leader uncleanly, unclean
> > leader
> > > >>> election and unclean recovery(log inspection) and we try to switch
> > > between
> > > >>> them.
> > > >>>
> > > >>> Do you mean we want to develop two ways of performing the unclean
> > > >>> recovery and one of them is using “unclean recovery manager”? I
> guess
> > > we
> > > >>> haven’t discussed the second way.
> > > >>>
> > > >>>
> > > >>> —-------
> > > >>>
> > > >>> >How do these 4 levels of overrides interact with your new
> > > >>> configurations?
> > > >>>
> > > >>>
> > > >>> I do notice in the Kraft controller code, the method to check
> whether
> > > >>> perform unclean leader election is hard coded to false since
> > > >>> 2021(uncleanLeaderElectionEnabledForTopic). Isn’t it a good chance
> to
> > > >>> completely deprecate the unclean.leader.election.enable? We don’t
> > even
> > > have
> > > >>> to worry about the config conversion.
> > > >>>
> > > >>> On the other hand, whatever the override is, as long as the
> > controller
> > > >>> can have the final effective unclean.leader.election.enable, the
> > topic
> > > >>> level config unclean.recovery.strategy, the cluster level config
> > > >>> unclean.recovery.Enabled, the controller can calculate the correct
> > > methods
> > > >>> to use right?
> > > >>>
> > > >>>
> > > >>> On Fri, Sep 15, 2023 at 10:02 AM Colin McCabe <cm...@apache.org>
> > > wrote:
> > > >>>
> > > >>>> On Thu, Sep 14, 2023, at 22:23, Calvin Liu wrote:
> > > >>>> > Hi Colin
> > > >>>> > 1. I think using the new config name is more clear.
> > > >>>> >        a. The unclean leader election is actually removed if
> > unclean
> > > >>>> > recovery is in use.
> > > >>>> >        b. Using multiple values in
> unclean.leader.election.enable
> > is
> > > >>>> > confusing and it will be more confusing after people forget
> about
> > > this
> > > >>>> > discussion.
> > > >>>>
> > > >>>> Hi Calvin,
> > > >>>>
> > > >>>> So, the proposal is that if someone sets
> > > "unclean.leader.election.enable
> > > >>>> = true" but then sets one of your new configurations, the value of
> > > >>>> unclean.leader.election.enable is ignored? That seems less clear
> to
> > > me, not
> > > >>>> more. Just in general, having multiple configuration keys to
> control
> > > the
> > > >>>> same thing confuses users. Basically, they are sitting at a giant
> > > control
> > > >>>> panel, and some of the levers do nothing.
> > > >>>>
> > > >>>> > 2. Sorry I forgot to mention in the response that I did add the
> > > >>>> > unclean.recovery.Enabled flag.
> > > >>>>
> > > >>>> The configuration key should be
> "unclean.recovery.manager.enabled",
> > > >>>> right? Becuase we can do "unclean recovery" without the manager.
> > > Disabling
> > > >>>> the manager just means we use a different mechanism for recovery.
> > > >>>>
> > > >>>> >        c. Maybe I underestimated the challenge of replacing the
> > > >>>> config. Any
> > > >>>> > implementation problems ahead?
> > > >>>>
> > > >>>> There are four levels of overrides for
> > unclean.leader.election.enable.
> > > >>>>
> > > >>>> 1. static configuration for node.
> > > >>>>     This goes in the configuration file, typically named
> > > >>>> server.properties
> > > >>>>
> > > >>>> 2. dynamic configuration for node default
> > > >>>>   ConfigResource(type=BROKER, name="")
> > > >>>>
> > > >>>> 3. dynamic configuration for node
> > > >>>>   ConfigResource(type=BROKER, name=<controller id>)
> > > >>>>
> > > >>>> 4. dynamic configuration for topic
> > > >>>>   ConfigResource(type=TOPIC, name=<topic-name>)
> > > >>>>
> > > >>>> How do these 4 levels of overrides interact with your new
> > > >>>> configurations? If the new configurations dominate over the old
> > ones,
> > > it
> > > >>>> seems like this will get a lot more confusing to implement (and
> also
> > > to
> > > >>>> use.)
> > > >>>>
> > > >>>> Again, I'd recommend just adding some new values to
> > > >>>> unclean.leader.election.enable. It's simple and will prevent user
> > > confusion
> > > >>>> (as well as developer confusion.)
> > > >>>>
> > > >>>> best,
> > > >>>> Colin
> > > >>>>
> > > >>>>
> > > >>>> > 3. About the admin client, I mentioned 3 changes in the client.
> > > >>>> Anything
> > > >>>> > else I missed in the KIP?
> > > >>>> >       a. The client will switch to using the new RPC instead of
> > > >>>> > MetadataRequest for the topics.
> > > >>>> >       b. The TopicPartitionInfo used in TopicDescription needs
> to
> > > add
> > > >>>> new
> > > >>>> > fields related to the ELR.
> > > >>>> >       c. The outputs will add the ELR related fields.
> > > >>>> >
> > > >>>> > On Thu, Sep 14, 2023 at 9:19 PM Colin McCabe <
> cmccabe@apache.org>
> > > >>>> wrote:
> > > >>>> >
> > > >>>> >> Hi Calvin,
> > > >>>> >>
> > > >>>> >> Thanks for the changes.
> > > >>>> >>
> > > >>>> >> 1. Earlier I commented that creating
> "unclean.recovery.strategy "
> > > is
> > > >>>> not
> > > >>>> >> necessary, and we can just reuse the existing
> > > >>>> >> "unclean.leader.election.enable" configuration key. Let's
> discuss
> > > >>>> that.
> > > >>>> >>
> > > >>>> >> 2.I also don't understand why you didn't add a configuration to
> > > >>>> enable or
> > > >>>> >> disable the Unclean Recovery Manager. This seems like a very
> > simple
> > > >>>> way to
> > > >>>> >> handle the staging issue which we discussed. The URM can just
> be
> > > >>>> turned off
> > > >>>> >> until it is production ready. Let's discuss this.
> > > >>>> >>
> > > >>>> >> 3. You still need to describe the changes to AdminClient that
> are
> > > >>>> needed
> > > >>>> >> to use DescribeTopicRequest.
> > > >>>> >>
> > > >>>> >> Keep at it. It's looking better. :)
> > > >>>> >>
> > > >>>> >> best,
> > > >>>> >> Colin
> > > >>>> >>
> > > >>>> >>
> > > >>>> >> On Thu, Sep 14, 2023, at 11:03, Calvin Liu wrote:
> > > >>>> >> > Hi Colin
> > > >>>> >> > Thanks for the comments!
> > > >>>> >> >
> > > >>>> >> > I did the following changes
> > > >>>> >> >
> > > >>>> >> >    1.
> > > >>>> >> >
> > > >>>> >> >    Simplified the API spec section to only include the diff.
> > > >>>> >> >    2.
> > > >>>> >> >
> > > >>>> >> >    Reordered the HWM requirement section.
> > > >>>> >> >    3.
> > > >>>> >> >
> > > >>>> >> >    Removed the URM implementation details to keep the
> necessary
> > > >>>> >> >    characteristics to perform the unclean recovery.
> > > >>>> >> >    1.
> > > >>>> >> >
> > > >>>> >> >       When to perform the unclean recovery
> > > >>>> >> >       2.
> > > >>>> >> >
> > > >>>> >> >       Under different config, how the unclean recovery finds
> > the
> > > >>>> leader.
> > > >>>> >> >       3.
> > > >>>> >> >
> > > >>>> >> >       How the config unclean.leader.election.enable and
> > > >>>> >> >       unclean.recovery.strategy are converted when users
> > > >>>> enable/disable
> > > >>>> >> the
> > > >>>> >> >       unclean recovery.
> > > >>>> >> >       4.
> > > >>>> >> >
> > > >>>> >> >    More details about how we change admin client.
> > > >>>> >> >    5.
> > > >>>> >> >
> > > >>>> >> >    API limits on the GetReplicaLogInfoRequest and
> > > >>>> DescribeTopicRequest.
> > > >>>> >> >    6.
> > > >>>> >> >
> > > >>>> >> >    Two metrics added
> > > >>>> >> >    1.
> > > >>>> >> >
> > > >>>> >> >       Kafka.controller.global_under_min_isr_partition_count
> > > >>>> >> >       2.
> > > >>>> >> >
> > > >>>> >> >       kafka.controller.unclean_recovery_finished_count
> > > >>>> >> >
> > > >>>> >> >
> > > >>>> >> > On Wed, Sep 13, 2023 at 10:46 AM Colin McCabe <
> > > cmccabe@apache.org>
> > > >>>> >> wrote:
> > > >>>> >> >
> > > >>>> >> >> On Tue, Sep 12, 2023, at 17:21, Calvin Liu wrote:
> > > >>>> >> >> > Hi Colin
> > > >>>> >> >> > Thanks for the comments!
> > > >>>> >> >> >
> > > >>>> >> >>
> > > >>>> >> >> Hi Calvin,
> > > >>>> >> >>
> > > >>>> >> >> Thanks again for the KIP.
> > > >>>> >> >>
> > > >>>> >> >> One meta-comment: it's usually better to just do a diff on a
> > > >>>> message
> > > >>>> >> spec
> > > >>>> >> >> file or java file if you're including changes to it in the
> > KIP.
> > > >>>> This is
> > > >>>> >> >> easier to read than looking for "new fields begin" etc. in
> the
> > > >>>> text, and
> > > >>>> >> >> gracefully handles the case where existing fields were
> > changed.
> > > >>>> >> >>
> > > >>>> >> >> > Rewrite the Additional High Watermark advancement
> > requirement
> > > >>>> >> >> > There was feedback on this section that some readers may
> not
> > > be
> > > >>>> >> familiar
> > > >>>> >> >> > with HWM and Ack=0,1,all requests. This can help them
> > > understand
> > > >>>> the
> > > >>>> >> >> > proposal. I will rewrite this part for more readability.
> > > >>>> >> >> >
> > > >>>> >> >>
> > > >>>> >> >> To be clear, I wasn't suggesting dropping either section. I
> > > agree
> > > >>>> that
> > > >>>> >> >> they add useful background. I was just suggesting that we
> > should
> > > >>>> discuss
> > > >>>> >> >> the "acks" setting AFTER discussing the new high watermark
> > > >>>> advancement
> > > >>>> >> >> conditions. We also should discuss acks=0. While it isn't
> > > >>>> conceptually
> > > >>>> >> much
> > > >>>> >> >> different than acks=1 here, its omission from this section
> is
> > > >>>> confusing.
> > > >>>> >> >>
> > > >>>> >> >> > Unclean recovery
> > > >>>> >> >> >
> > > >>>> >> >> > The plan is to replace the unclean.leader.election.enable
> > with
> > > >>>> >> >> > unclean.recovery.strategy. If the Unclean Recovery is
> > enabled
> > > >>>> then it
> > > >>>> >> >> deals
> > > >>>> >> >> > with the three options in the unclean.recovery.strategy.
> > > >>>> >> >> >
> > > >>>> >> >> >
> > > >>>> >> >> > Let’s refine the Unclean Recovery. We have already taken a
> > > lot of
> > > >>>> >> >> > suggestions and I hope to enhance the durability of Kafka
> to
> > > the
> > > >>>> next
> > > >>>> >> >> level
> > > >>>> >> >> > with this KIP.
> > > >>>> >> >>
> > > >>>> >> >> I am OK with doing the unclean leader recovery improvements
> in
> > > >>>> this KIP.
> > > >>>> >> >> However, I think we need to really work on the configuration
> > > >>>> settings.
> > > >>>> >> >>
> > > >>>> >> >> Configuration overrides are often quite messy. For example,
> > the
> > > >>>> cases
> > > >>>> >> >> where we have log.roll.hours and log.roll.segment.ms, the
> > user
> > > >>>> has to
> > > >>>> >> >> remember which one takes precedence, and it is not obvious.
> > So,
> > > >>>> rather
> > > >>>> >> than
> > > >>>> >> >> creating a new configuration, why not add additional values
> to
> > > >>>> >> >> "unclean.leader.election.enable"? I think this will be
> simpler
> > > for
> > > >>>> >> people
> > > >>>> >> >> to understand, and simpler in the code as well.
> > > >>>> >> >>
> > > >>>> >> >> What if we continued to use "unclean.leader.election.enable"
> > but
> > > >>>> >> extended
> > > >>>> >> >> it so that it took a string? Then the string could have
> these
> > > >>>> values:
> > > >>>> >> >>
> > > >>>> >> >> never
> > > >>>> >> >>     never automatically do an unclean leader election under
> > any
> > > >>>> >> conditions
> > > >>>> >> >>
> > > >>>> >> >> false / default
> > > >>>> >> >>     only do an unclean leader election if there may be
> > possible
> > > >>>> data
> > > >>>> >> loss
> > > >>>> >> >>
> > > >>>> >> >> true / always
> > > >>>> >> >>     always do an unclean leader election if we can't
> > immediately
> > > >>>> elect a
> > > >>>> >> >> leader
> > > >>>> >> >>
> > > >>>> >> >> It's a bit awkward that false maps to default rather than to
> > > >>>> never. But
> > > >>>> >> >> this awkwardness exists if we use two different
> configuration
> > > keys
> > > >>>> as
> > > >>>> >> well.
> > > >>>> >> >> The reason for the awkwardness is that we simply don't want
> > most
> > > >>>> of the
> > > >>>> >> >> people currently setting
> unclean.leader.election.enable=false
> > to
> > > >>>> get the
> > > >>>> >> >> "never" behavior. We have to bite that bullet. Better to be
> > > clear
> > > >>>> and
> > > >>>> >> >> explicit than hide it.
> > > >>>> >> >>
> > > >>>> >> >> Another thing that's a bit awkward is having two different
> > ways
> > > to
> > > >>>> do
> > > >>>> >> >> unclean leader election specified in the KIP. You descirbe
> two
> > > >>>> methods:
> > > >>>> >> the
> > > >>>> >> >> simple "choose the last leader" method, and the "unclean
> > > recovery
> > > >>>> >> manager"
> > > >>>> >> >> method. I understand why you did it this way -- "choose the
> > last
> > > >>>> >> leader" is
> > > >>>> >> >> simple, and will help us deliver an implementation quickly,
> > > while
> > > >>>> the
> > > >>>> >> URM
> > > >>>> >> >> is preferable in the long term. My suggestion here is to
> > > separate
> > > >>>> the
> > > >>>> >> >> decision of HOW to do unclean leader election from the
> > decision
> > > of
> > > >>>> WHEN
> > > >>>> >> to
> > > >>>> >> >> do it.
> > > >>>> >> >>
> > > >>>> >> >> So in other words, have "unclean.leader.election.enable"
> > specify
> > > >>>> when we
> > > >>>> >> >> do unclean leader election, and have a new configuration
> like
> > > >>>> >> >> "unclean.recovery.manager.enable" to determine if we use the
> > > URM.
> > > >>>> >> >> Presumably the URM will take some time to get fully stable,
> so
> > > >>>> this can
> > > >>>> >> >> default to false for a while, and we can flip the default to
> > > true
> > > >>>> when
> > > >>>> >> we
> > > >>>> >> >> feel ready.
> > > >>>> >> >>
> > > >>>> >> >> The URM is somewhat under-described here. I think we need a
> > few
> > > >>>> >> >> configurations here for it. For example, we need a
> > > configuration to
> > > >>>> >> specify
> > > >>>> >> >> how long it should wait for a broker to respond to its RPCs
> > > before
> > > >>>> >> moving
> > > >>>> >> >> on. We also need to understand how the URM interacts with
> > > >>>> >> >> unclean.leader.election.enable=always. I assume that with
> > > "always"
> > > >>>> we
> > > >>>> >> will
> > > >>>> >> >> just unconditionally use the URM rather than choosing
> > randomly.
> > > >>>> But this
> > > >>>> >> >> should be spelled out in the KIP.
> > > >>>> >> >>
> > > >>>> >> >> >
> > > >>>> >> >> > DescribeTopicRequest
> > > >>>> >> >> >
> > > >>>> >> >> >    1.
> > > >>>> >> >> >    Yes, the plan is to replace the MetadataRequest with
> the
> > > >>>> >> >> >    DescribeTopicRequest for the admin clients. Will check
> > the
> > > >>>> details.
> > > >>>> >> >>
> > > >>>> >> >> Sounds good. But as I said, you need to specify how
> > AdminClient
> > > >>>> >> interacts
> > > >>>> >> >> with the new request. This will involve adding some fields
> to
> > > >>>> >> >> TopicDescription.java. And you need to specify the changes
> to
> > > the
> > > >>>> >> >> kafka-topics.sh command line tool. Otherwise we cannot use
> the
> > > >>>> tool to
> > > >>>> >> see
> > > >>>> >> >> the new information.
> > > >>>> >> >>
> > > >>>> >> >> The new requests, DescribeTopicRequest and
> > > >>>> GetReplicaLogInfoRequest,
> > > >>>> >> need
> > > >>>> >> >> to have limits placed on them so that their size can't be
> > > >>>> infinite. We
> > > >>>> >> >> don't want to propagate the current problems of
> > MetadataRequest,
> > > >>>> where
> > > >>>> >> >> clients can request massive responses that can mess up the
> JVM
> > > when
> > > >>>> >> handled.
> > > >>>> >> >>
> > > >>>> >> >> Adding limits is simple for GetReplicaLogInfoRequest -- we
> can
> > > >>>> just say
> > > >>>> >> >> that only 2000 partitions at a time can be requested. For
> > > >>>> >> >> DescribeTopicRequest we can probably just limit to 20 topics
> > or
> > > >>>> >> something
> > > >>>> >> >> like that, to avoid the complexity of doing pagination in
> this
> > > KIP.
> > > >>>> >> >>
> > > >>>> >> >> >    2.
> > > >>>> >> >> >    I can let the broker load the ELR info so that they can
> > > serve
> > > >>>> the
> > > >>>> >> >> >    DescribeTopicRequest as well.
> > > >>>> >> >>
> > > >>>> >> >> Yes, it's fine to add to MetadataCache. In fact, you'll be
> > > loading
> > > >>>> it
> > > >>>> >> >> anyway once it's added to PartitionImage.
> > > >>>> >> >>
> > > >>>> >> >> >    3.
> > > >>>> >> >> >    Yeah, it does not make sense to have the topic id if
> > > >>>> >> >> >    DescribeTopicRequest is only used by the admin client.
> > > >>>> >> >>
> > > >>>> >> >> OK. That makes things simpler. We can always create a new
> API
> > > later
> > > >>>> >> >> (hopefully not in this KIP!) to query by topic ID.
> > > >>>> >> >>
> > > >>>> >> >> >
> > > >>>> >> >> >
> > > >>>> >> >> > Metrics
> > > >>>> >> >> >
> > > >>>> >> >> > As for overall cluster health metrics, I think
> under-min-ISR
> > > is
> > > >>>> still
> > > >>>> >> a
> > > >>>> >> >> > useful one. ELR is more like a safety belt. When the ELR
> is
> > > >>>> used, the
> > > >>>> >> >> > cluster availability has already been impacted.
> > > >>>> >> >> >
> > > >>>> >> >> > Maybe we can have a metric to count the partitions that
> > > sum(ISR,
> > > >>>> ELR)
> > > >>>> >> <
> > > >>>> >> >> min
> > > >>>> >> >> > ISR. What do you think?
> > > >>>> >> >>
> > > >>>> >> >> How about:
> > > >>>> >> >>
> > > >>>> >> >> A.  a metric for the totoal number of under-min-isr
> > partitions?
> > > We
> > > >>>> don't
> > > >>>> >> >> have that in Apache Kafka at the moment.
> > > >>>> >> >>
> > > >>>> >> >> B. a metric for the number of unclean leader elections we
> did
> > > (for
> > > >>>> >> >> simplicity, it can reset to 0 on controller restart: we
> expect
> > > >>>> people to
> > > >>>> >> >> monitor the change over time anyway)
> > > >>>> >> >>
> > > >>>> >> >> best,
> > > >>>> >> >> Colin
> > > >>>> >> >>
> > > >>>> >> >>
> > > >>>> >> >> >
> > > >>>> >> >> > Yeah, for the ongoing unclean recoveries, the controller
> can
> > > >>>> keep an
> > > >>>> >> >> > accurate count through failover because partition
> > registration
> > > >>>> can
> > > >>>> >> >> indicate
> > > >>>> >> >> > whether a recovery is needed. However, for the happened
> > ones,
> > > >>>> unless
> > > >>>> >> we
> > > >>>> >> >> > want to persist the number somewhere, we can only figure
> it
> > > out
> > > >>>> from
> > > >>>> >> the
> > > >>>> >> >> > log.
> > > >>>> >> >> >
> > > >>>> >> >> > On Tue, Sep 12, 2023 at 3:16 PM Colin McCabe <
> > > cmccabe@apache.org
> > > >>>> >
> > > >>>> >> wrote:
> > > >>>> >> >> >
> > > >>>> >> >> >> Also, we should have metrics that show what is going on
> > with
> > > >>>> regard
> > > >>>> >> to
> > > >>>> >> >> the
> > > >>>> >> >> >> eligible replica set. I'm not sure exactly what to
> suggest,
> > > but
> > > >>>> >> >> something
> > > >>>> >> >> >> that could identify when things are going wrong in the
> > > clsuter.
> > > >>>> >> >> >>
> > > >>>> >> >> >> For example, maybe a metric for partitions containing
> > > replicas
> > > >>>> that
> > > >>>> >> are
> > > >>>> >> >> >> ineligible to be leader? That would show a spike when a
> > > broker
> > > >>>> had an
> > > >>>> >> >> >> unclean restart.
> > > >>>> >> >> >>
> > > >>>> >> >> >> Ideally, we'd also have a metric that indicates when an
> > > unclear
> > > >>>> >> leader
> > > >>>> >> >> >> election or a recovery happened. It's a bit tricky
> because
> > > the
> > > >>>> simple
> > > >>>> >> >> >> thing, of tracking it per controller, may be a bit
> > confusing
> > > >>>> during
> > > >>>> >> >> >> failovers.
> > > >>>> >> >> >>
> > > >>>> >> >> >> best,
> > > >>>> >> >> >> Colin
> > > >>>> >> >> >>
> > > >>>> >> >> >>
> > > >>>> >> >> >> On Tue, Sep 12, 2023, at 14:25, Colin McCabe wrote:
> > > >>>> >> >> >> > Hi Calvin,
> > > >>>> >> >> >> >
> > > >>>> >> >> >> > Thanks for the KIP. I think this is a great
> improvement.
> > > >>>> >> >> >> >
> > > >>>> >> >> >> >> Additional High Watermark advance requirement
> > > >>>> >> >> >> >
> > > >>>> >> >> >> > Typo: change "advance" to "advancement"
> > > >>>> >> >> >> >
> > > >>>> >> >> >> >> A bit recap of some key concepts.
> > > >>>> >> >> >> >
> > > >>>> >> >> >> > Typo: change "bit" to "quick"
> > > >>>> >> >> >> >
> > > >>>> >> >> >> >> Ack=1/all produce request. It defines when the Kafka
> > > server
> > > >>>> should
> > > >>>> >> >> >> respond to the produce request
> > > >>>> >> >> >> >
> > > >>>> >> >> >> > I think this section would be clearer if we talked
> about
> > > the
> > > >>>> new
> > > >>>> >> high
> > > >>>> >> >> >> > watermark advancement requirement first, and THEN
> talked
> > > >>>> about its
> > > >>>> >> >> >> > impact on acks=0, acks=1, and     acks=all.  acks=all
> is
> > of
> > > >>>> course
> > > >>>> >> the
> > > >>>> >> >> >> > main case we care about here, so it would be good to
> lead
> > > with
> > > >>>> >> that,
> > > >>>> >> >> >> > rather than delving into the technicalities of acks=0/1
> > > first.
> > > >>>> >> >> >> >
> > > >>>> >> >> >> >> Unclean recovery
> > > >>>> >> >> >> >
> > > >>>> >> >> >> > So, here you are introducing a new configuration,
> > > >>>> >> >> >> > unclean.recovery.strategy. The difficult thing here is
> > that
> > > >>>> there
> > > >>>> >> is a
> > > >>>> >> >> >> > lot of overlap with unclean.leader.election.enable. So
> we
> > > >>>> have 3
> > > >>>> >> >> >> > different settings for unclean.recovery.strategy, plus
> 2
> > > >>>> different
> > > >>>> >> >> >> > settings for unclean.leader.election.enable, giving a
> > cross
> > > >>>> >> product of
> > > >>>> >> >> >> > 6 different options. The following "unclean recovery
> > > manager"
> > > >>>> >> section
> > > >>>> >> >> >> > only applies to one fo those 6 different possibilities
> (I
> > > >>>> think?)
> > > >>>> >> >> >> >
> > > >>>> >> >> >> > I simply don't think we need so many different election
> > > types.
> > > >>>> >> Really
> > > >>>> >> >> >> > the use-cases we need are people who want NO unclean
> > > >>>> elections,
> > > >>>> >> people
> > > >>>> >> >> >> > who want "the reasonable thing" and people who want
> > > >>>> avaialbility at
> > > >>>> >> >> all
> > > >>>> >> >> >> > costs.
> > > >>>> >> >> >> >
> > > >>>> >> >> >> > Overall, I feel like the first half of the KIP is about
> > the
> > > >>>> ELR,
> > > >>>> >> and
> > > >>>> >> >> >> > the second half is about reworking unclean leader
> > > election. It
> > > >>>> >> might
> > > >>>> >> >> be
> > > >>>> >> >> >> > better to move that second half to a separate KIP so
> that
> > > we
> > > >>>> can
> > > >>>> >> >> figure
> > > >>>> >> >> >> > it out fully. It should be fine to punt this until
> later
> > > and
> > > >>>> just
> > > >>>> >> have
> > > >>>> >> >> >> > the current behavior on empty ELR be waiting for the
> last
> > > >>>> known
> > > >>>> >> leader
> > > >>>> >> >> >> > to return. After all, that's what we do today.
> > > >>>> >> >> >> >
> > > >>>> >> >> >> >> DescribeTopicRequest
> > > >>>> >> >> >> >
> > > >>>> >> >> >> > Is the intention for AdminClient to use this RPC for
> > > >>>> >> >> >> > Admin#describeTopics ? If so, we need to describe all
> of
> > > the
> > > >>>> >> changes
> > > >>>> >> >> to
> > > >>>> >> >> >> > the admin client API, as well as changes to
> command-line
> > > >>>> tools like
> > > >>>> >> >> >> > kafka-topics.sh (if there are any). For example, you
> will
> > > >>>> probably
> > > >>>> >> >> need
> > > >>>> >> >> >> > changes to TopicDescription.java. You will also need to
> > > >>>> provide
> > > >>>> >> all of
> > > >>>> >> >> >> > the things that admin client needs -- for example,
> > > >>>> >> >> >> > TopicAuthorizedOperations.
> > > >>>> >> >> >> >
> > > >>>> >> >> >> > I also don't think the controller should serve this
> > > request.
> > > >>>> We
> > > >>>> >> want
> > > >>>> >> >> to
> > > >>>> >> >> >> > minimize load on the controller. Just like with the
> other
> > > >>>> metadata
> > > >>>> >> >> >> > requests like MetadataRequest, this should be served by
> > > >>>> brokers.
> > > >>>> >> >> >> >
> > > >>>> >> >> >> > It's a bit confusing why both topic ID and topic name
> are
> > > >>>> provided
> > > >>>> >> to
> > > >>>> >> >> >> > this API. Is the intention that callers should set one
> > but
> > > >>>> not the
> > > >>>> >> >> >> > other? Or both? This needs to be clarified. Also, if we
> > do
> > > >>>> want to
> > > >>>> >> >> >> > support lookups by UUID, that is another thing that
> needs
> > > to
> > > >>>> be
> > > >>>> >> added
> > > >>>> >> >> >> > to adminclient.
> > > >>>> >> >> >> >
> > > >>>> >> >> >> > In general, I feel like this should also probably be
> its
> > > own
> > > >>>> KIP
> > > >>>> >> since
> > > >>>> >> >> >> > it's fairly complex
> > > >>>> >> >> >> >
> > > >>>> >> >> >> > best,
> > > >>>> >> >> >> > Colin
> > > >>>> >> >> >> >
> > > >>>> >> >> >> >
> > > >>>> >> >> >> > On Thu, Aug 10, 2023, at 15:46, Calvin Liu 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
> > > >>>> >> >> >>
> > > >>>> >> >>
> > > >>>> >>
> > > >>>>
> > > >>>
> > >
> >
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Calvin Liu <ca...@confluent.io.INVALID>.
Hi Jun
Thanks for the comments.

40. If we change to None, it is not guaranteed for no data loss. For users
who are not able to validate the data with external resources, manual
intervention does not give a better result but a loss of availability. So
practically speaking, the Balance mode would be a better default value.

41. No, it represents how we want to do the unclean leader election. If it
is false, the unclean leader election will be the old random way.
Otherwise, the unclean recovery will be used.

42. Good catch. Updated.

43. Only the first 20 topics will be served. Others will be returned with
InvalidRequestError

44. The order matters. The desired leader entries match with the topic
partition list by the index.

45. Thanks! Updated.

46. Good advice! Updated.

47.1, updated the comment. Basically it will elect the replica in the
desiredLeader field to be the leader

47.2 We can let the admin client do the conversion. Using the desiredLeader
field in the json format seems easier for users.

48. Once the MV version is downgraded, all the ELR related fields will be
removed on the next partition change. The controller will also ignore the
ELR fields. Updated the KIP.

49. Yes, it would be deprecated/removed.


On Mon, Sep 25, 2023 at 3:49 PM Jun Rao <ju...@confluent.io.invalid> wrote:

> Hi, Calvin,
>
> Thanks for the updated KIP. Made another pass. A few more comments below.
>
> 40. unclean.leader.election.enable.false ->
> unclean.recovery.strategy.Balanced: The Balanced mode could still lead to
> data loss. So, I am wondering if unclean.leader.election.enable.false
> should map to None?
>
> 41. unclean.recovery.manager.enabled: I am not sure why we introduce this
> additional config. Is it the same as unclean.recovery.strategy=None?
>
> 42. DescribeTopicResponse.TopicAuthorizedOperations: Should this be at the
> topic level?
>
> 43. "Limit: 20 topics max per request": Could we describe what happens if
> the request includes more than 20 topics?
>
> 44. ElectLeadersRequest.DesiredLeaders: Could we describe whether the
> ordering matters?
>
> 45. GetReplicaLogInfo.TopicPartitions: "about": "The topic partitions to
> elect leaders.": The description in "about" is incorrect.
>
> 46. GetReplicaLogInfoResponse: Should we nest partitions under topicId to
> be consistent with other types of responses?
>
> 47. kafka-leader-election.sh:
> 47.1 Could we explain DESIGNATION?
> 47.2 desiredLeader: Should it be a list to match the field in
> ElectLeadersRequest?
>
> 48. We could add a section on downgrade?
>
> 49. LastKnownLeader: This seems only needed in the first phase of
> delivering ELR. Will it be removed when the complete KIP is delivered?
>
> Thanks,
>
> Jun
>
> On Tue, Sep 19, 2023 at 1:30 PM Colin McCabe <cm...@apache.org> wrote:
>
> > Hi Calvin,
> >
> > Thanks for the explanations. I like the idea of using none, balanced,
> > aggressive. We also had an offline discussion about why it is good to
> use a
> > new config key (basically, so that we can deprecate the old one which had
> > only false/true values in 4.0) With these changes, I am +1.
> >
> > best,
> > Colin
> >
> > On Mon, Sep 18, 2023, at 15:54, Calvin Liu wrote:
> > > Hi Colin,
> > > Also, can we deprecate unclean.leader.election.enable in 4.0? Before
> > that,
> > > we can have both the config unclean.recovery.strategy and
> > > unclean.leader.election.enable
> > > and using the unclean.recovery.Enabled to determine which config to use
> > > during the unclean leader election.
> > >
> > > On Mon, Sep 18, 2023 at 3:51 PM Calvin Liu <ca...@confluent.io> wrote:
> > >
> > >> Hi Colin,
> > >> For the unclean.recovery.strategy config name, how about we use the
> > >> following
> > >> None. It basically means no unclean recovery will be performed.
> > >> Aggressive. It means availability goes first. Whenever the partition
> > can't
> > >> elect a durable replica, the controller will try the unclean recovery.
> > >> Balanced. It is the balance point of the availability
> first(Aggressive)
> > >> and least availability(None). The controller performs unclean recovery
> > when
> > >> both ISR and ELR are empty.
> > >>
> > >>
> > >> On Fri, Sep 15, 2023 at 11:42 AM Calvin Liu <ca...@confluent.io>
> wrote:
> > >>
> > >>> Hi Colin,
> > >>>
> > >>> > So, the proposal is that if someone sets
> > "unclean.leader.election.enable
> > >>> = true"...
> > >>>
> > >>>
> > >>> The idea is to use one of the unclean.leader.election.enable and
> > >>> unclean.recovery.strategy based on the unclean.recovery.Enabled. A
> > possible
> > >>> version can be
> > >>>
> > >>> If unclean.recovery.Enabled:
> > >>>
> > >>> {
> > >>>
> > >>> Check unclean.recovery.strategy. If set, use it. Otherwise, check
> > >>> unclean.leader.election.enable and translate it to
> > >>> unclean.recovery.strategy.
> > >>>
> > >>> } else {
> > >>>
> > >>> Use unclean.leader.election.enable
> > >>>
> > >>> }
> > >>>
> > >>>
> > >>> —--------
> > >>>
> > >>> >The configuration key should be "unclean.recovery.manager.enabled",
> > >>> right?
> > >>>
> > >>>
> > >>> I think we have two ways of choosing a leader uncleanly, unclean
> leader
> > >>> election and unclean recovery(log inspection) and we try to switch
> > between
> > >>> them.
> > >>>
> > >>> Do you mean we want to develop two ways of performing the unclean
> > >>> recovery and one of them is using “unclean recovery manager”? I guess
> > we
> > >>> haven’t discussed the second way.
> > >>>
> > >>>
> > >>> —-------
> > >>>
> > >>> >How do these 4 levels of overrides interact with your new
> > >>> configurations?
> > >>>
> > >>>
> > >>> I do notice in the Kraft controller code, the method to check whether
> > >>> perform unclean leader election is hard coded to false since
> > >>> 2021(uncleanLeaderElectionEnabledForTopic). Isn’t it a good chance to
> > >>> completely deprecate the unclean.leader.election.enable? We don’t
> even
> > have
> > >>> to worry about the config conversion.
> > >>>
> > >>> On the other hand, whatever the override is, as long as the
> controller
> > >>> can have the final effective unclean.leader.election.enable, the
> topic
> > >>> level config unclean.recovery.strategy, the cluster level config
> > >>> unclean.recovery.Enabled, the controller can calculate the correct
> > methods
> > >>> to use right?
> > >>>
> > >>>
> > >>> On Fri, Sep 15, 2023 at 10:02 AM Colin McCabe <cm...@apache.org>
> > wrote:
> > >>>
> > >>>> On Thu, Sep 14, 2023, at 22:23, Calvin Liu wrote:
> > >>>> > Hi Colin
> > >>>> > 1. I think using the new config name is more clear.
> > >>>> >        a. The unclean leader election is actually removed if
> unclean
> > >>>> > recovery is in use.
> > >>>> >        b. Using multiple values in unclean.leader.election.enable
> is
> > >>>> > confusing and it will be more confusing after people forget about
> > this
> > >>>> > discussion.
> > >>>>
> > >>>> Hi Calvin,
> > >>>>
> > >>>> So, the proposal is that if someone sets
> > "unclean.leader.election.enable
> > >>>> = true" but then sets one of your new configurations, the value of
> > >>>> unclean.leader.election.enable is ignored? That seems less clear to
> > me, not
> > >>>> more. Just in general, having multiple configuration keys to control
> > the
> > >>>> same thing confuses users. Basically, they are sitting at a giant
> > control
> > >>>> panel, and some of the levers do nothing.
> > >>>>
> > >>>> > 2. Sorry I forgot to mention in the response that I did add the
> > >>>> > unclean.recovery.Enabled flag.
> > >>>>
> > >>>> The configuration key should be "unclean.recovery.manager.enabled",
> > >>>> right? Becuase we can do "unclean recovery" without the manager.
> > Disabling
> > >>>> the manager just means we use a different mechanism for recovery.
> > >>>>
> > >>>> >        c. Maybe I underestimated the challenge of replacing the
> > >>>> config. Any
> > >>>> > implementation problems ahead?
> > >>>>
> > >>>> There are four levels of overrides for
> unclean.leader.election.enable.
> > >>>>
> > >>>> 1. static configuration for node.
> > >>>>     This goes in the configuration file, typically named
> > >>>> server.properties
> > >>>>
> > >>>> 2. dynamic configuration for node default
> > >>>>   ConfigResource(type=BROKER, name="")
> > >>>>
> > >>>> 3. dynamic configuration for node
> > >>>>   ConfigResource(type=BROKER, name=<controller id>)
> > >>>>
> > >>>> 4. dynamic configuration for topic
> > >>>>   ConfigResource(type=TOPIC, name=<topic-name>)
> > >>>>
> > >>>> How do these 4 levels of overrides interact with your new
> > >>>> configurations? If the new configurations dominate over the old
> ones,
> > it
> > >>>> seems like this will get a lot more confusing to implement (and also
> > to
> > >>>> use.)
> > >>>>
> > >>>> Again, I'd recommend just adding some new values to
> > >>>> unclean.leader.election.enable. It's simple and will prevent user
> > confusion
> > >>>> (as well as developer confusion.)
> > >>>>
> > >>>> best,
> > >>>> Colin
> > >>>>
> > >>>>
> > >>>> > 3. About the admin client, I mentioned 3 changes in the client.
> > >>>> Anything
> > >>>> > else I missed in the KIP?
> > >>>> >       a. The client will switch to using the new RPC instead of
> > >>>> > MetadataRequest for the topics.
> > >>>> >       b. The TopicPartitionInfo used in TopicDescription needs to
> > add
> > >>>> new
> > >>>> > fields related to the ELR.
> > >>>> >       c. The outputs will add the ELR related fields.
> > >>>> >
> > >>>> > On Thu, Sep 14, 2023 at 9:19 PM Colin McCabe <cm...@apache.org>
> > >>>> wrote:
> > >>>> >
> > >>>> >> Hi Calvin,
> > >>>> >>
> > >>>> >> Thanks for the changes.
> > >>>> >>
> > >>>> >> 1. Earlier I commented that creating "unclean.recovery.strategy "
> > is
> > >>>> not
> > >>>> >> necessary, and we can just reuse the existing
> > >>>> >> "unclean.leader.election.enable" configuration key. Let's discuss
> > >>>> that.
> > >>>> >>
> > >>>> >> 2.I also don't understand why you didn't add a configuration to
> > >>>> enable or
> > >>>> >> disable the Unclean Recovery Manager. This seems like a very
> simple
> > >>>> way to
> > >>>> >> handle the staging issue which we discussed. The URM can just be
> > >>>> turned off
> > >>>> >> until it is production ready. Let's discuss this.
> > >>>> >>
> > >>>> >> 3. You still need to describe the changes to AdminClient that are
> > >>>> needed
> > >>>> >> to use DescribeTopicRequest.
> > >>>> >>
> > >>>> >> Keep at it. It's looking better. :)
> > >>>> >>
> > >>>> >> best,
> > >>>> >> Colin
> > >>>> >>
> > >>>> >>
> > >>>> >> On Thu, Sep 14, 2023, at 11:03, Calvin Liu wrote:
> > >>>> >> > Hi Colin
> > >>>> >> > Thanks for the comments!
> > >>>> >> >
> > >>>> >> > I did the following changes
> > >>>> >> >
> > >>>> >> >    1.
> > >>>> >> >
> > >>>> >> >    Simplified the API spec section to only include the diff.
> > >>>> >> >    2.
> > >>>> >> >
> > >>>> >> >    Reordered the HWM requirement section.
> > >>>> >> >    3.
> > >>>> >> >
> > >>>> >> >    Removed the URM implementation details to keep the necessary
> > >>>> >> >    characteristics to perform the unclean recovery.
> > >>>> >> >    1.
> > >>>> >> >
> > >>>> >> >       When to perform the unclean recovery
> > >>>> >> >       2.
> > >>>> >> >
> > >>>> >> >       Under different config, how the unclean recovery finds
> the
> > >>>> leader.
> > >>>> >> >       3.
> > >>>> >> >
> > >>>> >> >       How the config unclean.leader.election.enable and
> > >>>> >> >       unclean.recovery.strategy are converted when users
> > >>>> enable/disable
> > >>>> >> the
> > >>>> >> >       unclean recovery.
> > >>>> >> >       4.
> > >>>> >> >
> > >>>> >> >    More details about how we change admin client.
> > >>>> >> >    5.
> > >>>> >> >
> > >>>> >> >    API limits on the GetReplicaLogInfoRequest and
> > >>>> DescribeTopicRequest.
> > >>>> >> >    6.
> > >>>> >> >
> > >>>> >> >    Two metrics added
> > >>>> >> >    1.
> > >>>> >> >
> > >>>> >> >       Kafka.controller.global_under_min_isr_partition_count
> > >>>> >> >       2.
> > >>>> >> >
> > >>>> >> >       kafka.controller.unclean_recovery_finished_count
> > >>>> >> >
> > >>>> >> >
> > >>>> >> > On Wed, Sep 13, 2023 at 10:46 AM Colin McCabe <
> > cmccabe@apache.org>
> > >>>> >> wrote:
> > >>>> >> >
> > >>>> >> >> On Tue, Sep 12, 2023, at 17:21, Calvin Liu wrote:
> > >>>> >> >> > Hi Colin
> > >>>> >> >> > Thanks for the comments!
> > >>>> >> >> >
> > >>>> >> >>
> > >>>> >> >> Hi Calvin,
> > >>>> >> >>
> > >>>> >> >> Thanks again for the KIP.
> > >>>> >> >>
> > >>>> >> >> One meta-comment: it's usually better to just do a diff on a
> > >>>> message
> > >>>> >> spec
> > >>>> >> >> file or java file if you're including changes to it in the
> KIP.
> > >>>> This is
> > >>>> >> >> easier to read than looking for "new fields begin" etc. in the
> > >>>> text, and
> > >>>> >> >> gracefully handles the case where existing fields were
> changed.
> > >>>> >> >>
> > >>>> >> >> > Rewrite the Additional High Watermark advancement
> requirement
> > >>>> >> >> > There was feedback on this section that some readers may not
> > be
> > >>>> >> familiar
> > >>>> >> >> > with HWM and Ack=0,1,all requests. This can help them
> > understand
> > >>>> the
> > >>>> >> >> > proposal. I will rewrite this part for more readability.
> > >>>> >> >> >
> > >>>> >> >>
> > >>>> >> >> To be clear, I wasn't suggesting dropping either section. I
> > agree
> > >>>> that
> > >>>> >> >> they add useful background. I was just suggesting that we
> should
> > >>>> discuss
> > >>>> >> >> the "acks" setting AFTER discussing the new high watermark
> > >>>> advancement
> > >>>> >> >> conditions. We also should discuss acks=0. While it isn't
> > >>>> conceptually
> > >>>> >> much
> > >>>> >> >> different than acks=1 here, its omission from this section is
> > >>>> confusing.
> > >>>> >> >>
> > >>>> >> >> > Unclean recovery
> > >>>> >> >> >
> > >>>> >> >> > The plan is to replace the unclean.leader.election.enable
> with
> > >>>> >> >> > unclean.recovery.strategy. If the Unclean Recovery is
> enabled
> > >>>> then it
> > >>>> >> >> deals
> > >>>> >> >> > with the three options in the unclean.recovery.strategy.
> > >>>> >> >> >
> > >>>> >> >> >
> > >>>> >> >> > Let’s refine the Unclean Recovery. We have already taken a
> > lot of
> > >>>> >> >> > suggestions and I hope to enhance the durability of Kafka to
> > the
> > >>>> next
> > >>>> >> >> level
> > >>>> >> >> > with this KIP.
> > >>>> >> >>
> > >>>> >> >> I am OK with doing the unclean leader recovery improvements in
> > >>>> this KIP.
> > >>>> >> >> However, I think we need to really work on the configuration
> > >>>> settings.
> > >>>> >> >>
> > >>>> >> >> Configuration overrides are often quite messy. For example,
> the
> > >>>> cases
> > >>>> >> >> where we have log.roll.hours and log.roll.segment.ms, the
> user
> > >>>> has to
> > >>>> >> >> remember which one takes precedence, and it is not obvious.
> So,
> > >>>> rather
> > >>>> >> than
> > >>>> >> >> creating a new configuration, why not add additional values to
> > >>>> >> >> "unclean.leader.election.enable"? I think this will be simpler
> > for
> > >>>> >> people
> > >>>> >> >> to understand, and simpler in the code as well.
> > >>>> >> >>
> > >>>> >> >> What if we continued to use "unclean.leader.election.enable"
> but
> > >>>> >> extended
> > >>>> >> >> it so that it took a string? Then the string could have these
> > >>>> values:
> > >>>> >> >>
> > >>>> >> >> never
> > >>>> >> >>     never automatically do an unclean leader election under
> any
> > >>>> >> conditions
> > >>>> >> >>
> > >>>> >> >> false / default
> > >>>> >> >>     only do an unclean leader election if there may be
> possible
> > >>>> data
> > >>>> >> loss
> > >>>> >> >>
> > >>>> >> >> true / always
> > >>>> >> >>     always do an unclean leader election if we can't
> immediately
> > >>>> elect a
> > >>>> >> >> leader
> > >>>> >> >>
> > >>>> >> >> It's a bit awkward that false maps to default rather than to
> > >>>> never. But
> > >>>> >> >> this awkwardness exists if we use two different configuration
> > keys
> > >>>> as
> > >>>> >> well.
> > >>>> >> >> The reason for the awkwardness is that we simply don't want
> most
> > >>>> of the
> > >>>> >> >> people currently setting unclean.leader.election.enable=false
> to
> > >>>> get the
> > >>>> >> >> "never" behavior. We have to bite that bullet. Better to be
> > clear
> > >>>> and
> > >>>> >> >> explicit than hide it.
> > >>>> >> >>
> > >>>> >> >> Another thing that's a bit awkward is having two different
> ways
> > to
> > >>>> do
> > >>>> >> >> unclean leader election specified in the KIP. You descirbe two
> > >>>> methods:
> > >>>> >> the
> > >>>> >> >> simple "choose the last leader" method, and the "unclean
> > recovery
> > >>>> >> manager"
> > >>>> >> >> method. I understand why you did it this way -- "choose the
> last
> > >>>> >> leader" is
> > >>>> >> >> simple, and will help us deliver an implementation quickly,
> > while
> > >>>> the
> > >>>> >> URM
> > >>>> >> >> is preferable in the long term. My suggestion here is to
> > separate
> > >>>> the
> > >>>> >> >> decision of HOW to do unclean leader election from the
> decision
> > of
> > >>>> WHEN
> > >>>> >> to
> > >>>> >> >> do it.
> > >>>> >> >>
> > >>>> >> >> So in other words, have "unclean.leader.election.enable"
> specify
> > >>>> when we
> > >>>> >> >> do unclean leader election, and have a new configuration like
> > >>>> >> >> "unclean.recovery.manager.enable" to determine if we use the
> > URM.
> > >>>> >> >> Presumably the URM will take some time to get fully stable, so
> > >>>> this can
> > >>>> >> >> default to false for a while, and we can flip the default to
> > true
> > >>>> when
> > >>>> >> we
> > >>>> >> >> feel ready.
> > >>>> >> >>
> > >>>> >> >> The URM is somewhat under-described here. I think we need a
> few
> > >>>> >> >> configurations here for it. For example, we need a
> > configuration to
> > >>>> >> specify
> > >>>> >> >> how long it should wait for a broker to respond to its RPCs
> > before
> > >>>> >> moving
> > >>>> >> >> on. We also need to understand how the URM interacts with
> > >>>> >> >> unclean.leader.election.enable=always. I assume that with
> > "always"
> > >>>> we
> > >>>> >> will
> > >>>> >> >> just unconditionally use the URM rather than choosing
> randomly.
> > >>>> But this
> > >>>> >> >> should be spelled out in the KIP.
> > >>>> >> >>
> > >>>> >> >> >
> > >>>> >> >> > DescribeTopicRequest
> > >>>> >> >> >
> > >>>> >> >> >    1.
> > >>>> >> >> >    Yes, the plan is to replace the MetadataRequest with the
> > >>>> >> >> >    DescribeTopicRequest for the admin clients. Will check
> the
> > >>>> details.
> > >>>> >> >>
> > >>>> >> >> Sounds good. But as I said, you need to specify how
> AdminClient
> > >>>> >> interacts
> > >>>> >> >> with the new request. This will involve adding some fields to
> > >>>> >> >> TopicDescription.java. And you need to specify the changes to
> > the
> > >>>> >> >> kafka-topics.sh command line tool. Otherwise we cannot use the
> > >>>> tool to
> > >>>> >> see
> > >>>> >> >> the new information.
> > >>>> >> >>
> > >>>> >> >> The new requests, DescribeTopicRequest and
> > >>>> GetReplicaLogInfoRequest,
> > >>>> >> need
> > >>>> >> >> to have limits placed on them so that their size can't be
> > >>>> infinite. We
> > >>>> >> >> don't want to propagate the current problems of
> MetadataRequest,
> > >>>> where
> > >>>> >> >> clients can request massive responses that can mess up the JVM
> > when
> > >>>> >> handled.
> > >>>> >> >>
> > >>>> >> >> Adding limits is simple for GetReplicaLogInfoRequest -- we can
> > >>>> just say
> > >>>> >> >> that only 2000 partitions at a time can be requested. For
> > >>>> >> >> DescribeTopicRequest we can probably just limit to 20 topics
> or
> > >>>> >> something
> > >>>> >> >> like that, to avoid the complexity of doing pagination in this
> > KIP.
> > >>>> >> >>
> > >>>> >> >> >    2.
> > >>>> >> >> >    I can let the broker load the ELR info so that they can
> > serve
> > >>>> the
> > >>>> >> >> >    DescribeTopicRequest as well.
> > >>>> >> >>
> > >>>> >> >> Yes, it's fine to add to MetadataCache. In fact, you'll be
> > loading
> > >>>> it
> > >>>> >> >> anyway once it's added to PartitionImage.
> > >>>> >> >>
> > >>>> >> >> >    3.
> > >>>> >> >> >    Yeah, it does not make sense to have the topic id if
> > >>>> >> >> >    DescribeTopicRequest is only used by the admin client.
> > >>>> >> >>
> > >>>> >> >> OK. That makes things simpler. We can always create a new API
> > later
> > >>>> >> >> (hopefully not in this KIP!) to query by topic ID.
> > >>>> >> >>
> > >>>> >> >> >
> > >>>> >> >> >
> > >>>> >> >> > Metrics
> > >>>> >> >> >
> > >>>> >> >> > As for overall cluster health metrics, I think under-min-ISR
> > is
> > >>>> still
> > >>>> >> a
> > >>>> >> >> > useful one. ELR is more like a safety belt. When the ELR is
> > >>>> used, the
> > >>>> >> >> > cluster availability has already been impacted.
> > >>>> >> >> >
> > >>>> >> >> > Maybe we can have a metric to count the partitions that
> > sum(ISR,
> > >>>> ELR)
> > >>>> >> <
> > >>>> >> >> min
> > >>>> >> >> > ISR. What do you think?
> > >>>> >> >>
> > >>>> >> >> How about:
> > >>>> >> >>
> > >>>> >> >> A.  a metric for the totoal number of under-min-isr
> partitions?
> > We
> > >>>> don't
> > >>>> >> >> have that in Apache Kafka at the moment.
> > >>>> >> >>
> > >>>> >> >> B. a metric for the number of unclean leader elections we did
> > (for
> > >>>> >> >> simplicity, it can reset to 0 on controller restart: we expect
> > >>>> people to
> > >>>> >> >> monitor the change over time anyway)
> > >>>> >> >>
> > >>>> >> >> best,
> > >>>> >> >> Colin
> > >>>> >> >>
> > >>>> >> >>
> > >>>> >> >> >
> > >>>> >> >> > Yeah, for the ongoing unclean recoveries, the controller can
> > >>>> keep an
> > >>>> >> >> > accurate count through failover because partition
> registration
> > >>>> can
> > >>>> >> >> indicate
> > >>>> >> >> > whether a recovery is needed. However, for the happened
> ones,
> > >>>> unless
> > >>>> >> we
> > >>>> >> >> > want to persist the number somewhere, we can only figure it
> > out
> > >>>> from
> > >>>> >> the
> > >>>> >> >> > log.
> > >>>> >> >> >
> > >>>> >> >> > On Tue, Sep 12, 2023 at 3:16 PM Colin McCabe <
> > cmccabe@apache.org
> > >>>> >
> > >>>> >> wrote:
> > >>>> >> >> >
> > >>>> >> >> >> Also, we should have metrics that show what is going on
> with
> > >>>> regard
> > >>>> >> to
> > >>>> >> >> the
> > >>>> >> >> >> eligible replica set. I'm not sure exactly what to suggest,
> > but
> > >>>> >> >> something
> > >>>> >> >> >> that could identify when things are going wrong in the
> > clsuter.
> > >>>> >> >> >>
> > >>>> >> >> >> For example, maybe a metric for partitions containing
> > replicas
> > >>>> that
> > >>>> >> are
> > >>>> >> >> >> ineligible to be leader? That would show a spike when a
> > broker
> > >>>> had an
> > >>>> >> >> >> unclean restart.
> > >>>> >> >> >>
> > >>>> >> >> >> Ideally, we'd also have a metric that indicates when an
> > unclear
> > >>>> >> leader
> > >>>> >> >> >> election or a recovery happened. It's a bit tricky because
> > the
> > >>>> simple
> > >>>> >> >> >> thing, of tracking it per controller, may be a bit
> confusing
> > >>>> during
> > >>>> >> >> >> failovers.
> > >>>> >> >> >>
> > >>>> >> >> >> best,
> > >>>> >> >> >> Colin
> > >>>> >> >> >>
> > >>>> >> >> >>
> > >>>> >> >> >> On Tue, Sep 12, 2023, at 14:25, Colin McCabe wrote:
> > >>>> >> >> >> > Hi Calvin,
> > >>>> >> >> >> >
> > >>>> >> >> >> > Thanks for the KIP. I think this is a great improvement.
> > >>>> >> >> >> >
> > >>>> >> >> >> >> Additional High Watermark advance requirement
> > >>>> >> >> >> >
> > >>>> >> >> >> > Typo: change "advance" to "advancement"
> > >>>> >> >> >> >
> > >>>> >> >> >> >> A bit recap of some key concepts.
> > >>>> >> >> >> >
> > >>>> >> >> >> > Typo: change "bit" to "quick"
> > >>>> >> >> >> >
> > >>>> >> >> >> >> Ack=1/all produce request. It defines when the Kafka
> > server
> > >>>> should
> > >>>> >> >> >> respond to the produce request
> > >>>> >> >> >> >
> > >>>> >> >> >> > I think this section would be clearer if we talked about
> > the
> > >>>> new
> > >>>> >> high
> > >>>> >> >> >> > watermark advancement requirement first, and THEN talked
> > >>>> about its
> > >>>> >> >> >> > impact on acks=0, acks=1, and     acks=all.  acks=all is
> of
> > >>>> course
> > >>>> >> the
> > >>>> >> >> >> > main case we care about here, so it would be good to lead
> > with
> > >>>> >> that,
> > >>>> >> >> >> > rather than delving into the technicalities of acks=0/1
> > first.
> > >>>> >> >> >> >
> > >>>> >> >> >> >> Unclean recovery
> > >>>> >> >> >> >
> > >>>> >> >> >> > So, here you are introducing a new configuration,
> > >>>> >> >> >> > unclean.recovery.strategy. The difficult thing here is
> that
> > >>>> there
> > >>>> >> is a
> > >>>> >> >> >> > lot of overlap with unclean.leader.election.enable. So we
> > >>>> have 3
> > >>>> >> >> >> > different settings for unclean.recovery.strategy, plus 2
> > >>>> different
> > >>>> >> >> >> > settings for unclean.leader.election.enable, giving a
> cross
> > >>>> >> product of
> > >>>> >> >> >> > 6 different options. The following "unclean recovery
> > manager"
> > >>>> >> section
> > >>>> >> >> >> > only applies to one fo those 6 different possibilities (I
> > >>>> think?)
> > >>>> >> >> >> >
> > >>>> >> >> >> > I simply don't think we need so many different election
> > types.
> > >>>> >> Really
> > >>>> >> >> >> > the use-cases we need are people who want NO unclean
> > >>>> elections,
> > >>>> >> people
> > >>>> >> >> >> > who want "the reasonable thing" and people who want
> > >>>> avaialbility at
> > >>>> >> >> all
> > >>>> >> >> >> > costs.
> > >>>> >> >> >> >
> > >>>> >> >> >> > Overall, I feel like the first half of the KIP is about
> the
> > >>>> ELR,
> > >>>> >> and
> > >>>> >> >> >> > the second half is about reworking unclean leader
> > election. It
> > >>>> >> might
> > >>>> >> >> be
> > >>>> >> >> >> > better to move that second half to a separate KIP so that
> > we
> > >>>> can
> > >>>> >> >> figure
> > >>>> >> >> >> > it out fully. It should be fine to punt this until later
> > and
> > >>>> just
> > >>>> >> have
> > >>>> >> >> >> > the current behavior on empty ELR be waiting for the last
> > >>>> known
> > >>>> >> leader
> > >>>> >> >> >> > to return. After all, that's what we do today.
> > >>>> >> >> >> >
> > >>>> >> >> >> >> DescribeTopicRequest
> > >>>> >> >> >> >
> > >>>> >> >> >> > Is the intention for AdminClient to use this RPC for
> > >>>> >> >> >> > Admin#describeTopics ? If so, we need to describe all of
> > the
> > >>>> >> changes
> > >>>> >> >> to
> > >>>> >> >> >> > the admin client API, as well as changes to command-line
> > >>>> tools like
> > >>>> >> >> >> > kafka-topics.sh (if there are any). For example, you will
> > >>>> probably
> > >>>> >> >> need
> > >>>> >> >> >> > changes to TopicDescription.java. You will also need to
> > >>>> provide
> > >>>> >> all of
> > >>>> >> >> >> > the things that admin client needs -- for example,
> > >>>> >> >> >> > TopicAuthorizedOperations.
> > >>>> >> >> >> >
> > >>>> >> >> >> > I also don't think the controller should serve this
> > request.
> > >>>> We
> > >>>> >> want
> > >>>> >> >> to
> > >>>> >> >> >> > minimize load on the controller. Just like with the other
> > >>>> metadata
> > >>>> >> >> >> > requests like MetadataRequest, this should be served by
> > >>>> brokers.
> > >>>> >> >> >> >
> > >>>> >> >> >> > It's a bit confusing why both topic ID and topic name are
> > >>>> provided
> > >>>> >> to
> > >>>> >> >> >> > this API. Is the intention that callers should set one
> but
> > >>>> not the
> > >>>> >> >> >> > other? Or both? This needs to be clarified. Also, if we
> do
> > >>>> want to
> > >>>> >> >> >> > support lookups by UUID, that is another thing that needs
> > to
> > >>>> be
> > >>>> >> added
> > >>>> >> >> >> > to adminclient.
> > >>>> >> >> >> >
> > >>>> >> >> >> > In general, I feel like this should also probably be its
> > own
> > >>>> KIP
> > >>>> >> since
> > >>>> >> >> >> > it's fairly complex
> > >>>> >> >> >> >
> > >>>> >> >> >> > best,
> > >>>> >> >> >> > Colin
> > >>>> >> >> >> >
> > >>>> >> >> >> >
> > >>>> >> >> >> > On Thu, Aug 10, 2023, at 15:46, Calvin Liu 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
> > >>>> >> >> >>
> > >>>> >> >>
> > >>>> >>
> > >>>>
> > >>>
> >
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

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

Thanks for the updated KIP. Made another pass. A few more comments below.

40. unclean.leader.election.enable.false ->
unclean.recovery.strategy.Balanced: The Balanced mode could still lead to
data loss. So, I am wondering if unclean.leader.election.enable.false
should map to None?

41. unclean.recovery.manager.enabled: I am not sure why we introduce this
additional config. Is it the same as unclean.recovery.strategy=None?

42. DescribeTopicResponse.TopicAuthorizedOperations: Should this be at the
topic level?

43. "Limit: 20 topics max per request": Could we describe what happens if
the request includes more than 20 topics?

44. ElectLeadersRequest.DesiredLeaders: Could we describe whether the
ordering matters?

45. GetReplicaLogInfo.TopicPartitions: "about": "The topic partitions to
elect leaders.": The description in "about" is incorrect.

46. GetReplicaLogInfoResponse: Should we nest partitions under topicId to
be consistent with other types of responses?

47. kafka-leader-election.sh:
47.1 Could we explain DESIGNATION?
47.2 desiredLeader: Should it be a list to match the field in
ElectLeadersRequest?

48. We could add a section on downgrade?

49. LastKnownLeader: This seems only needed in the first phase of
delivering ELR. Will it be removed when the complete KIP is delivered?

Thanks,

Jun

On Tue, Sep 19, 2023 at 1:30 PM Colin McCabe <cm...@apache.org> wrote:

> Hi Calvin,
>
> Thanks for the explanations. I like the idea of using none, balanced,
> aggressive. We also had an offline discussion about why it is good to use a
> new config key (basically, so that we can deprecate the old one which had
> only false/true values in 4.0) With these changes, I am +1.
>
> best,
> Colin
>
> On Mon, Sep 18, 2023, at 15:54, Calvin Liu wrote:
> > Hi Colin,
> > Also, can we deprecate unclean.leader.election.enable in 4.0? Before
> that,
> > we can have both the config unclean.recovery.strategy and
> > unclean.leader.election.enable
> > and using the unclean.recovery.Enabled to determine which config to use
> > during the unclean leader election.
> >
> > On Mon, Sep 18, 2023 at 3:51 PM Calvin Liu <ca...@confluent.io> wrote:
> >
> >> Hi Colin,
> >> For the unclean.recovery.strategy config name, how about we use the
> >> following
> >> None. It basically means no unclean recovery will be performed.
> >> Aggressive. It means availability goes first. Whenever the partition
> can't
> >> elect a durable replica, the controller will try the unclean recovery.
> >> Balanced. It is the balance point of the availability first(Aggressive)
> >> and least availability(None). The controller performs unclean recovery
> when
> >> both ISR and ELR are empty.
> >>
> >>
> >> On Fri, Sep 15, 2023 at 11:42 AM Calvin Liu <ca...@confluent.io> wrote:
> >>
> >>> Hi Colin,
> >>>
> >>> > So, the proposal is that if someone sets
> "unclean.leader.election.enable
> >>> = true"...
> >>>
> >>>
> >>> The idea is to use one of the unclean.leader.election.enable and
> >>> unclean.recovery.strategy based on the unclean.recovery.Enabled. A
> possible
> >>> version can be
> >>>
> >>> If unclean.recovery.Enabled:
> >>>
> >>> {
> >>>
> >>> Check unclean.recovery.strategy. If set, use it. Otherwise, check
> >>> unclean.leader.election.enable and translate it to
> >>> unclean.recovery.strategy.
> >>>
> >>> } else {
> >>>
> >>> Use unclean.leader.election.enable
> >>>
> >>> }
> >>>
> >>>
> >>> —--------
> >>>
> >>> >The configuration key should be "unclean.recovery.manager.enabled",
> >>> right?
> >>>
> >>>
> >>> I think we have two ways of choosing a leader uncleanly, unclean leader
> >>> election and unclean recovery(log inspection) and we try to switch
> between
> >>> them.
> >>>
> >>> Do you mean we want to develop two ways of performing the unclean
> >>> recovery and one of them is using “unclean recovery manager”? I guess
> we
> >>> haven’t discussed the second way.
> >>>
> >>>
> >>> —-------
> >>>
> >>> >How do these 4 levels of overrides interact with your new
> >>> configurations?
> >>>
> >>>
> >>> I do notice in the Kraft controller code, the method to check whether
> >>> perform unclean leader election is hard coded to false since
> >>> 2021(uncleanLeaderElectionEnabledForTopic). Isn’t it a good chance to
> >>> completely deprecate the unclean.leader.election.enable? We don’t even
> have
> >>> to worry about the config conversion.
> >>>
> >>> On the other hand, whatever the override is, as long as the controller
> >>> can have the final effective unclean.leader.election.enable, the topic
> >>> level config unclean.recovery.strategy, the cluster level config
> >>> unclean.recovery.Enabled, the controller can calculate the correct
> methods
> >>> to use right?
> >>>
> >>>
> >>> On Fri, Sep 15, 2023 at 10:02 AM Colin McCabe <cm...@apache.org>
> wrote:
> >>>
> >>>> On Thu, Sep 14, 2023, at 22:23, Calvin Liu wrote:
> >>>> > Hi Colin
> >>>> > 1. I think using the new config name is more clear.
> >>>> >        a. The unclean leader election is actually removed if unclean
> >>>> > recovery is in use.
> >>>> >        b. Using multiple values in unclean.leader.election.enable is
> >>>> > confusing and it will be more confusing after people forget about
> this
> >>>> > discussion.
> >>>>
> >>>> Hi Calvin,
> >>>>
> >>>> So, the proposal is that if someone sets
> "unclean.leader.election.enable
> >>>> = true" but then sets one of your new configurations, the value of
> >>>> unclean.leader.election.enable is ignored? That seems less clear to
> me, not
> >>>> more. Just in general, having multiple configuration keys to control
> the
> >>>> same thing confuses users. Basically, they are sitting at a giant
> control
> >>>> panel, and some of the levers do nothing.
> >>>>
> >>>> > 2. Sorry I forgot to mention in the response that I did add the
> >>>> > unclean.recovery.Enabled flag.
> >>>>
> >>>> The configuration key should be "unclean.recovery.manager.enabled",
> >>>> right? Becuase we can do "unclean recovery" without the manager.
> Disabling
> >>>> the manager just means we use a different mechanism for recovery.
> >>>>
> >>>> >        c. Maybe I underestimated the challenge of replacing the
> >>>> config. Any
> >>>> > implementation problems ahead?
> >>>>
> >>>> There are four levels of overrides for unclean.leader.election.enable.
> >>>>
> >>>> 1. static configuration for node.
> >>>>     This goes in the configuration file, typically named
> >>>> server.properties
> >>>>
> >>>> 2. dynamic configuration for node default
> >>>>   ConfigResource(type=BROKER, name="")
> >>>>
> >>>> 3. dynamic configuration for node
> >>>>   ConfigResource(type=BROKER, name=<controller id>)
> >>>>
> >>>> 4. dynamic configuration for topic
> >>>>   ConfigResource(type=TOPIC, name=<topic-name>)
> >>>>
> >>>> How do these 4 levels of overrides interact with your new
> >>>> configurations? If the new configurations dominate over the old ones,
> it
> >>>> seems like this will get a lot more confusing to implement (and also
> to
> >>>> use.)
> >>>>
> >>>> Again, I'd recommend just adding some new values to
> >>>> unclean.leader.election.enable. It's simple and will prevent user
> confusion
> >>>> (as well as developer confusion.)
> >>>>
> >>>> best,
> >>>> Colin
> >>>>
> >>>>
> >>>> > 3. About the admin client, I mentioned 3 changes in the client.
> >>>> Anything
> >>>> > else I missed in the KIP?
> >>>> >       a. The client will switch to using the new RPC instead of
> >>>> > MetadataRequest for the topics.
> >>>> >       b. The TopicPartitionInfo used in TopicDescription needs to
> add
> >>>> new
> >>>> > fields related to the ELR.
> >>>> >       c. The outputs will add the ELR related fields.
> >>>> >
> >>>> > On Thu, Sep 14, 2023 at 9:19 PM Colin McCabe <cm...@apache.org>
> >>>> wrote:
> >>>> >
> >>>> >> Hi Calvin,
> >>>> >>
> >>>> >> Thanks for the changes.
> >>>> >>
> >>>> >> 1. Earlier I commented that creating "unclean.recovery.strategy "
> is
> >>>> not
> >>>> >> necessary, and we can just reuse the existing
> >>>> >> "unclean.leader.election.enable" configuration key. Let's discuss
> >>>> that.
> >>>> >>
> >>>> >> 2.I also don't understand why you didn't add a configuration to
> >>>> enable or
> >>>> >> disable the Unclean Recovery Manager. This seems like a very simple
> >>>> way to
> >>>> >> handle the staging issue which we discussed. The URM can just be
> >>>> turned off
> >>>> >> until it is production ready. Let's discuss this.
> >>>> >>
> >>>> >> 3. You still need to describe the changes to AdminClient that are
> >>>> needed
> >>>> >> to use DescribeTopicRequest.
> >>>> >>
> >>>> >> Keep at it. It's looking better. :)
> >>>> >>
> >>>> >> best,
> >>>> >> Colin
> >>>> >>
> >>>> >>
> >>>> >> On Thu, Sep 14, 2023, at 11:03, Calvin Liu wrote:
> >>>> >> > Hi Colin
> >>>> >> > Thanks for the comments!
> >>>> >> >
> >>>> >> > I did the following changes
> >>>> >> >
> >>>> >> >    1.
> >>>> >> >
> >>>> >> >    Simplified the API spec section to only include the diff.
> >>>> >> >    2.
> >>>> >> >
> >>>> >> >    Reordered the HWM requirement section.
> >>>> >> >    3.
> >>>> >> >
> >>>> >> >    Removed the URM implementation details to keep the necessary
> >>>> >> >    characteristics to perform the unclean recovery.
> >>>> >> >    1.
> >>>> >> >
> >>>> >> >       When to perform the unclean recovery
> >>>> >> >       2.
> >>>> >> >
> >>>> >> >       Under different config, how the unclean recovery finds the
> >>>> leader.
> >>>> >> >       3.
> >>>> >> >
> >>>> >> >       How the config unclean.leader.election.enable and
> >>>> >> >       unclean.recovery.strategy are converted when users
> >>>> enable/disable
> >>>> >> the
> >>>> >> >       unclean recovery.
> >>>> >> >       4.
> >>>> >> >
> >>>> >> >    More details about how we change admin client.
> >>>> >> >    5.
> >>>> >> >
> >>>> >> >    API limits on the GetReplicaLogInfoRequest and
> >>>> DescribeTopicRequest.
> >>>> >> >    6.
> >>>> >> >
> >>>> >> >    Two metrics added
> >>>> >> >    1.
> >>>> >> >
> >>>> >> >       Kafka.controller.global_under_min_isr_partition_count
> >>>> >> >       2.
> >>>> >> >
> >>>> >> >       kafka.controller.unclean_recovery_finished_count
> >>>> >> >
> >>>> >> >
> >>>> >> > On Wed, Sep 13, 2023 at 10:46 AM Colin McCabe <
> cmccabe@apache.org>
> >>>> >> wrote:
> >>>> >> >
> >>>> >> >> On Tue, Sep 12, 2023, at 17:21, Calvin Liu wrote:
> >>>> >> >> > Hi Colin
> >>>> >> >> > Thanks for the comments!
> >>>> >> >> >
> >>>> >> >>
> >>>> >> >> Hi Calvin,
> >>>> >> >>
> >>>> >> >> Thanks again for the KIP.
> >>>> >> >>
> >>>> >> >> One meta-comment: it's usually better to just do a diff on a
> >>>> message
> >>>> >> spec
> >>>> >> >> file or java file if you're including changes to it in the KIP.
> >>>> This is
> >>>> >> >> easier to read than looking for "new fields begin" etc. in the
> >>>> text, and
> >>>> >> >> gracefully handles the case where existing fields were changed.
> >>>> >> >>
> >>>> >> >> > Rewrite the Additional High Watermark advancement requirement
> >>>> >> >> > There was feedback on this section that some readers may not
> be
> >>>> >> familiar
> >>>> >> >> > with HWM and Ack=0,1,all requests. This can help them
> understand
> >>>> the
> >>>> >> >> > proposal. I will rewrite this part for more readability.
> >>>> >> >> >
> >>>> >> >>
> >>>> >> >> To be clear, I wasn't suggesting dropping either section. I
> agree
> >>>> that
> >>>> >> >> they add useful background. I was just suggesting that we should
> >>>> discuss
> >>>> >> >> the "acks" setting AFTER discussing the new high watermark
> >>>> advancement
> >>>> >> >> conditions. We also should discuss acks=0. While it isn't
> >>>> conceptually
> >>>> >> much
> >>>> >> >> different than acks=1 here, its omission from this section is
> >>>> confusing.
> >>>> >> >>
> >>>> >> >> > Unclean recovery
> >>>> >> >> >
> >>>> >> >> > The plan is to replace the unclean.leader.election.enable with
> >>>> >> >> > unclean.recovery.strategy. If the Unclean Recovery is enabled
> >>>> then it
> >>>> >> >> deals
> >>>> >> >> > with the three options in the unclean.recovery.strategy.
> >>>> >> >> >
> >>>> >> >> >
> >>>> >> >> > Let’s refine the Unclean Recovery. We have already taken a
> lot of
> >>>> >> >> > suggestions and I hope to enhance the durability of Kafka to
> the
> >>>> next
> >>>> >> >> level
> >>>> >> >> > with this KIP.
> >>>> >> >>
> >>>> >> >> I am OK with doing the unclean leader recovery improvements in
> >>>> this KIP.
> >>>> >> >> However, I think we need to really work on the configuration
> >>>> settings.
> >>>> >> >>
> >>>> >> >> Configuration overrides are often quite messy. For example, the
> >>>> cases
> >>>> >> >> where we have log.roll.hours and log.roll.segment.ms, the user
> >>>> has to
> >>>> >> >> remember which one takes precedence, and it is not obvious. So,
> >>>> rather
> >>>> >> than
> >>>> >> >> creating a new configuration, why not add additional values to
> >>>> >> >> "unclean.leader.election.enable"? I think this will be simpler
> for
> >>>> >> people
> >>>> >> >> to understand, and simpler in the code as well.
> >>>> >> >>
> >>>> >> >> What if we continued to use "unclean.leader.election.enable" but
> >>>> >> extended
> >>>> >> >> it so that it took a string? Then the string could have these
> >>>> values:
> >>>> >> >>
> >>>> >> >> never
> >>>> >> >>     never automatically do an unclean leader election under any
> >>>> >> conditions
> >>>> >> >>
> >>>> >> >> false / default
> >>>> >> >>     only do an unclean leader election if there may be possible
> >>>> data
> >>>> >> loss
> >>>> >> >>
> >>>> >> >> true / always
> >>>> >> >>     always do an unclean leader election if we can't immediately
> >>>> elect a
> >>>> >> >> leader
> >>>> >> >>
> >>>> >> >> It's a bit awkward that false maps to default rather than to
> >>>> never. But
> >>>> >> >> this awkwardness exists if we use two different configuration
> keys
> >>>> as
> >>>> >> well.
> >>>> >> >> The reason for the awkwardness is that we simply don't want most
> >>>> of the
> >>>> >> >> people currently setting unclean.leader.election.enable=false to
> >>>> get the
> >>>> >> >> "never" behavior. We have to bite that bullet. Better to be
> clear
> >>>> and
> >>>> >> >> explicit than hide it.
> >>>> >> >>
> >>>> >> >> Another thing that's a bit awkward is having two different ways
> to
> >>>> do
> >>>> >> >> unclean leader election specified in the KIP. You descirbe two
> >>>> methods:
> >>>> >> the
> >>>> >> >> simple "choose the last leader" method, and the "unclean
> recovery
> >>>> >> manager"
> >>>> >> >> method. I understand why you did it this way -- "choose the last
> >>>> >> leader" is
> >>>> >> >> simple, and will help us deliver an implementation quickly,
> while
> >>>> the
> >>>> >> URM
> >>>> >> >> is preferable in the long term. My suggestion here is to
> separate
> >>>> the
> >>>> >> >> decision of HOW to do unclean leader election from the decision
> of
> >>>> WHEN
> >>>> >> to
> >>>> >> >> do it.
> >>>> >> >>
> >>>> >> >> So in other words, have "unclean.leader.election.enable" specify
> >>>> when we
> >>>> >> >> do unclean leader election, and have a new configuration like
> >>>> >> >> "unclean.recovery.manager.enable" to determine if we use the
> URM.
> >>>> >> >> Presumably the URM will take some time to get fully stable, so
> >>>> this can
> >>>> >> >> default to false for a while, and we can flip the default to
> true
> >>>> when
> >>>> >> we
> >>>> >> >> feel ready.
> >>>> >> >>
> >>>> >> >> The URM is somewhat under-described here. I think we need a few
> >>>> >> >> configurations here for it. For example, we need a
> configuration to
> >>>> >> specify
> >>>> >> >> how long it should wait for a broker to respond to its RPCs
> before
> >>>> >> moving
> >>>> >> >> on. We also need to understand how the URM interacts with
> >>>> >> >> unclean.leader.election.enable=always. I assume that with
> "always"
> >>>> we
> >>>> >> will
> >>>> >> >> just unconditionally use the URM rather than choosing randomly.
> >>>> But this
> >>>> >> >> should be spelled out in the KIP.
> >>>> >> >>
> >>>> >> >> >
> >>>> >> >> > DescribeTopicRequest
> >>>> >> >> >
> >>>> >> >> >    1.
> >>>> >> >> >    Yes, the plan is to replace the MetadataRequest with the
> >>>> >> >> >    DescribeTopicRequest for the admin clients. Will check the
> >>>> details.
> >>>> >> >>
> >>>> >> >> Sounds good. But as I said, you need to specify how AdminClient
> >>>> >> interacts
> >>>> >> >> with the new request. This will involve adding some fields to
> >>>> >> >> TopicDescription.java. And you need to specify the changes to
> the
> >>>> >> >> kafka-topics.sh command line tool. Otherwise we cannot use the
> >>>> tool to
> >>>> >> see
> >>>> >> >> the new information.
> >>>> >> >>
> >>>> >> >> The new requests, DescribeTopicRequest and
> >>>> GetReplicaLogInfoRequest,
> >>>> >> need
> >>>> >> >> to have limits placed on them so that their size can't be
> >>>> infinite. We
> >>>> >> >> don't want to propagate the current problems of MetadataRequest,
> >>>> where
> >>>> >> >> clients can request massive responses that can mess up the JVM
> when
> >>>> >> handled.
> >>>> >> >>
> >>>> >> >> Adding limits is simple for GetReplicaLogInfoRequest -- we can
> >>>> just say
> >>>> >> >> that only 2000 partitions at a time can be requested. For
> >>>> >> >> DescribeTopicRequest we can probably just limit to 20 topics or
> >>>> >> something
> >>>> >> >> like that, to avoid the complexity of doing pagination in this
> KIP.
> >>>> >> >>
> >>>> >> >> >    2.
> >>>> >> >> >    I can let the broker load the ELR info so that they can
> serve
> >>>> the
> >>>> >> >> >    DescribeTopicRequest as well.
> >>>> >> >>
> >>>> >> >> Yes, it's fine to add to MetadataCache. In fact, you'll be
> loading
> >>>> it
> >>>> >> >> anyway once it's added to PartitionImage.
> >>>> >> >>
> >>>> >> >> >    3.
> >>>> >> >> >    Yeah, it does not make sense to have the topic id if
> >>>> >> >> >    DescribeTopicRequest is only used by the admin client.
> >>>> >> >>
> >>>> >> >> OK. That makes things simpler. We can always create a new API
> later
> >>>> >> >> (hopefully not in this KIP!) to query by topic ID.
> >>>> >> >>
> >>>> >> >> >
> >>>> >> >> >
> >>>> >> >> > Metrics
> >>>> >> >> >
> >>>> >> >> > As for overall cluster health metrics, I think under-min-ISR
> is
> >>>> still
> >>>> >> a
> >>>> >> >> > useful one. ELR is more like a safety belt. When the ELR is
> >>>> used, the
> >>>> >> >> > cluster availability has already been impacted.
> >>>> >> >> >
> >>>> >> >> > Maybe we can have a metric to count the partitions that
> sum(ISR,
> >>>> ELR)
> >>>> >> <
> >>>> >> >> min
> >>>> >> >> > ISR. What do you think?
> >>>> >> >>
> >>>> >> >> How about:
> >>>> >> >>
> >>>> >> >> A.  a metric for the totoal number of under-min-isr partitions?
> We
> >>>> don't
> >>>> >> >> have that in Apache Kafka at the moment.
> >>>> >> >>
> >>>> >> >> B. a metric for the number of unclean leader elections we did
> (for
> >>>> >> >> simplicity, it can reset to 0 on controller restart: we expect
> >>>> people to
> >>>> >> >> monitor the change over time anyway)
> >>>> >> >>
> >>>> >> >> best,
> >>>> >> >> Colin
> >>>> >> >>
> >>>> >> >>
> >>>> >> >> >
> >>>> >> >> > Yeah, for the ongoing unclean recoveries, the controller can
> >>>> keep an
> >>>> >> >> > accurate count through failover because partition registration
> >>>> can
> >>>> >> >> indicate
> >>>> >> >> > whether a recovery is needed. However, for the happened ones,
> >>>> unless
> >>>> >> we
> >>>> >> >> > want to persist the number somewhere, we can only figure it
> out
> >>>> from
> >>>> >> the
> >>>> >> >> > log.
> >>>> >> >> >
> >>>> >> >> > On Tue, Sep 12, 2023 at 3:16 PM Colin McCabe <
> cmccabe@apache.org
> >>>> >
> >>>> >> wrote:
> >>>> >> >> >
> >>>> >> >> >> Also, we should have metrics that show what is going on with
> >>>> regard
> >>>> >> to
> >>>> >> >> the
> >>>> >> >> >> eligible replica set. I'm not sure exactly what to suggest,
> but
> >>>> >> >> something
> >>>> >> >> >> that could identify when things are going wrong in the
> clsuter.
> >>>> >> >> >>
> >>>> >> >> >> For example, maybe a metric for partitions containing
> replicas
> >>>> that
> >>>> >> are
> >>>> >> >> >> ineligible to be leader? That would show a spike when a
> broker
> >>>> had an
> >>>> >> >> >> unclean restart.
> >>>> >> >> >>
> >>>> >> >> >> Ideally, we'd also have a metric that indicates when an
> unclear
> >>>> >> leader
> >>>> >> >> >> election or a recovery happened. It's a bit tricky because
> the
> >>>> simple
> >>>> >> >> >> thing, of tracking it per controller, may be a bit confusing
> >>>> during
> >>>> >> >> >> failovers.
> >>>> >> >> >>
> >>>> >> >> >> best,
> >>>> >> >> >> Colin
> >>>> >> >> >>
> >>>> >> >> >>
> >>>> >> >> >> On Tue, Sep 12, 2023, at 14:25, Colin McCabe wrote:
> >>>> >> >> >> > Hi Calvin,
> >>>> >> >> >> >
> >>>> >> >> >> > Thanks for the KIP. I think this is a great improvement.
> >>>> >> >> >> >
> >>>> >> >> >> >> Additional High Watermark advance requirement
> >>>> >> >> >> >
> >>>> >> >> >> > Typo: change "advance" to "advancement"
> >>>> >> >> >> >
> >>>> >> >> >> >> A bit recap of some key concepts.
> >>>> >> >> >> >
> >>>> >> >> >> > Typo: change "bit" to "quick"
> >>>> >> >> >> >
> >>>> >> >> >> >> Ack=1/all produce request. It defines when the Kafka
> server
> >>>> should
> >>>> >> >> >> respond to the produce request
> >>>> >> >> >> >
> >>>> >> >> >> > I think this section would be clearer if we talked about
> the
> >>>> new
> >>>> >> high
> >>>> >> >> >> > watermark advancement requirement first, and THEN talked
> >>>> about its
> >>>> >> >> >> > impact on acks=0, acks=1, and     acks=all.  acks=all is of
> >>>> course
> >>>> >> the
> >>>> >> >> >> > main case we care about here, so it would be good to lead
> with
> >>>> >> that,
> >>>> >> >> >> > rather than delving into the technicalities of acks=0/1
> first.
> >>>> >> >> >> >
> >>>> >> >> >> >> Unclean recovery
> >>>> >> >> >> >
> >>>> >> >> >> > So, here you are introducing a new configuration,
> >>>> >> >> >> > unclean.recovery.strategy. The difficult thing here is that
> >>>> there
> >>>> >> is a
> >>>> >> >> >> > lot of overlap with unclean.leader.election.enable. So we
> >>>> have 3
> >>>> >> >> >> > different settings for unclean.recovery.strategy, plus 2
> >>>> different
> >>>> >> >> >> > settings for unclean.leader.election.enable, giving a cross
> >>>> >> product of
> >>>> >> >> >> > 6 different options. The following "unclean recovery
> manager"
> >>>> >> section
> >>>> >> >> >> > only applies to one fo those 6 different possibilities (I
> >>>> think?)
> >>>> >> >> >> >
> >>>> >> >> >> > I simply don't think we need so many different election
> types.
> >>>> >> Really
> >>>> >> >> >> > the use-cases we need are people who want NO unclean
> >>>> elections,
> >>>> >> people
> >>>> >> >> >> > who want "the reasonable thing" and people who want
> >>>> avaialbility at
> >>>> >> >> all
> >>>> >> >> >> > costs.
> >>>> >> >> >> >
> >>>> >> >> >> > Overall, I feel like the first half of the KIP is about the
> >>>> ELR,
> >>>> >> and
> >>>> >> >> >> > the second half is about reworking unclean leader
> election. It
> >>>> >> might
> >>>> >> >> be
> >>>> >> >> >> > better to move that second half to a separate KIP so that
> we
> >>>> can
> >>>> >> >> figure
> >>>> >> >> >> > it out fully. It should be fine to punt this until later
> and
> >>>> just
> >>>> >> have
> >>>> >> >> >> > the current behavior on empty ELR be waiting for the last
> >>>> known
> >>>> >> leader
> >>>> >> >> >> > to return. After all, that's what we do today.
> >>>> >> >> >> >
> >>>> >> >> >> >> DescribeTopicRequest
> >>>> >> >> >> >
> >>>> >> >> >> > Is the intention for AdminClient to use this RPC for
> >>>> >> >> >> > Admin#describeTopics ? If so, we need to describe all of
> the
> >>>> >> changes
> >>>> >> >> to
> >>>> >> >> >> > the admin client API, as well as changes to command-line
> >>>> tools like
> >>>> >> >> >> > kafka-topics.sh (if there are any). For example, you will
> >>>> probably
> >>>> >> >> need
> >>>> >> >> >> > changes to TopicDescription.java. You will also need to
> >>>> provide
> >>>> >> all of
> >>>> >> >> >> > the things that admin client needs -- for example,
> >>>> >> >> >> > TopicAuthorizedOperations.
> >>>> >> >> >> >
> >>>> >> >> >> > I also don't think the controller should serve this
> request.
> >>>> We
> >>>> >> want
> >>>> >> >> to
> >>>> >> >> >> > minimize load on the controller. Just like with the other
> >>>> metadata
> >>>> >> >> >> > requests like MetadataRequest, this should be served by
> >>>> brokers.
> >>>> >> >> >> >
> >>>> >> >> >> > It's a bit confusing why both topic ID and topic name are
> >>>> provided
> >>>> >> to
> >>>> >> >> >> > this API. Is the intention that callers should set one but
> >>>> not the
> >>>> >> >> >> > other? Or both? This needs to be clarified. Also, if we do
> >>>> want to
> >>>> >> >> >> > support lookups by UUID, that is another thing that needs
> to
> >>>> be
> >>>> >> added
> >>>> >> >> >> > to adminclient.
> >>>> >> >> >> >
> >>>> >> >> >> > In general, I feel like this should also probably be its
> own
> >>>> KIP
> >>>> >> since
> >>>> >> >> >> > it's fairly complex
> >>>> >> >> >> >
> >>>> >> >> >> > best,
> >>>> >> >> >> > Colin
> >>>> >> >> >> >
> >>>> >> >> >> >
> >>>> >> >> >> > On Thu, Aug 10, 2023, at 15:46, Calvin Liu 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
> >>>> >> >> >>
> >>>> >> >>
> >>>> >>
> >>>>
> >>>
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Colin McCabe <cm...@apache.org>.
Hi Calvin,

Thanks for the explanations. I like the idea of using none, balanced, aggressive. We also had an offline discussion about why it is good to use a new config key (basically, so that we can deprecate the old one which had only false/true values in 4.0) With these changes, I am +1.

best,
Colin

On Mon, Sep 18, 2023, at 15:54, Calvin Liu wrote:
> Hi Colin,
> Also, can we deprecate unclean.leader.election.enable in 4.0? Before that,
> we can have both the config unclean.recovery.strategy and
> unclean.leader.election.enable
> and using the unclean.recovery.Enabled to determine which config to use
> during the unclean leader election.
>
> On Mon, Sep 18, 2023 at 3:51 PM Calvin Liu <ca...@confluent.io> wrote:
>
>> Hi Colin,
>> For the unclean.recovery.strategy config name, how about we use the
>> following
>> None. It basically means no unclean recovery will be performed.
>> Aggressive. It means availability goes first. Whenever the partition can't
>> elect a durable replica, the controller will try the unclean recovery.
>> Balanced. It is the balance point of the availability first(Aggressive)
>> and least availability(None). The controller performs unclean recovery when
>> both ISR and ELR are empty.
>>
>>
>> On Fri, Sep 15, 2023 at 11:42 AM Calvin Liu <ca...@confluent.io> wrote:
>>
>>> Hi Colin,
>>>
>>> > So, the proposal is that if someone sets "unclean.leader.election.enable
>>> = true"...
>>>
>>>
>>> The idea is to use one of the unclean.leader.election.enable and
>>> unclean.recovery.strategy based on the unclean.recovery.Enabled. A possible
>>> version can be
>>>
>>> If unclean.recovery.Enabled:
>>>
>>> {
>>>
>>> Check unclean.recovery.strategy. If set, use it. Otherwise, check
>>> unclean.leader.election.enable and translate it to
>>> unclean.recovery.strategy.
>>>
>>> } else {
>>>
>>> Use unclean.leader.election.enable
>>>
>>> }
>>>
>>>
>>> —--------
>>>
>>> >The configuration key should be "unclean.recovery.manager.enabled",
>>> right?
>>>
>>>
>>> I think we have two ways of choosing a leader uncleanly, unclean leader
>>> election and unclean recovery(log inspection) and we try to switch between
>>> them.
>>>
>>> Do you mean we want to develop two ways of performing the unclean
>>> recovery and one of them is using “unclean recovery manager”? I guess we
>>> haven’t discussed the second way.
>>>
>>>
>>> —-------
>>>
>>> >How do these 4 levels of overrides interact with your new
>>> configurations?
>>>
>>>
>>> I do notice in the Kraft controller code, the method to check whether
>>> perform unclean leader election is hard coded to false since
>>> 2021(uncleanLeaderElectionEnabledForTopic). Isn’t it a good chance to
>>> completely deprecate the unclean.leader.election.enable? We don’t even have
>>> to worry about the config conversion.
>>>
>>> On the other hand, whatever the override is, as long as the controller
>>> can have the final effective unclean.leader.election.enable, the topic
>>> level config unclean.recovery.strategy, the cluster level config
>>> unclean.recovery.Enabled, the controller can calculate the correct methods
>>> to use right?
>>>
>>>
>>> On Fri, Sep 15, 2023 at 10:02 AM Colin McCabe <cm...@apache.org> wrote:
>>>
>>>> On Thu, Sep 14, 2023, at 22:23, Calvin Liu wrote:
>>>> > Hi Colin
>>>> > 1. I think using the new config name is more clear.
>>>> >        a. The unclean leader election is actually removed if unclean
>>>> > recovery is in use.
>>>> >        b. Using multiple values in unclean.leader.election.enable is
>>>> > confusing and it will be more confusing after people forget about this
>>>> > discussion.
>>>>
>>>> Hi Calvin,
>>>>
>>>> So, the proposal is that if someone sets "unclean.leader.election.enable
>>>> = true" but then sets one of your new configurations, the value of
>>>> unclean.leader.election.enable is ignored? That seems less clear to me, not
>>>> more. Just in general, having multiple configuration keys to control the
>>>> same thing confuses users. Basically, they are sitting at a giant control
>>>> panel, and some of the levers do nothing.
>>>>
>>>> > 2. Sorry I forgot to mention in the response that I did add the
>>>> > unclean.recovery.Enabled flag.
>>>>
>>>> The configuration key should be "unclean.recovery.manager.enabled",
>>>> right? Becuase we can do "unclean recovery" without the manager. Disabling
>>>> the manager just means we use a different mechanism for recovery.
>>>>
>>>> >        c. Maybe I underestimated the challenge of replacing the
>>>> config. Any
>>>> > implementation problems ahead?
>>>>
>>>> There are four levels of overrides for unclean.leader.election.enable.
>>>>
>>>> 1. static configuration for node.
>>>>     This goes in the configuration file, typically named
>>>> server.properties
>>>>
>>>> 2. dynamic configuration for node default
>>>>   ConfigResource(type=BROKER, name="")
>>>>
>>>> 3. dynamic configuration for node
>>>>   ConfigResource(type=BROKER, name=<controller id>)
>>>>
>>>> 4. dynamic configuration for topic
>>>>   ConfigResource(type=TOPIC, name=<topic-name>)
>>>>
>>>> How do these 4 levels of overrides interact with your new
>>>> configurations? If the new configurations dominate over the old ones, it
>>>> seems like this will get a lot more confusing to implement (and also to
>>>> use.)
>>>>
>>>> Again, I'd recommend just adding some new values to
>>>> unclean.leader.election.enable. It's simple and will prevent user confusion
>>>> (as well as developer confusion.)
>>>>
>>>> best,
>>>> Colin
>>>>
>>>>
>>>> > 3. About the admin client, I mentioned 3 changes in the client.
>>>> Anything
>>>> > else I missed in the KIP?
>>>> >       a. The client will switch to using the new RPC instead of
>>>> > MetadataRequest for the topics.
>>>> >       b. The TopicPartitionInfo used in TopicDescription needs to add
>>>> new
>>>> > fields related to the ELR.
>>>> >       c. The outputs will add the ELR related fields.
>>>> >
>>>> > On Thu, Sep 14, 2023 at 9:19 PM Colin McCabe <cm...@apache.org>
>>>> wrote:
>>>> >
>>>> >> Hi Calvin,
>>>> >>
>>>> >> Thanks for the changes.
>>>> >>
>>>> >> 1. Earlier I commented that creating "unclean.recovery.strategy " is
>>>> not
>>>> >> necessary, and we can just reuse the existing
>>>> >> "unclean.leader.election.enable" configuration key. Let's discuss
>>>> that.
>>>> >>
>>>> >> 2.I also don't understand why you didn't add a configuration to
>>>> enable or
>>>> >> disable the Unclean Recovery Manager. This seems like a very simple
>>>> way to
>>>> >> handle the staging issue which we discussed. The URM can just be
>>>> turned off
>>>> >> until it is production ready. Let's discuss this.
>>>> >>
>>>> >> 3. You still need to describe the changes to AdminClient that are
>>>> needed
>>>> >> to use DescribeTopicRequest.
>>>> >>
>>>> >> Keep at it. It's looking better. :)
>>>> >>
>>>> >> best,
>>>> >> Colin
>>>> >>
>>>> >>
>>>> >> On Thu, Sep 14, 2023, at 11:03, Calvin Liu wrote:
>>>> >> > Hi Colin
>>>> >> > Thanks for the comments!
>>>> >> >
>>>> >> > I did the following changes
>>>> >> >
>>>> >> >    1.
>>>> >> >
>>>> >> >    Simplified the API spec section to only include the diff.
>>>> >> >    2.
>>>> >> >
>>>> >> >    Reordered the HWM requirement section.
>>>> >> >    3.
>>>> >> >
>>>> >> >    Removed the URM implementation details to keep the necessary
>>>> >> >    characteristics to perform the unclean recovery.
>>>> >> >    1.
>>>> >> >
>>>> >> >       When to perform the unclean recovery
>>>> >> >       2.
>>>> >> >
>>>> >> >       Under different config, how the unclean recovery finds the
>>>> leader.
>>>> >> >       3.
>>>> >> >
>>>> >> >       How the config unclean.leader.election.enable and
>>>> >> >       unclean.recovery.strategy are converted when users
>>>> enable/disable
>>>> >> the
>>>> >> >       unclean recovery.
>>>> >> >       4.
>>>> >> >
>>>> >> >    More details about how we change admin client.
>>>> >> >    5.
>>>> >> >
>>>> >> >    API limits on the GetReplicaLogInfoRequest and
>>>> DescribeTopicRequest.
>>>> >> >    6.
>>>> >> >
>>>> >> >    Two metrics added
>>>> >> >    1.
>>>> >> >
>>>> >> >       Kafka.controller.global_under_min_isr_partition_count
>>>> >> >       2.
>>>> >> >
>>>> >> >       kafka.controller.unclean_recovery_finished_count
>>>> >> >
>>>> >> >
>>>> >> > On Wed, Sep 13, 2023 at 10:46 AM Colin McCabe <cm...@apache.org>
>>>> >> wrote:
>>>> >> >
>>>> >> >> On Tue, Sep 12, 2023, at 17:21, Calvin Liu wrote:
>>>> >> >> > Hi Colin
>>>> >> >> > Thanks for the comments!
>>>> >> >> >
>>>> >> >>
>>>> >> >> Hi Calvin,
>>>> >> >>
>>>> >> >> Thanks again for the KIP.
>>>> >> >>
>>>> >> >> One meta-comment: it's usually better to just do a diff on a
>>>> message
>>>> >> spec
>>>> >> >> file or java file if you're including changes to it in the KIP.
>>>> This is
>>>> >> >> easier to read than looking for "new fields begin" etc. in the
>>>> text, and
>>>> >> >> gracefully handles the case where existing fields were changed.
>>>> >> >>
>>>> >> >> > Rewrite the Additional High Watermark advancement requirement
>>>> >> >> > There was feedback on this section that some readers may not be
>>>> >> familiar
>>>> >> >> > with HWM and Ack=0,1,all requests. This can help them understand
>>>> the
>>>> >> >> > proposal. I will rewrite this part for more readability.
>>>> >> >> >
>>>> >> >>
>>>> >> >> To be clear, I wasn't suggesting dropping either section. I agree
>>>> that
>>>> >> >> they add useful background. I was just suggesting that we should
>>>> discuss
>>>> >> >> the "acks" setting AFTER discussing the new high watermark
>>>> advancement
>>>> >> >> conditions. We also should discuss acks=0. While it isn't
>>>> conceptually
>>>> >> much
>>>> >> >> different than acks=1 here, its omission from this section is
>>>> confusing.
>>>> >> >>
>>>> >> >> > Unclean recovery
>>>> >> >> >
>>>> >> >> > The plan is to replace the unclean.leader.election.enable with
>>>> >> >> > unclean.recovery.strategy. If the Unclean Recovery is enabled
>>>> then it
>>>> >> >> deals
>>>> >> >> > with the three options in the unclean.recovery.strategy.
>>>> >> >> >
>>>> >> >> >
>>>> >> >> > Let’s refine the Unclean Recovery. We have already taken a lot of
>>>> >> >> > suggestions and I hope to enhance the durability of Kafka to the
>>>> next
>>>> >> >> level
>>>> >> >> > with this KIP.
>>>> >> >>
>>>> >> >> I am OK with doing the unclean leader recovery improvements in
>>>> this KIP.
>>>> >> >> However, I think we need to really work on the configuration
>>>> settings.
>>>> >> >>
>>>> >> >> Configuration overrides are often quite messy. For example, the
>>>> cases
>>>> >> >> where we have log.roll.hours and log.roll.segment.ms, the user
>>>> has to
>>>> >> >> remember which one takes precedence, and it is not obvious. So,
>>>> rather
>>>> >> than
>>>> >> >> creating a new configuration, why not add additional values to
>>>> >> >> "unclean.leader.election.enable"? I think this will be simpler for
>>>> >> people
>>>> >> >> to understand, and simpler in the code as well.
>>>> >> >>
>>>> >> >> What if we continued to use "unclean.leader.election.enable" but
>>>> >> extended
>>>> >> >> it so that it took a string? Then the string could have these
>>>> values:
>>>> >> >>
>>>> >> >> never
>>>> >> >>     never automatically do an unclean leader election under any
>>>> >> conditions
>>>> >> >>
>>>> >> >> false / default
>>>> >> >>     only do an unclean leader election if there may be possible
>>>> data
>>>> >> loss
>>>> >> >>
>>>> >> >> true / always
>>>> >> >>     always do an unclean leader election if we can't immediately
>>>> elect a
>>>> >> >> leader
>>>> >> >>
>>>> >> >> It's a bit awkward that false maps to default rather than to
>>>> never. But
>>>> >> >> this awkwardness exists if we use two different configuration keys
>>>> as
>>>> >> well.
>>>> >> >> The reason for the awkwardness is that we simply don't want most
>>>> of the
>>>> >> >> people currently setting unclean.leader.election.enable=false to
>>>> get the
>>>> >> >> "never" behavior. We have to bite that bullet. Better to be clear
>>>> and
>>>> >> >> explicit than hide it.
>>>> >> >>
>>>> >> >> Another thing that's a bit awkward is having two different ways to
>>>> do
>>>> >> >> unclean leader election specified in the KIP. You descirbe two
>>>> methods:
>>>> >> the
>>>> >> >> simple "choose the last leader" method, and the "unclean recovery
>>>> >> manager"
>>>> >> >> method. I understand why you did it this way -- "choose the last
>>>> >> leader" is
>>>> >> >> simple, and will help us deliver an implementation quickly, while
>>>> the
>>>> >> URM
>>>> >> >> is preferable in the long term. My suggestion here is to separate
>>>> the
>>>> >> >> decision of HOW to do unclean leader election from the decision of
>>>> WHEN
>>>> >> to
>>>> >> >> do it.
>>>> >> >>
>>>> >> >> So in other words, have "unclean.leader.election.enable" specify
>>>> when we
>>>> >> >> do unclean leader election, and have a new configuration like
>>>> >> >> "unclean.recovery.manager.enable" to determine if we use the URM.
>>>> >> >> Presumably the URM will take some time to get fully stable, so
>>>> this can
>>>> >> >> default to false for a while, and we can flip the default to true
>>>> when
>>>> >> we
>>>> >> >> feel ready.
>>>> >> >>
>>>> >> >> The URM is somewhat under-described here. I think we need a few
>>>> >> >> configurations here for it. For example, we need a configuration to
>>>> >> specify
>>>> >> >> how long it should wait for a broker to respond to its RPCs before
>>>> >> moving
>>>> >> >> on. We also need to understand how the URM interacts with
>>>> >> >> unclean.leader.election.enable=always. I assume that with "always"
>>>> we
>>>> >> will
>>>> >> >> just unconditionally use the URM rather than choosing randomly.
>>>> But this
>>>> >> >> should be spelled out in the KIP.
>>>> >> >>
>>>> >> >> >
>>>> >> >> > DescribeTopicRequest
>>>> >> >> >
>>>> >> >> >    1.
>>>> >> >> >    Yes, the plan is to replace the MetadataRequest with the
>>>> >> >> >    DescribeTopicRequest for the admin clients. Will check the
>>>> details.
>>>> >> >>
>>>> >> >> Sounds good. But as I said, you need to specify how AdminClient
>>>> >> interacts
>>>> >> >> with the new request. This will involve adding some fields to
>>>> >> >> TopicDescription.java. And you need to specify the changes to the
>>>> >> >> kafka-topics.sh command line tool. Otherwise we cannot use the
>>>> tool to
>>>> >> see
>>>> >> >> the new information.
>>>> >> >>
>>>> >> >> The new requests, DescribeTopicRequest and
>>>> GetReplicaLogInfoRequest,
>>>> >> need
>>>> >> >> to have limits placed on them so that their size can't be
>>>> infinite. We
>>>> >> >> don't want to propagate the current problems of MetadataRequest,
>>>> where
>>>> >> >> clients can request massive responses that can mess up the JVM when
>>>> >> handled.
>>>> >> >>
>>>> >> >> Adding limits is simple for GetReplicaLogInfoRequest -- we can
>>>> just say
>>>> >> >> that only 2000 partitions at a time can be requested. For
>>>> >> >> DescribeTopicRequest we can probably just limit to 20 topics or
>>>> >> something
>>>> >> >> like that, to avoid the complexity of doing pagination in this KIP.
>>>> >> >>
>>>> >> >> >    2.
>>>> >> >> >    I can let the broker load the ELR info so that they can serve
>>>> the
>>>> >> >> >    DescribeTopicRequest as well.
>>>> >> >>
>>>> >> >> Yes, it's fine to add to MetadataCache. In fact, you'll be loading
>>>> it
>>>> >> >> anyway once it's added to PartitionImage.
>>>> >> >>
>>>> >> >> >    3.
>>>> >> >> >    Yeah, it does not make sense to have the topic id if
>>>> >> >> >    DescribeTopicRequest is only used by the admin client.
>>>> >> >>
>>>> >> >> OK. That makes things simpler. We can always create a new API later
>>>> >> >> (hopefully not in this KIP!) to query by topic ID.
>>>> >> >>
>>>> >> >> >
>>>> >> >> >
>>>> >> >> > Metrics
>>>> >> >> >
>>>> >> >> > As for overall cluster health metrics, I think under-min-ISR is
>>>> still
>>>> >> a
>>>> >> >> > useful one. ELR is more like a safety belt. When the ELR is
>>>> used, the
>>>> >> >> > cluster availability has already been impacted.
>>>> >> >> >
>>>> >> >> > Maybe we can have a metric to count the partitions that sum(ISR,
>>>> ELR)
>>>> >> <
>>>> >> >> min
>>>> >> >> > ISR. What do you think?
>>>> >> >>
>>>> >> >> How about:
>>>> >> >>
>>>> >> >> A.  a metric for the totoal number of under-min-isr partitions? We
>>>> don't
>>>> >> >> have that in Apache Kafka at the moment.
>>>> >> >>
>>>> >> >> B. a metric for the number of unclean leader elections we did (for
>>>> >> >> simplicity, it can reset to 0 on controller restart: we expect
>>>> people to
>>>> >> >> monitor the change over time anyway)
>>>> >> >>
>>>> >> >> best,
>>>> >> >> Colin
>>>> >> >>
>>>> >> >>
>>>> >> >> >
>>>> >> >> > Yeah, for the ongoing unclean recoveries, the controller can
>>>> keep an
>>>> >> >> > accurate count through failover because partition registration
>>>> can
>>>> >> >> indicate
>>>> >> >> > whether a recovery is needed. However, for the happened ones,
>>>> unless
>>>> >> we
>>>> >> >> > want to persist the number somewhere, we can only figure it out
>>>> from
>>>> >> the
>>>> >> >> > log.
>>>> >> >> >
>>>> >> >> > On Tue, Sep 12, 2023 at 3:16 PM Colin McCabe <cmccabe@apache.org
>>>> >
>>>> >> wrote:
>>>> >> >> >
>>>> >> >> >> Also, we should have metrics that show what is going on with
>>>> regard
>>>> >> to
>>>> >> >> the
>>>> >> >> >> eligible replica set. I'm not sure exactly what to suggest, but
>>>> >> >> something
>>>> >> >> >> that could identify when things are going wrong in the clsuter.
>>>> >> >> >>
>>>> >> >> >> For example, maybe a metric for partitions containing replicas
>>>> that
>>>> >> are
>>>> >> >> >> ineligible to be leader? That would show a spike when a broker
>>>> had an
>>>> >> >> >> unclean restart.
>>>> >> >> >>
>>>> >> >> >> Ideally, we'd also have a metric that indicates when an unclear
>>>> >> leader
>>>> >> >> >> election or a recovery happened. It's a bit tricky because the
>>>> simple
>>>> >> >> >> thing, of tracking it per controller, may be a bit confusing
>>>> during
>>>> >> >> >> failovers.
>>>> >> >> >>
>>>> >> >> >> best,
>>>> >> >> >> Colin
>>>> >> >> >>
>>>> >> >> >>
>>>> >> >> >> On Tue, Sep 12, 2023, at 14:25, Colin McCabe wrote:
>>>> >> >> >> > Hi Calvin,
>>>> >> >> >> >
>>>> >> >> >> > Thanks for the KIP. I think this is a great improvement.
>>>> >> >> >> >
>>>> >> >> >> >> Additional High Watermark advance requirement
>>>> >> >> >> >
>>>> >> >> >> > Typo: change "advance" to "advancement"
>>>> >> >> >> >
>>>> >> >> >> >> A bit recap of some key concepts.
>>>> >> >> >> >
>>>> >> >> >> > Typo: change "bit" to "quick"
>>>> >> >> >> >
>>>> >> >> >> >> Ack=1/all produce request. It defines when the Kafka server
>>>> should
>>>> >> >> >> respond to the produce request
>>>> >> >> >> >
>>>> >> >> >> > I think this section would be clearer if we talked about the
>>>> new
>>>> >> high
>>>> >> >> >> > watermark advancement requirement first, and THEN talked
>>>> about its
>>>> >> >> >> > impact on acks=0, acks=1, and     acks=all.  acks=all is of
>>>> course
>>>> >> the
>>>> >> >> >> > main case we care about here, so it would be good to lead with
>>>> >> that,
>>>> >> >> >> > rather than delving into the technicalities of acks=0/1 first.
>>>> >> >> >> >
>>>> >> >> >> >> Unclean recovery
>>>> >> >> >> >
>>>> >> >> >> > So, here you are introducing a new configuration,
>>>> >> >> >> > unclean.recovery.strategy. The difficult thing here is that
>>>> there
>>>> >> is a
>>>> >> >> >> > lot of overlap with unclean.leader.election.enable. So we
>>>> have 3
>>>> >> >> >> > different settings for unclean.recovery.strategy, plus 2
>>>> different
>>>> >> >> >> > settings for unclean.leader.election.enable, giving a cross
>>>> >> product of
>>>> >> >> >> > 6 different options. The following "unclean recovery manager"
>>>> >> section
>>>> >> >> >> > only applies to one fo those 6 different possibilities (I
>>>> think?)
>>>> >> >> >> >
>>>> >> >> >> > I simply don't think we need so many different election types.
>>>> >> Really
>>>> >> >> >> > the use-cases we need are people who want NO unclean
>>>> elections,
>>>> >> people
>>>> >> >> >> > who want "the reasonable thing" and people who want
>>>> avaialbility at
>>>> >> >> all
>>>> >> >> >> > costs.
>>>> >> >> >> >
>>>> >> >> >> > Overall, I feel like the first half of the KIP is about the
>>>> ELR,
>>>> >> and
>>>> >> >> >> > the second half is about reworking unclean leader election. It
>>>> >> might
>>>> >> >> be
>>>> >> >> >> > better to move that second half to a separate KIP so that we
>>>> can
>>>> >> >> figure
>>>> >> >> >> > it out fully. It should be fine to punt this until later and
>>>> just
>>>> >> have
>>>> >> >> >> > the current behavior on empty ELR be waiting for the last
>>>> known
>>>> >> leader
>>>> >> >> >> > to return. After all, that's what we do today.
>>>> >> >> >> >
>>>> >> >> >> >> DescribeTopicRequest
>>>> >> >> >> >
>>>> >> >> >> > Is the intention for AdminClient to use this RPC for
>>>> >> >> >> > Admin#describeTopics ? If so, we need to describe all of the
>>>> >> changes
>>>> >> >> to
>>>> >> >> >> > the admin client API, as well as changes to command-line
>>>> tools like
>>>> >> >> >> > kafka-topics.sh (if there are any). For example, you will
>>>> probably
>>>> >> >> need
>>>> >> >> >> > changes to TopicDescription.java. You will also need to
>>>> provide
>>>> >> all of
>>>> >> >> >> > the things that admin client needs -- for example,
>>>> >> >> >> > TopicAuthorizedOperations.
>>>> >> >> >> >
>>>> >> >> >> > I also don't think the controller should serve this request.
>>>> We
>>>> >> want
>>>> >> >> to
>>>> >> >> >> > minimize load on the controller. Just like with the other
>>>> metadata
>>>> >> >> >> > requests like MetadataRequest, this should be served by
>>>> brokers.
>>>> >> >> >> >
>>>> >> >> >> > It's a bit confusing why both topic ID and topic name are
>>>> provided
>>>> >> to
>>>> >> >> >> > this API. Is the intention that callers should set one but
>>>> not the
>>>> >> >> >> > other? Or both? This needs to be clarified. Also, if we do
>>>> want to
>>>> >> >> >> > support lookups by UUID, that is another thing that needs to
>>>> be
>>>> >> added
>>>> >> >> >> > to adminclient.
>>>> >> >> >> >
>>>> >> >> >> > In general, I feel like this should also probably be its own
>>>> KIP
>>>> >> since
>>>> >> >> >> > it's fairly complex
>>>> >> >> >> >
>>>> >> >> >> > best,
>>>> >> >> >> > Colin
>>>> >> >> >> >
>>>> >> >> >> >
>>>> >> >> >> > On Thu, Aug 10, 2023, at 15:46, Calvin Liu 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
>>>> >> >> >>
>>>> >> >>
>>>> >>
>>>>
>>>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Calvin Liu <ca...@confluent.io.INVALID>.
Hi Colin,
Also, can we deprecate unclean.leader.election.enable in 4.0? Before that,
we can have both the config unclean.recovery.strategy and
unclean.leader.election.enable
and using the unclean.recovery.Enabled to determine which config to use
during the unclean leader election.

On Mon, Sep 18, 2023 at 3:51 PM Calvin Liu <ca...@confluent.io> wrote:

> Hi Colin,
> For the unclean.recovery.strategy config name, how about we use the
> following
> None. It basically means no unclean recovery will be performed.
> Aggressive. It means availability goes first. Whenever the partition can't
> elect a durable replica, the controller will try the unclean recovery.
> Balanced. It is the balance point of the availability first(Aggressive)
> and least availability(None). The controller performs unclean recovery when
> both ISR and ELR are empty.
>
>
> On Fri, Sep 15, 2023 at 11:42 AM Calvin Liu <ca...@confluent.io> wrote:
>
>> Hi Colin,
>>
>> > So, the proposal is that if someone sets "unclean.leader.election.enable
>> = true"...
>>
>>
>> The idea is to use one of the unclean.leader.election.enable and
>> unclean.recovery.strategy based on the unclean.recovery.Enabled. A possible
>> version can be
>>
>> If unclean.recovery.Enabled:
>>
>> {
>>
>> Check unclean.recovery.strategy. If set, use it. Otherwise, check
>> unclean.leader.election.enable and translate it to
>> unclean.recovery.strategy.
>>
>> } else {
>>
>> Use unclean.leader.election.enable
>>
>> }
>>
>>
>> —--------
>>
>> >The configuration key should be "unclean.recovery.manager.enabled",
>> right?
>>
>>
>> I think we have two ways of choosing a leader uncleanly, unclean leader
>> election and unclean recovery(log inspection) and we try to switch between
>> them.
>>
>> Do you mean we want to develop two ways of performing the unclean
>> recovery and one of them is using “unclean recovery manager”? I guess we
>> haven’t discussed the second way.
>>
>>
>> —-------
>>
>> >How do these 4 levels of overrides interact with your new
>> configurations?
>>
>>
>> I do notice in the Kraft controller code, the method to check whether
>> perform unclean leader election is hard coded to false since
>> 2021(uncleanLeaderElectionEnabledForTopic). Isn’t it a good chance to
>> completely deprecate the unclean.leader.election.enable? We don’t even have
>> to worry about the config conversion.
>>
>> On the other hand, whatever the override is, as long as the controller
>> can have the final effective unclean.leader.election.enable, the topic
>> level config unclean.recovery.strategy, the cluster level config
>> unclean.recovery.Enabled, the controller can calculate the correct methods
>> to use right?
>>
>>
>> On Fri, Sep 15, 2023 at 10:02 AM Colin McCabe <cm...@apache.org> wrote:
>>
>>> On Thu, Sep 14, 2023, at 22:23, Calvin Liu wrote:
>>> > Hi Colin
>>> > 1. I think using the new config name is more clear.
>>> >        a. The unclean leader election is actually removed if unclean
>>> > recovery is in use.
>>> >        b. Using multiple values in unclean.leader.election.enable is
>>> > confusing and it will be more confusing after people forget about this
>>> > discussion.
>>>
>>> Hi Calvin,
>>>
>>> So, the proposal is that if someone sets "unclean.leader.election.enable
>>> = true" but then sets one of your new configurations, the value of
>>> unclean.leader.election.enable is ignored? That seems less clear to me, not
>>> more. Just in general, having multiple configuration keys to control the
>>> same thing confuses users. Basically, they are sitting at a giant control
>>> panel, and some of the levers do nothing.
>>>
>>> > 2. Sorry I forgot to mention in the response that I did add the
>>> > unclean.recovery.Enabled flag.
>>>
>>> The configuration key should be "unclean.recovery.manager.enabled",
>>> right? Becuase we can do "unclean recovery" without the manager. Disabling
>>> the manager just means we use a different mechanism for recovery.
>>>
>>> >        c. Maybe I underestimated the challenge of replacing the
>>> config. Any
>>> > implementation problems ahead?
>>>
>>> There are four levels of overrides for unclean.leader.election.enable.
>>>
>>> 1. static configuration for node.
>>>     This goes in the configuration file, typically named
>>> server.properties
>>>
>>> 2. dynamic configuration for node default
>>>   ConfigResource(type=BROKER, name="")
>>>
>>> 3. dynamic configuration for node
>>>   ConfigResource(type=BROKER, name=<controller id>)
>>>
>>> 4. dynamic configuration for topic
>>>   ConfigResource(type=TOPIC, name=<topic-name>)
>>>
>>> How do these 4 levels of overrides interact with your new
>>> configurations? If the new configurations dominate over the old ones, it
>>> seems like this will get a lot more confusing to implement (and also to
>>> use.)
>>>
>>> Again, I'd recommend just adding some new values to
>>> unclean.leader.election.enable. It's simple and will prevent user confusion
>>> (as well as developer confusion.)
>>>
>>> best,
>>> Colin
>>>
>>>
>>> > 3. About the admin client, I mentioned 3 changes in the client.
>>> Anything
>>> > else I missed in the KIP?
>>> >       a. The client will switch to using the new RPC instead of
>>> > MetadataRequest for the topics.
>>> >       b. The TopicPartitionInfo used in TopicDescription needs to add
>>> new
>>> > fields related to the ELR.
>>> >       c. The outputs will add the ELR related fields.
>>> >
>>> > On Thu, Sep 14, 2023 at 9:19 PM Colin McCabe <cm...@apache.org>
>>> wrote:
>>> >
>>> >> Hi Calvin,
>>> >>
>>> >> Thanks for the changes.
>>> >>
>>> >> 1. Earlier I commented that creating "unclean.recovery.strategy " is
>>> not
>>> >> necessary, and we can just reuse the existing
>>> >> "unclean.leader.election.enable" configuration key. Let's discuss
>>> that.
>>> >>
>>> >> 2.I also don't understand why you didn't add a configuration to
>>> enable or
>>> >> disable the Unclean Recovery Manager. This seems like a very simple
>>> way to
>>> >> handle the staging issue which we discussed. The URM can just be
>>> turned off
>>> >> until it is production ready. Let's discuss this.
>>> >>
>>> >> 3. You still need to describe the changes to AdminClient that are
>>> needed
>>> >> to use DescribeTopicRequest.
>>> >>
>>> >> Keep at it. It's looking better. :)
>>> >>
>>> >> best,
>>> >> Colin
>>> >>
>>> >>
>>> >> On Thu, Sep 14, 2023, at 11:03, Calvin Liu wrote:
>>> >> > Hi Colin
>>> >> > Thanks for the comments!
>>> >> >
>>> >> > I did the following changes
>>> >> >
>>> >> >    1.
>>> >> >
>>> >> >    Simplified the API spec section to only include the diff.
>>> >> >    2.
>>> >> >
>>> >> >    Reordered the HWM requirement section.
>>> >> >    3.
>>> >> >
>>> >> >    Removed the URM implementation details to keep the necessary
>>> >> >    characteristics to perform the unclean recovery.
>>> >> >    1.
>>> >> >
>>> >> >       When to perform the unclean recovery
>>> >> >       2.
>>> >> >
>>> >> >       Under different config, how the unclean recovery finds the
>>> leader.
>>> >> >       3.
>>> >> >
>>> >> >       How the config unclean.leader.election.enable and
>>> >> >       unclean.recovery.strategy are converted when users
>>> enable/disable
>>> >> the
>>> >> >       unclean recovery.
>>> >> >       4.
>>> >> >
>>> >> >    More details about how we change admin client.
>>> >> >    5.
>>> >> >
>>> >> >    API limits on the GetReplicaLogInfoRequest and
>>> DescribeTopicRequest.
>>> >> >    6.
>>> >> >
>>> >> >    Two metrics added
>>> >> >    1.
>>> >> >
>>> >> >       Kafka.controller.global_under_min_isr_partition_count
>>> >> >       2.
>>> >> >
>>> >> >       kafka.controller.unclean_recovery_finished_count
>>> >> >
>>> >> >
>>> >> > On Wed, Sep 13, 2023 at 10:46 AM Colin McCabe <cm...@apache.org>
>>> >> wrote:
>>> >> >
>>> >> >> On Tue, Sep 12, 2023, at 17:21, Calvin Liu wrote:
>>> >> >> > Hi Colin
>>> >> >> > Thanks for the comments!
>>> >> >> >
>>> >> >>
>>> >> >> Hi Calvin,
>>> >> >>
>>> >> >> Thanks again for the KIP.
>>> >> >>
>>> >> >> One meta-comment: it's usually better to just do a diff on a
>>> message
>>> >> spec
>>> >> >> file or java file if you're including changes to it in the KIP.
>>> This is
>>> >> >> easier to read than looking for "new fields begin" etc. in the
>>> text, and
>>> >> >> gracefully handles the case where existing fields were changed.
>>> >> >>
>>> >> >> > Rewrite the Additional High Watermark advancement requirement
>>> >> >> > There was feedback on this section that some readers may not be
>>> >> familiar
>>> >> >> > with HWM and Ack=0,1,all requests. This can help them understand
>>> the
>>> >> >> > proposal. I will rewrite this part for more readability.
>>> >> >> >
>>> >> >>
>>> >> >> To be clear, I wasn't suggesting dropping either section. I agree
>>> that
>>> >> >> they add useful background. I was just suggesting that we should
>>> discuss
>>> >> >> the "acks" setting AFTER discussing the new high watermark
>>> advancement
>>> >> >> conditions. We also should discuss acks=0. While it isn't
>>> conceptually
>>> >> much
>>> >> >> different than acks=1 here, its omission from this section is
>>> confusing.
>>> >> >>
>>> >> >> > Unclean recovery
>>> >> >> >
>>> >> >> > The plan is to replace the unclean.leader.election.enable with
>>> >> >> > unclean.recovery.strategy. If the Unclean Recovery is enabled
>>> then it
>>> >> >> deals
>>> >> >> > with the three options in the unclean.recovery.strategy.
>>> >> >> >
>>> >> >> >
>>> >> >> > Let’s refine the Unclean Recovery. We have already taken a lot of
>>> >> >> > suggestions and I hope to enhance the durability of Kafka to the
>>> next
>>> >> >> level
>>> >> >> > with this KIP.
>>> >> >>
>>> >> >> I am OK with doing the unclean leader recovery improvements in
>>> this KIP.
>>> >> >> However, I think we need to really work on the configuration
>>> settings.
>>> >> >>
>>> >> >> Configuration overrides are often quite messy. For example, the
>>> cases
>>> >> >> where we have log.roll.hours and log.roll.segment.ms, the user
>>> has to
>>> >> >> remember which one takes precedence, and it is not obvious. So,
>>> rather
>>> >> than
>>> >> >> creating a new configuration, why not add additional values to
>>> >> >> "unclean.leader.election.enable"? I think this will be simpler for
>>> >> people
>>> >> >> to understand, and simpler in the code as well.
>>> >> >>
>>> >> >> What if we continued to use "unclean.leader.election.enable" but
>>> >> extended
>>> >> >> it so that it took a string? Then the string could have these
>>> values:
>>> >> >>
>>> >> >> never
>>> >> >>     never automatically do an unclean leader election under any
>>> >> conditions
>>> >> >>
>>> >> >> false / default
>>> >> >>     only do an unclean leader election if there may be possible
>>> data
>>> >> loss
>>> >> >>
>>> >> >> true / always
>>> >> >>     always do an unclean leader election if we can't immediately
>>> elect a
>>> >> >> leader
>>> >> >>
>>> >> >> It's a bit awkward that false maps to default rather than to
>>> never. But
>>> >> >> this awkwardness exists if we use two different configuration keys
>>> as
>>> >> well.
>>> >> >> The reason for the awkwardness is that we simply don't want most
>>> of the
>>> >> >> people currently setting unclean.leader.election.enable=false to
>>> get the
>>> >> >> "never" behavior. We have to bite that bullet. Better to be clear
>>> and
>>> >> >> explicit than hide it.
>>> >> >>
>>> >> >> Another thing that's a bit awkward is having two different ways to
>>> do
>>> >> >> unclean leader election specified in the KIP. You descirbe two
>>> methods:
>>> >> the
>>> >> >> simple "choose the last leader" method, and the "unclean recovery
>>> >> manager"
>>> >> >> method. I understand why you did it this way -- "choose the last
>>> >> leader" is
>>> >> >> simple, and will help us deliver an implementation quickly, while
>>> the
>>> >> URM
>>> >> >> is preferable in the long term. My suggestion here is to separate
>>> the
>>> >> >> decision of HOW to do unclean leader election from the decision of
>>> WHEN
>>> >> to
>>> >> >> do it.
>>> >> >>
>>> >> >> So in other words, have "unclean.leader.election.enable" specify
>>> when we
>>> >> >> do unclean leader election, and have a new configuration like
>>> >> >> "unclean.recovery.manager.enable" to determine if we use the URM.
>>> >> >> Presumably the URM will take some time to get fully stable, so
>>> this can
>>> >> >> default to false for a while, and we can flip the default to true
>>> when
>>> >> we
>>> >> >> feel ready.
>>> >> >>
>>> >> >> The URM is somewhat under-described here. I think we need a few
>>> >> >> configurations here for it. For example, we need a configuration to
>>> >> specify
>>> >> >> how long it should wait for a broker to respond to its RPCs before
>>> >> moving
>>> >> >> on. We also need to understand how the URM interacts with
>>> >> >> unclean.leader.election.enable=always. I assume that with "always"
>>> we
>>> >> will
>>> >> >> just unconditionally use the URM rather than choosing randomly.
>>> But this
>>> >> >> should be spelled out in the KIP.
>>> >> >>
>>> >> >> >
>>> >> >> > DescribeTopicRequest
>>> >> >> >
>>> >> >> >    1.
>>> >> >> >    Yes, the plan is to replace the MetadataRequest with the
>>> >> >> >    DescribeTopicRequest for the admin clients. Will check the
>>> details.
>>> >> >>
>>> >> >> Sounds good. But as I said, you need to specify how AdminClient
>>> >> interacts
>>> >> >> with the new request. This will involve adding some fields to
>>> >> >> TopicDescription.java. And you need to specify the changes to the
>>> >> >> kafka-topics.sh command line tool. Otherwise we cannot use the
>>> tool to
>>> >> see
>>> >> >> the new information.
>>> >> >>
>>> >> >> The new requests, DescribeTopicRequest and
>>> GetReplicaLogInfoRequest,
>>> >> need
>>> >> >> to have limits placed on them so that their size can't be
>>> infinite. We
>>> >> >> don't want to propagate the current problems of MetadataRequest,
>>> where
>>> >> >> clients can request massive responses that can mess up the JVM when
>>> >> handled.
>>> >> >>
>>> >> >> Adding limits is simple for GetReplicaLogInfoRequest -- we can
>>> just say
>>> >> >> that only 2000 partitions at a time can be requested. For
>>> >> >> DescribeTopicRequest we can probably just limit to 20 topics or
>>> >> something
>>> >> >> like that, to avoid the complexity of doing pagination in this KIP.
>>> >> >>
>>> >> >> >    2.
>>> >> >> >    I can let the broker load the ELR info so that they can serve
>>> the
>>> >> >> >    DescribeTopicRequest as well.
>>> >> >>
>>> >> >> Yes, it's fine to add to MetadataCache. In fact, you'll be loading
>>> it
>>> >> >> anyway once it's added to PartitionImage.
>>> >> >>
>>> >> >> >    3.
>>> >> >> >    Yeah, it does not make sense to have the topic id if
>>> >> >> >    DescribeTopicRequest is only used by the admin client.
>>> >> >>
>>> >> >> OK. That makes things simpler. We can always create a new API later
>>> >> >> (hopefully not in this KIP!) to query by topic ID.
>>> >> >>
>>> >> >> >
>>> >> >> >
>>> >> >> > Metrics
>>> >> >> >
>>> >> >> > As for overall cluster health metrics, I think under-min-ISR is
>>> still
>>> >> a
>>> >> >> > useful one. ELR is more like a safety belt. When the ELR is
>>> used, the
>>> >> >> > cluster availability has already been impacted.
>>> >> >> >
>>> >> >> > Maybe we can have a metric to count the partitions that sum(ISR,
>>> ELR)
>>> >> <
>>> >> >> min
>>> >> >> > ISR. What do you think?
>>> >> >>
>>> >> >> How about:
>>> >> >>
>>> >> >> A.  a metric for the totoal number of under-min-isr partitions? We
>>> don't
>>> >> >> have that in Apache Kafka at the moment.
>>> >> >>
>>> >> >> B. a metric for the number of unclean leader elections we did (for
>>> >> >> simplicity, it can reset to 0 on controller restart: we expect
>>> people to
>>> >> >> monitor the change over time anyway)
>>> >> >>
>>> >> >> best,
>>> >> >> Colin
>>> >> >>
>>> >> >>
>>> >> >> >
>>> >> >> > Yeah, for the ongoing unclean recoveries, the controller can
>>> keep an
>>> >> >> > accurate count through failover because partition registration
>>> can
>>> >> >> indicate
>>> >> >> > whether a recovery is needed. However, for the happened ones,
>>> unless
>>> >> we
>>> >> >> > want to persist the number somewhere, we can only figure it out
>>> from
>>> >> the
>>> >> >> > log.
>>> >> >> >
>>> >> >> > On Tue, Sep 12, 2023 at 3:16 PM Colin McCabe <cmccabe@apache.org
>>> >
>>> >> wrote:
>>> >> >> >
>>> >> >> >> Also, we should have metrics that show what is going on with
>>> regard
>>> >> to
>>> >> >> the
>>> >> >> >> eligible replica set. I'm not sure exactly what to suggest, but
>>> >> >> something
>>> >> >> >> that could identify when things are going wrong in the clsuter.
>>> >> >> >>
>>> >> >> >> For example, maybe a metric for partitions containing replicas
>>> that
>>> >> are
>>> >> >> >> ineligible to be leader? That would show a spike when a broker
>>> had an
>>> >> >> >> unclean restart.
>>> >> >> >>
>>> >> >> >> Ideally, we'd also have a metric that indicates when an unclear
>>> >> leader
>>> >> >> >> election or a recovery happened. It's a bit tricky because the
>>> simple
>>> >> >> >> thing, of tracking it per controller, may be a bit confusing
>>> during
>>> >> >> >> failovers.
>>> >> >> >>
>>> >> >> >> best,
>>> >> >> >> Colin
>>> >> >> >>
>>> >> >> >>
>>> >> >> >> On Tue, Sep 12, 2023, at 14:25, Colin McCabe wrote:
>>> >> >> >> > Hi Calvin,
>>> >> >> >> >
>>> >> >> >> > Thanks for the KIP. I think this is a great improvement.
>>> >> >> >> >
>>> >> >> >> >> Additional High Watermark advance requirement
>>> >> >> >> >
>>> >> >> >> > Typo: change "advance" to "advancement"
>>> >> >> >> >
>>> >> >> >> >> A bit recap of some key concepts.
>>> >> >> >> >
>>> >> >> >> > Typo: change "bit" to "quick"
>>> >> >> >> >
>>> >> >> >> >> Ack=1/all produce request. It defines when the Kafka server
>>> should
>>> >> >> >> respond to the produce request
>>> >> >> >> >
>>> >> >> >> > I think this section would be clearer if we talked about the
>>> new
>>> >> high
>>> >> >> >> > watermark advancement requirement first, and THEN talked
>>> about its
>>> >> >> >> > impact on acks=0, acks=1, and     acks=all.  acks=all is of
>>> course
>>> >> the
>>> >> >> >> > main case we care about here, so it would be good to lead with
>>> >> that,
>>> >> >> >> > rather than delving into the technicalities of acks=0/1 first.
>>> >> >> >> >
>>> >> >> >> >> Unclean recovery
>>> >> >> >> >
>>> >> >> >> > So, here you are introducing a new configuration,
>>> >> >> >> > unclean.recovery.strategy. The difficult thing here is that
>>> there
>>> >> is a
>>> >> >> >> > lot of overlap with unclean.leader.election.enable. So we
>>> have 3
>>> >> >> >> > different settings for unclean.recovery.strategy, plus 2
>>> different
>>> >> >> >> > settings for unclean.leader.election.enable, giving a cross
>>> >> product of
>>> >> >> >> > 6 different options. The following "unclean recovery manager"
>>> >> section
>>> >> >> >> > only applies to one fo those 6 different possibilities (I
>>> think?)
>>> >> >> >> >
>>> >> >> >> > I simply don't think we need so many different election types.
>>> >> Really
>>> >> >> >> > the use-cases we need are people who want NO unclean
>>> elections,
>>> >> people
>>> >> >> >> > who want "the reasonable thing" and people who want
>>> avaialbility at
>>> >> >> all
>>> >> >> >> > costs.
>>> >> >> >> >
>>> >> >> >> > Overall, I feel like the first half of the KIP is about the
>>> ELR,
>>> >> and
>>> >> >> >> > the second half is about reworking unclean leader election. It
>>> >> might
>>> >> >> be
>>> >> >> >> > better to move that second half to a separate KIP so that we
>>> can
>>> >> >> figure
>>> >> >> >> > it out fully. It should be fine to punt this until later and
>>> just
>>> >> have
>>> >> >> >> > the current behavior on empty ELR be waiting for the last
>>> known
>>> >> leader
>>> >> >> >> > to return. After all, that's what we do today.
>>> >> >> >> >
>>> >> >> >> >> DescribeTopicRequest
>>> >> >> >> >
>>> >> >> >> > Is the intention for AdminClient to use this RPC for
>>> >> >> >> > Admin#describeTopics ? If so, we need to describe all of the
>>> >> changes
>>> >> >> to
>>> >> >> >> > the admin client API, as well as changes to command-line
>>> tools like
>>> >> >> >> > kafka-topics.sh (if there are any). For example, you will
>>> probably
>>> >> >> need
>>> >> >> >> > changes to TopicDescription.java. You will also need to
>>> provide
>>> >> all of
>>> >> >> >> > the things that admin client needs -- for example,
>>> >> >> >> > TopicAuthorizedOperations.
>>> >> >> >> >
>>> >> >> >> > I also don't think the controller should serve this request.
>>> We
>>> >> want
>>> >> >> to
>>> >> >> >> > minimize load on the controller. Just like with the other
>>> metadata
>>> >> >> >> > requests like MetadataRequest, this should be served by
>>> brokers.
>>> >> >> >> >
>>> >> >> >> > It's a bit confusing why both topic ID and topic name are
>>> provided
>>> >> to
>>> >> >> >> > this API. Is the intention that callers should set one but
>>> not the
>>> >> >> >> > other? Or both? This needs to be clarified. Also, if we do
>>> want to
>>> >> >> >> > support lookups by UUID, that is another thing that needs to
>>> be
>>> >> added
>>> >> >> >> > to adminclient.
>>> >> >> >> >
>>> >> >> >> > In general, I feel like this should also probably be its own
>>> KIP
>>> >> since
>>> >> >> >> > it's fairly complex
>>> >> >> >> >
>>> >> >> >> > best,
>>> >> >> >> > Colin
>>> >> >> >> >
>>> >> >> >> >
>>> >> >> >> > On Thu, Aug 10, 2023, at 15:46, Calvin Liu 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
>>> >> >> >>
>>> >> >>
>>> >>
>>>
>>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Calvin Liu <ca...@confluent.io.INVALID>.
Hi Colin,
For the unclean.recovery.strategy config name, how about we use the
following
None. It basically means no unclean recovery will be performed.
Aggressive. It means availability goes first. Whenever the partition can't
elect a durable replica, the controller will try the unclean recovery.
Balanced. It is the balance point of the availability first(Aggressive) and
least availability(None). The controller performs unclean recovery when
both ISR and ELR are empty.


On Fri, Sep 15, 2023 at 11:42 AM Calvin Liu <ca...@confluent.io> wrote:

> Hi Colin,
>
> > So, the proposal is that if someone sets "unclean.leader.election.enable
> = true"...
>
>
> The idea is to use one of the unclean.leader.election.enable and
> unclean.recovery.strategy based on the unclean.recovery.Enabled. A possible
> version can be
>
> If unclean.recovery.Enabled:
>
> {
>
> Check unclean.recovery.strategy. If set, use it. Otherwise, check
> unclean.leader.election.enable and translate it to
> unclean.recovery.strategy.
>
> } else {
>
> Use unclean.leader.election.enable
>
> }
>
>
> —--------
>
> >The configuration key should be "unclean.recovery.manager.enabled",
> right?
>
>
> I think we have two ways of choosing a leader uncleanly, unclean leader
> election and unclean recovery(log inspection) and we try to switch between
> them.
>
> Do you mean we want to develop two ways of performing the unclean recovery
> and one of them is using “unclean recovery manager”? I guess we haven’t
> discussed the second way.
>
>
> —-------
>
> >How do these 4 levels of overrides interact with your new configurations?
>
>
> I do notice in the Kraft controller code, the method to check whether
> perform unclean leader election is hard coded to false since
> 2021(uncleanLeaderElectionEnabledForTopic). Isn’t it a good chance to
> completely deprecate the unclean.leader.election.enable? We don’t even have
> to worry about the config conversion.
>
> On the other hand, whatever the override is, as long as the controller can
> have the final effective unclean.leader.election.enable, the topic level
> config unclean.recovery.strategy, the cluster level config
> unclean.recovery.Enabled, the controller can calculate the correct methods
> to use right?
>
>
> On Fri, Sep 15, 2023 at 10:02 AM Colin McCabe <cm...@apache.org> wrote:
>
>> On Thu, Sep 14, 2023, at 22:23, Calvin Liu wrote:
>> > Hi Colin
>> > 1. I think using the new config name is more clear.
>> >        a. The unclean leader election is actually removed if unclean
>> > recovery is in use.
>> >        b. Using multiple values in unclean.leader.election.enable is
>> > confusing and it will be more confusing after people forget about this
>> > discussion.
>>
>> Hi Calvin,
>>
>> So, the proposal is that if someone sets "unclean.leader.election.enable
>> = true" but then sets one of your new configurations, the value of
>> unclean.leader.election.enable is ignored? That seems less clear to me, not
>> more. Just in general, having multiple configuration keys to control the
>> same thing confuses users. Basically, they are sitting at a giant control
>> panel, and some of the levers do nothing.
>>
>> > 2. Sorry I forgot to mention in the response that I did add the
>> > unclean.recovery.Enabled flag.
>>
>> The configuration key should be "unclean.recovery.manager.enabled",
>> right? Becuase we can do "unclean recovery" without the manager. Disabling
>> the manager just means we use a different mechanism for recovery.
>>
>> >        c. Maybe I underestimated the challenge of replacing the config.
>> Any
>> > implementation problems ahead?
>>
>> There are four levels of overrides for unclean.leader.election.enable.
>>
>> 1. static configuration for node.
>>     This goes in the configuration file, typically named server.properties
>>
>> 2. dynamic configuration for node default
>>   ConfigResource(type=BROKER, name="")
>>
>> 3. dynamic configuration for node
>>   ConfigResource(type=BROKER, name=<controller id>)
>>
>> 4. dynamic configuration for topic
>>   ConfigResource(type=TOPIC, name=<topic-name>)
>>
>> How do these 4 levels of overrides interact with your new configurations?
>> If the new configurations dominate over the old ones, it seems like this
>> will get a lot more confusing to implement (and also to use.)
>>
>> Again, I'd recommend just adding some new values to
>> unclean.leader.election.enable. It's simple and will prevent user confusion
>> (as well as developer confusion.)
>>
>> best,
>> Colin
>>
>>
>> > 3. About the admin client, I mentioned 3 changes in the client. Anything
>> > else I missed in the KIP?
>> >       a. The client will switch to using the new RPC instead of
>> > MetadataRequest for the topics.
>> >       b. The TopicPartitionInfo used in TopicDescription needs to add
>> new
>> > fields related to the ELR.
>> >       c. The outputs will add the ELR related fields.
>> >
>> > On Thu, Sep 14, 2023 at 9:19 PM Colin McCabe <cm...@apache.org>
>> wrote:
>> >
>> >> Hi Calvin,
>> >>
>> >> Thanks for the changes.
>> >>
>> >> 1. Earlier I commented that creating "unclean.recovery.strategy " is
>> not
>> >> necessary, and we can just reuse the existing
>> >> "unclean.leader.election.enable" configuration key. Let's discuss that.
>> >>
>> >> 2.I also don't understand why you didn't add a configuration to enable
>> or
>> >> disable the Unclean Recovery Manager. This seems like a very simple
>> way to
>> >> handle the staging issue which we discussed. The URM can just be
>> turned off
>> >> until it is production ready. Let's discuss this.
>> >>
>> >> 3. You still need to describe the changes to AdminClient that are
>> needed
>> >> to use DescribeTopicRequest.
>> >>
>> >> Keep at it. It's looking better. :)
>> >>
>> >> best,
>> >> Colin
>> >>
>> >>
>> >> On Thu, Sep 14, 2023, at 11:03, Calvin Liu wrote:
>> >> > Hi Colin
>> >> > Thanks for the comments!
>> >> >
>> >> > I did the following changes
>> >> >
>> >> >    1.
>> >> >
>> >> >    Simplified the API spec section to only include the diff.
>> >> >    2.
>> >> >
>> >> >    Reordered the HWM requirement section.
>> >> >    3.
>> >> >
>> >> >    Removed the URM implementation details to keep the necessary
>> >> >    characteristics to perform the unclean recovery.
>> >> >    1.
>> >> >
>> >> >       When to perform the unclean recovery
>> >> >       2.
>> >> >
>> >> >       Under different config, how the unclean recovery finds the
>> leader.
>> >> >       3.
>> >> >
>> >> >       How the config unclean.leader.election.enable and
>> >> >       unclean.recovery.strategy are converted when users
>> enable/disable
>> >> the
>> >> >       unclean recovery.
>> >> >       4.
>> >> >
>> >> >    More details about how we change admin client.
>> >> >    5.
>> >> >
>> >> >    API limits on the GetReplicaLogInfoRequest and
>> DescribeTopicRequest.
>> >> >    6.
>> >> >
>> >> >    Two metrics added
>> >> >    1.
>> >> >
>> >> >       Kafka.controller.global_under_min_isr_partition_count
>> >> >       2.
>> >> >
>> >> >       kafka.controller.unclean_recovery_finished_count
>> >> >
>> >> >
>> >> > On Wed, Sep 13, 2023 at 10:46 AM Colin McCabe <cm...@apache.org>
>> >> wrote:
>> >> >
>> >> >> On Tue, Sep 12, 2023, at 17:21, Calvin Liu wrote:
>> >> >> > Hi Colin
>> >> >> > Thanks for the comments!
>> >> >> >
>> >> >>
>> >> >> Hi Calvin,
>> >> >>
>> >> >> Thanks again for the KIP.
>> >> >>
>> >> >> One meta-comment: it's usually better to just do a diff on a message
>> >> spec
>> >> >> file or java file if you're including changes to it in the KIP.
>> This is
>> >> >> easier to read than looking for "new fields begin" etc. in the
>> text, and
>> >> >> gracefully handles the case where existing fields were changed.
>> >> >>
>> >> >> > Rewrite the Additional High Watermark advancement requirement
>> >> >> > There was feedback on this section that some readers may not be
>> >> familiar
>> >> >> > with HWM and Ack=0,1,all requests. This can help them understand
>> the
>> >> >> > proposal. I will rewrite this part for more readability.
>> >> >> >
>> >> >>
>> >> >> To be clear, I wasn't suggesting dropping either section. I agree
>> that
>> >> >> they add useful background. I was just suggesting that we should
>> discuss
>> >> >> the "acks" setting AFTER discussing the new high watermark
>> advancement
>> >> >> conditions. We also should discuss acks=0. While it isn't
>> conceptually
>> >> much
>> >> >> different than acks=1 here, its omission from this section is
>> confusing.
>> >> >>
>> >> >> > Unclean recovery
>> >> >> >
>> >> >> > The plan is to replace the unclean.leader.election.enable with
>> >> >> > unclean.recovery.strategy. If the Unclean Recovery is enabled
>> then it
>> >> >> deals
>> >> >> > with the three options in the unclean.recovery.strategy.
>> >> >> >
>> >> >> >
>> >> >> > Let’s refine the Unclean Recovery. We have already taken a lot of
>> >> >> > suggestions and I hope to enhance the durability of Kafka to the
>> next
>> >> >> level
>> >> >> > with this KIP.
>> >> >>
>> >> >> I am OK with doing the unclean leader recovery improvements in this
>> KIP.
>> >> >> However, I think we need to really work on the configuration
>> settings.
>> >> >>
>> >> >> Configuration overrides are often quite messy. For example, the
>> cases
>> >> >> where we have log.roll.hours and log.roll.segment.ms, the user has
>> to
>> >> >> remember which one takes precedence, and it is not obvious. So,
>> rather
>> >> than
>> >> >> creating a new configuration, why not add additional values to
>> >> >> "unclean.leader.election.enable"? I think this will be simpler for
>> >> people
>> >> >> to understand, and simpler in the code as well.
>> >> >>
>> >> >> What if we continued to use "unclean.leader.election.enable" but
>> >> extended
>> >> >> it so that it took a string? Then the string could have these
>> values:
>> >> >>
>> >> >> never
>> >> >>     never automatically do an unclean leader election under any
>> >> conditions
>> >> >>
>> >> >> false / default
>> >> >>     only do an unclean leader election if there may be possible data
>> >> loss
>> >> >>
>> >> >> true / always
>> >> >>     always do an unclean leader election if we can't immediately
>> elect a
>> >> >> leader
>> >> >>
>> >> >> It's a bit awkward that false maps to default rather than to never.
>> But
>> >> >> this awkwardness exists if we use two different configuration keys
>> as
>> >> well.
>> >> >> The reason for the awkwardness is that we simply don't want most of
>> the
>> >> >> people currently setting unclean.leader.election.enable=false to
>> get the
>> >> >> "never" behavior. We have to bite that bullet. Better to be clear
>> and
>> >> >> explicit than hide it.
>> >> >>
>> >> >> Another thing that's a bit awkward is having two different ways to
>> do
>> >> >> unclean leader election specified in the KIP. You descirbe two
>> methods:
>> >> the
>> >> >> simple "choose the last leader" method, and the "unclean recovery
>> >> manager"
>> >> >> method. I understand why you did it this way -- "choose the last
>> >> leader" is
>> >> >> simple, and will help us deliver an implementation quickly, while
>> the
>> >> URM
>> >> >> is preferable in the long term. My suggestion here is to separate
>> the
>> >> >> decision of HOW to do unclean leader election from the decision of
>> WHEN
>> >> to
>> >> >> do it.
>> >> >>
>> >> >> So in other words, have "unclean.leader.election.enable" specify
>> when we
>> >> >> do unclean leader election, and have a new configuration like
>> >> >> "unclean.recovery.manager.enable" to determine if we use the URM.
>> >> >> Presumably the URM will take some time to get fully stable, so this
>> can
>> >> >> default to false for a while, and we can flip the default to true
>> when
>> >> we
>> >> >> feel ready.
>> >> >>
>> >> >> The URM is somewhat under-described here. I think we need a few
>> >> >> configurations here for it. For example, we need a configuration to
>> >> specify
>> >> >> how long it should wait for a broker to respond to its RPCs before
>> >> moving
>> >> >> on. We also need to understand how the URM interacts with
>> >> >> unclean.leader.election.enable=always. I assume that with "always"
>> we
>> >> will
>> >> >> just unconditionally use the URM rather than choosing randomly. But
>> this
>> >> >> should be spelled out in the KIP.
>> >> >>
>> >> >> >
>> >> >> > DescribeTopicRequest
>> >> >> >
>> >> >> >    1.
>> >> >> >    Yes, the plan is to replace the MetadataRequest with the
>> >> >> >    DescribeTopicRequest for the admin clients. Will check the
>> details.
>> >> >>
>> >> >> Sounds good. But as I said, you need to specify how AdminClient
>> >> interacts
>> >> >> with the new request. This will involve adding some fields to
>> >> >> TopicDescription.java. And you need to specify the changes to the
>> >> >> kafka-topics.sh command line tool. Otherwise we cannot use the tool
>> to
>> >> see
>> >> >> the new information.
>> >> >>
>> >> >> The new requests, DescribeTopicRequest and GetReplicaLogInfoRequest,
>> >> need
>> >> >> to have limits placed on them so that their size can't be infinite.
>> We
>> >> >> don't want to propagate the current problems of MetadataRequest,
>> where
>> >> >> clients can request massive responses that can mess up the JVM when
>> >> handled.
>> >> >>
>> >> >> Adding limits is simple for GetReplicaLogInfoRequest -- we can just
>> say
>> >> >> that only 2000 partitions at a time can be requested. For
>> >> >> DescribeTopicRequest we can probably just limit to 20 topics or
>> >> something
>> >> >> like that, to avoid the complexity of doing pagination in this KIP.
>> >> >>
>> >> >> >    2.
>> >> >> >    I can let the broker load the ELR info so that they can serve
>> the
>> >> >> >    DescribeTopicRequest as well.
>> >> >>
>> >> >> Yes, it's fine to add to MetadataCache. In fact, you'll be loading
>> it
>> >> >> anyway once it's added to PartitionImage.
>> >> >>
>> >> >> >    3.
>> >> >> >    Yeah, it does not make sense to have the topic id if
>> >> >> >    DescribeTopicRequest is only used by the admin client.
>> >> >>
>> >> >> OK. That makes things simpler. We can always create a new API later
>> >> >> (hopefully not in this KIP!) to query by topic ID.
>> >> >>
>> >> >> >
>> >> >> >
>> >> >> > Metrics
>> >> >> >
>> >> >> > As for overall cluster health metrics, I think under-min-ISR is
>> still
>> >> a
>> >> >> > useful one. ELR is more like a safety belt. When the ELR is used,
>> the
>> >> >> > cluster availability has already been impacted.
>> >> >> >
>> >> >> > Maybe we can have a metric to count the partitions that sum(ISR,
>> ELR)
>> >> <
>> >> >> min
>> >> >> > ISR. What do you think?
>> >> >>
>> >> >> How about:
>> >> >>
>> >> >> A.  a metric for the totoal number of under-min-isr partitions? We
>> don't
>> >> >> have that in Apache Kafka at the moment.
>> >> >>
>> >> >> B. a metric for the number of unclean leader elections we did (for
>> >> >> simplicity, it can reset to 0 on controller restart: we expect
>> people to
>> >> >> monitor the change over time anyway)
>> >> >>
>> >> >> best,
>> >> >> Colin
>> >> >>
>> >> >>
>> >> >> >
>> >> >> > Yeah, for the ongoing unclean recoveries, the controller can keep
>> an
>> >> >> > accurate count through failover because partition registration can
>> >> >> indicate
>> >> >> > whether a recovery is needed. However, for the happened ones,
>> unless
>> >> we
>> >> >> > want to persist the number somewhere, we can only figure it out
>> from
>> >> the
>> >> >> > log.
>> >> >> >
>> >> >> > On Tue, Sep 12, 2023 at 3:16 PM Colin McCabe <cm...@apache.org>
>> >> wrote:
>> >> >> >
>> >> >> >> Also, we should have metrics that show what is going on with
>> regard
>> >> to
>> >> >> the
>> >> >> >> eligible replica set. I'm not sure exactly what to suggest, but
>> >> >> something
>> >> >> >> that could identify when things are going wrong in the clsuter.
>> >> >> >>
>> >> >> >> For example, maybe a metric for partitions containing replicas
>> that
>> >> are
>> >> >> >> ineligible to be leader? That would show a spike when a broker
>> had an
>> >> >> >> unclean restart.
>> >> >> >>
>> >> >> >> Ideally, we'd also have a metric that indicates when an unclear
>> >> leader
>> >> >> >> election or a recovery happened. It's a bit tricky because the
>> simple
>> >> >> >> thing, of tracking it per controller, may be a bit confusing
>> during
>> >> >> >> failovers.
>> >> >> >>
>> >> >> >> best,
>> >> >> >> Colin
>> >> >> >>
>> >> >> >>
>> >> >> >> On Tue, Sep 12, 2023, at 14:25, Colin McCabe wrote:
>> >> >> >> > Hi Calvin,
>> >> >> >> >
>> >> >> >> > Thanks for the KIP. I think this is a great improvement.
>> >> >> >> >
>> >> >> >> >> Additional High Watermark advance requirement
>> >> >> >> >
>> >> >> >> > Typo: change "advance" to "advancement"
>> >> >> >> >
>> >> >> >> >> A bit recap of some key concepts.
>> >> >> >> >
>> >> >> >> > Typo: change "bit" to "quick"
>> >> >> >> >
>> >> >> >> >> Ack=1/all produce request. It defines when the Kafka server
>> should
>> >> >> >> respond to the produce request
>> >> >> >> >
>> >> >> >> > I think this section would be clearer if we talked about the
>> new
>> >> high
>> >> >> >> > watermark advancement requirement first, and THEN talked about
>> its
>> >> >> >> > impact on acks=0, acks=1, and     acks=all.  acks=all is of
>> course
>> >> the
>> >> >> >> > main case we care about here, so it would be good to lead with
>> >> that,
>> >> >> >> > rather than delving into the technicalities of acks=0/1 first.
>> >> >> >> >
>> >> >> >> >> Unclean recovery
>> >> >> >> >
>> >> >> >> > So, here you are introducing a new configuration,
>> >> >> >> > unclean.recovery.strategy. The difficult thing here is that
>> there
>> >> is a
>> >> >> >> > lot of overlap with unclean.leader.election.enable. So we have
>> 3
>> >> >> >> > different settings for unclean.recovery.strategy, plus 2
>> different
>> >> >> >> > settings for unclean.leader.election.enable, giving a cross
>> >> product of
>> >> >> >> > 6 different options. The following "unclean recovery manager"
>> >> section
>> >> >> >> > only applies to one fo those 6 different possibilities (I
>> think?)
>> >> >> >> >
>> >> >> >> > I simply don't think we need so many different election types.
>> >> Really
>> >> >> >> > the use-cases we need are people who want NO unclean elections,
>> >> people
>> >> >> >> > who want "the reasonable thing" and people who want
>> avaialbility at
>> >> >> all
>> >> >> >> > costs.
>> >> >> >> >
>> >> >> >> > Overall, I feel like the first half of the KIP is about the
>> ELR,
>> >> and
>> >> >> >> > the second half is about reworking unclean leader election. It
>> >> might
>> >> >> be
>> >> >> >> > better to move that second half to a separate KIP so that we
>> can
>> >> >> figure
>> >> >> >> > it out fully. It should be fine to punt this until later and
>> just
>> >> have
>> >> >> >> > the current behavior on empty ELR be waiting for the last known
>> >> leader
>> >> >> >> > to return. After all, that's what we do today.
>> >> >> >> >
>> >> >> >> >> DescribeTopicRequest
>> >> >> >> >
>> >> >> >> > Is the intention for AdminClient to use this RPC for
>> >> >> >> > Admin#describeTopics ? If so, we need to describe all of the
>> >> changes
>> >> >> to
>> >> >> >> > the admin client API, as well as changes to command-line tools
>> like
>> >> >> >> > kafka-topics.sh (if there are any). For example, you will
>> probably
>> >> >> need
>> >> >> >> > changes to TopicDescription.java. You will also need to provide
>> >> all of
>> >> >> >> > the things that admin client needs -- for example,
>> >> >> >> > TopicAuthorizedOperations.
>> >> >> >> >
>> >> >> >> > I also don't think the controller should serve this request. We
>> >> want
>> >> >> to
>> >> >> >> > minimize load on the controller. Just like with the other
>> metadata
>> >> >> >> > requests like MetadataRequest, this should be served by
>> brokers.
>> >> >> >> >
>> >> >> >> > It's a bit confusing why both topic ID and topic name are
>> provided
>> >> to
>> >> >> >> > this API. Is the intention that callers should set one but not
>> the
>> >> >> >> > other? Or both? This needs to be clarified. Also, if we do
>> want to
>> >> >> >> > support lookups by UUID, that is another thing that needs to be
>> >> added
>> >> >> >> > to adminclient.
>> >> >> >> >
>> >> >> >> > In general, I feel like this should also probably be its own
>> KIP
>> >> since
>> >> >> >> > it's fairly complex
>> >> >> >> >
>> >> >> >> > best,
>> >> >> >> > Colin
>> >> >> >> >
>> >> >> >> >
>> >> >> >> > On Thu, Aug 10, 2023, at 15:46, Calvin Liu 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
>> >> >> >>
>> >> >>
>> >>
>>
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Calvin Liu <ca...@confluent.io.INVALID>.
Hi Colin,

> So, the proposal is that if someone sets "unclean.leader.election.enable
= true"...


The idea is to use one of the unclean.leader.election.enable and
unclean.recovery.strategy based on the unclean.recovery.Enabled. A possible
version can be

If unclean.recovery.Enabled:

{

Check unclean.recovery.strategy. If set, use it. Otherwise, check
unclean.leader.election.enable and translate it to
unclean.recovery.strategy.

} else {

Use unclean.leader.election.enable

}


—--------

>The configuration key should be "unclean.recovery.manager.enabled", right?


I think we have two ways of choosing a leader uncleanly, unclean leader
election and unclean recovery(log inspection) and we try to switch between
them.

Do you mean we want to develop two ways of performing the unclean recovery
and one of them is using “unclean recovery manager”? I guess we haven’t
discussed the second way.


—-------

>How do these 4 levels of overrides interact with your new configurations?


I do notice in the Kraft controller code, the method to check whether
perform unclean leader election is hard coded to false since
2021(uncleanLeaderElectionEnabledForTopic). Isn’t it a good chance to
completely deprecate the unclean.leader.election.enable? We don’t even have
to worry about the config conversion.

On the other hand, whatever the override is, as long as the controller can
have the final effective unclean.leader.election.enable, the topic level
config unclean.recovery.strategy, the cluster level config
unclean.recovery.Enabled, the controller can calculate the correct methods
to use right?


On Fri, Sep 15, 2023 at 10:02 AM Colin McCabe <cm...@apache.org> wrote:

> On Thu, Sep 14, 2023, at 22:23, Calvin Liu wrote:
> > Hi Colin
> > 1. I think using the new config name is more clear.
> >        a. The unclean leader election is actually removed if unclean
> > recovery is in use.
> >        b. Using multiple values in unclean.leader.election.enable is
> > confusing and it will be more confusing after people forget about this
> > discussion.
>
> Hi Calvin,
>
> So, the proposal is that if someone sets "unclean.leader.election.enable =
> true" but then sets one of your new configurations, the value of
> unclean.leader.election.enable is ignored? That seems less clear to me, not
> more. Just in general, having multiple configuration keys to control the
> same thing confuses users. Basically, they are sitting at a giant control
> panel, and some of the levers do nothing.
>
> > 2. Sorry I forgot to mention in the response that I did add the
> > unclean.recovery.Enabled flag.
>
> The configuration key should be "unclean.recovery.manager.enabled", right?
> Becuase we can do "unclean recovery" without the manager. Disabling the
> manager just means we use a different mechanism for recovery.
>
> >        c. Maybe I underestimated the challenge of replacing the config.
> Any
> > implementation problems ahead?
>
> There are four levels of overrides for unclean.leader.election.enable.
>
> 1. static configuration for node.
>     This goes in the configuration file, typically named server.properties
>
> 2. dynamic configuration for node default
>   ConfigResource(type=BROKER, name="")
>
> 3. dynamic configuration for node
>   ConfigResource(type=BROKER, name=<controller id>)
>
> 4. dynamic configuration for topic
>   ConfigResource(type=TOPIC, name=<topic-name>)
>
> How do these 4 levels of overrides interact with your new configurations?
> If the new configurations dominate over the old ones, it seems like this
> will get a lot more confusing to implement (and also to use.)
>
> Again, I'd recommend just adding some new values to
> unclean.leader.election.enable. It's simple and will prevent user confusion
> (as well as developer confusion.)
>
> best,
> Colin
>
>
> > 3. About the admin client, I mentioned 3 changes in the client. Anything
> > else I missed in the KIP?
> >       a. The client will switch to using the new RPC instead of
> > MetadataRequest for the topics.
> >       b. The TopicPartitionInfo used in TopicDescription needs to add new
> > fields related to the ELR.
> >       c. The outputs will add the ELR related fields.
> >
> > On Thu, Sep 14, 2023 at 9:19 PM Colin McCabe <cm...@apache.org> wrote:
> >
> >> Hi Calvin,
> >>
> >> Thanks for the changes.
> >>
> >> 1. Earlier I commented that creating "unclean.recovery.strategy " is not
> >> necessary, and we can just reuse the existing
> >> "unclean.leader.election.enable" configuration key. Let's discuss that.
> >>
> >> 2.I also don't understand why you didn't add a configuration to enable
> or
> >> disable the Unclean Recovery Manager. This seems like a very simple way
> to
> >> handle the staging issue which we discussed. The URM can just be turned
> off
> >> until it is production ready. Let's discuss this.
> >>
> >> 3. You still need to describe the changes to AdminClient that are needed
> >> to use DescribeTopicRequest.
> >>
> >> Keep at it. It's looking better. :)
> >>
> >> best,
> >> Colin
> >>
> >>
> >> On Thu, Sep 14, 2023, at 11:03, Calvin Liu wrote:
> >> > Hi Colin
> >> > Thanks for the comments!
> >> >
> >> > I did the following changes
> >> >
> >> >    1.
> >> >
> >> >    Simplified the API spec section to only include the diff.
> >> >    2.
> >> >
> >> >    Reordered the HWM requirement section.
> >> >    3.
> >> >
> >> >    Removed the URM implementation details to keep the necessary
> >> >    characteristics to perform the unclean recovery.
> >> >    1.
> >> >
> >> >       When to perform the unclean recovery
> >> >       2.
> >> >
> >> >       Under different config, how the unclean recovery finds the
> leader.
> >> >       3.
> >> >
> >> >       How the config unclean.leader.election.enable and
> >> >       unclean.recovery.strategy are converted when users
> enable/disable
> >> the
> >> >       unclean recovery.
> >> >       4.
> >> >
> >> >    More details about how we change admin client.
> >> >    5.
> >> >
> >> >    API limits on the GetReplicaLogInfoRequest and
> DescribeTopicRequest.
> >> >    6.
> >> >
> >> >    Two metrics added
> >> >    1.
> >> >
> >> >       Kafka.controller.global_under_min_isr_partition_count
> >> >       2.
> >> >
> >> >       kafka.controller.unclean_recovery_finished_count
> >> >
> >> >
> >> > On Wed, Sep 13, 2023 at 10:46 AM Colin McCabe <cm...@apache.org>
> >> wrote:
> >> >
> >> >> On Tue, Sep 12, 2023, at 17:21, Calvin Liu wrote:
> >> >> > Hi Colin
> >> >> > Thanks for the comments!
> >> >> >
> >> >>
> >> >> Hi Calvin,
> >> >>
> >> >> Thanks again for the KIP.
> >> >>
> >> >> One meta-comment: it's usually better to just do a diff on a message
> >> spec
> >> >> file or java file if you're including changes to it in the KIP. This
> is
> >> >> easier to read than looking for "new fields begin" etc. in the text,
> and
> >> >> gracefully handles the case where existing fields were changed.
> >> >>
> >> >> > Rewrite the Additional High Watermark advancement requirement
> >> >> > There was feedback on this section that some readers may not be
> >> familiar
> >> >> > with HWM and Ack=0,1,all requests. This can help them understand
> the
> >> >> > proposal. I will rewrite this part for more readability.
> >> >> >
> >> >>
> >> >> To be clear, I wasn't suggesting dropping either section. I agree
> that
> >> >> they add useful background. I was just suggesting that we should
> discuss
> >> >> the "acks" setting AFTER discussing the new high watermark
> advancement
> >> >> conditions. We also should discuss acks=0. While it isn't
> conceptually
> >> much
> >> >> different than acks=1 here, its omission from this section is
> confusing.
> >> >>
> >> >> > Unclean recovery
> >> >> >
> >> >> > The plan is to replace the unclean.leader.election.enable with
> >> >> > unclean.recovery.strategy. If the Unclean Recovery is enabled then
> it
> >> >> deals
> >> >> > with the three options in the unclean.recovery.strategy.
> >> >> >
> >> >> >
> >> >> > Let’s refine the Unclean Recovery. We have already taken a lot of
> >> >> > suggestions and I hope to enhance the durability of Kafka to the
> next
> >> >> level
> >> >> > with this KIP.
> >> >>
> >> >> I am OK with doing the unclean leader recovery improvements in this
> KIP.
> >> >> However, I think we need to really work on the configuration
> settings.
> >> >>
> >> >> Configuration overrides are often quite messy. For example, the cases
> >> >> where we have log.roll.hours and log.roll.segment.ms, the user has
> to
> >> >> remember which one takes precedence, and it is not obvious. So,
> rather
> >> than
> >> >> creating a new configuration, why not add additional values to
> >> >> "unclean.leader.election.enable"? I think this will be simpler for
> >> people
> >> >> to understand, and simpler in the code as well.
> >> >>
> >> >> What if we continued to use "unclean.leader.election.enable" but
> >> extended
> >> >> it so that it took a string? Then the string could have these values:
> >> >>
> >> >> never
> >> >>     never automatically do an unclean leader election under any
> >> conditions
> >> >>
> >> >> false / default
> >> >>     only do an unclean leader election if there may be possible data
> >> loss
> >> >>
> >> >> true / always
> >> >>     always do an unclean leader election if we can't immediately
> elect a
> >> >> leader
> >> >>
> >> >> It's a bit awkward that false maps to default rather than to never.
> But
> >> >> this awkwardness exists if we use two different configuration keys as
> >> well.
> >> >> The reason for the awkwardness is that we simply don't want most of
> the
> >> >> people currently setting unclean.leader.election.enable=false to get
> the
> >> >> "never" behavior. We have to bite that bullet. Better to be clear and
> >> >> explicit than hide it.
> >> >>
> >> >> Another thing that's a bit awkward is having two different ways to do
> >> >> unclean leader election specified in the KIP. You descirbe two
> methods:
> >> the
> >> >> simple "choose the last leader" method, and the "unclean recovery
> >> manager"
> >> >> method. I understand why you did it this way -- "choose the last
> >> leader" is
> >> >> simple, and will help us deliver an implementation quickly, while the
> >> URM
> >> >> is preferable in the long term. My suggestion here is to separate the
> >> >> decision of HOW to do unclean leader election from the decision of
> WHEN
> >> to
> >> >> do it.
> >> >>
> >> >> So in other words, have "unclean.leader.election.enable" specify
> when we
> >> >> do unclean leader election, and have a new configuration like
> >> >> "unclean.recovery.manager.enable" to determine if we use the URM.
> >> >> Presumably the URM will take some time to get fully stable, so this
> can
> >> >> default to false for a while, and we can flip the default to true
> when
> >> we
> >> >> feel ready.
> >> >>
> >> >> The URM is somewhat under-described here. I think we need a few
> >> >> configurations here for it. For example, we need a configuration to
> >> specify
> >> >> how long it should wait for a broker to respond to its RPCs before
> >> moving
> >> >> on. We also need to understand how the URM interacts with
> >> >> unclean.leader.election.enable=always. I assume that with "always" we
> >> will
> >> >> just unconditionally use the URM rather than choosing randomly. But
> this
> >> >> should be spelled out in the KIP.
> >> >>
> >> >> >
> >> >> > DescribeTopicRequest
> >> >> >
> >> >> >    1.
> >> >> >    Yes, the plan is to replace the MetadataRequest with the
> >> >> >    DescribeTopicRequest for the admin clients. Will check the
> details.
> >> >>
> >> >> Sounds good. But as I said, you need to specify how AdminClient
> >> interacts
> >> >> with the new request. This will involve adding some fields to
> >> >> TopicDescription.java. And you need to specify the changes to the
> >> >> kafka-topics.sh command line tool. Otherwise we cannot use the tool
> to
> >> see
> >> >> the new information.
> >> >>
> >> >> The new requests, DescribeTopicRequest and GetReplicaLogInfoRequest,
> >> need
> >> >> to have limits placed on them so that their size can't be infinite.
> We
> >> >> don't want to propagate the current problems of MetadataRequest,
> where
> >> >> clients can request massive responses that can mess up the JVM when
> >> handled.
> >> >>
> >> >> Adding limits is simple for GetReplicaLogInfoRequest -- we can just
> say
> >> >> that only 2000 partitions at a time can be requested. For
> >> >> DescribeTopicRequest we can probably just limit to 20 topics or
> >> something
> >> >> like that, to avoid the complexity of doing pagination in this KIP.
> >> >>
> >> >> >    2.
> >> >> >    I can let the broker load the ELR info so that they can serve
> the
> >> >> >    DescribeTopicRequest as well.
> >> >>
> >> >> Yes, it's fine to add to MetadataCache. In fact, you'll be loading it
> >> >> anyway once it's added to PartitionImage.
> >> >>
> >> >> >    3.
> >> >> >    Yeah, it does not make sense to have the topic id if
> >> >> >    DescribeTopicRequest is only used by the admin client.
> >> >>
> >> >> OK. That makes things simpler. We can always create a new API later
> >> >> (hopefully not in this KIP!) to query by topic ID.
> >> >>
> >> >> >
> >> >> >
> >> >> > Metrics
> >> >> >
> >> >> > As for overall cluster health metrics, I think under-min-ISR is
> still
> >> a
> >> >> > useful one. ELR is more like a safety belt. When the ELR is used,
> the
> >> >> > cluster availability has already been impacted.
> >> >> >
> >> >> > Maybe we can have a metric to count the partitions that sum(ISR,
> ELR)
> >> <
> >> >> min
> >> >> > ISR. What do you think?
> >> >>
> >> >> How about:
> >> >>
> >> >> A.  a metric for the totoal number of under-min-isr partitions? We
> don't
> >> >> have that in Apache Kafka at the moment.
> >> >>
> >> >> B. a metric for the number of unclean leader elections we did (for
> >> >> simplicity, it can reset to 0 on controller restart: we expect
> people to
> >> >> monitor the change over time anyway)
> >> >>
> >> >> best,
> >> >> Colin
> >> >>
> >> >>
> >> >> >
> >> >> > Yeah, for the ongoing unclean recoveries, the controller can keep
> an
> >> >> > accurate count through failover because partition registration can
> >> >> indicate
> >> >> > whether a recovery is needed. However, for the happened ones,
> unless
> >> we
> >> >> > want to persist the number somewhere, we can only figure it out
> from
> >> the
> >> >> > log.
> >> >> >
> >> >> > On Tue, Sep 12, 2023 at 3:16 PM Colin McCabe <cm...@apache.org>
> >> wrote:
> >> >> >
> >> >> >> Also, we should have metrics that show what is going on with
> regard
> >> to
> >> >> the
> >> >> >> eligible replica set. I'm not sure exactly what to suggest, but
> >> >> something
> >> >> >> that could identify when things are going wrong in the clsuter.
> >> >> >>
> >> >> >> For example, maybe a metric for partitions containing replicas
> that
> >> are
> >> >> >> ineligible to be leader? That would show a spike when a broker
> had an
> >> >> >> unclean restart.
> >> >> >>
> >> >> >> Ideally, we'd also have a metric that indicates when an unclear
> >> leader
> >> >> >> election or a recovery happened. It's a bit tricky because the
> simple
> >> >> >> thing, of tracking it per controller, may be a bit confusing
> during
> >> >> >> failovers.
> >> >> >>
> >> >> >> best,
> >> >> >> Colin
> >> >> >>
> >> >> >>
> >> >> >> On Tue, Sep 12, 2023, at 14:25, Colin McCabe wrote:
> >> >> >> > Hi Calvin,
> >> >> >> >
> >> >> >> > Thanks for the KIP. I think this is a great improvement.
> >> >> >> >
> >> >> >> >> Additional High Watermark advance requirement
> >> >> >> >
> >> >> >> > Typo: change "advance" to "advancement"
> >> >> >> >
> >> >> >> >> A bit recap of some key concepts.
> >> >> >> >
> >> >> >> > Typo: change "bit" to "quick"
> >> >> >> >
> >> >> >> >> Ack=1/all produce request. It defines when the Kafka server
> should
> >> >> >> respond to the produce request
> >> >> >> >
> >> >> >> > I think this section would be clearer if we talked about the new
> >> high
> >> >> >> > watermark advancement requirement first, and THEN talked about
> its
> >> >> >> > impact on acks=0, acks=1, and     acks=all.  acks=all is of
> course
> >> the
> >> >> >> > main case we care about here, so it would be good to lead with
> >> that,
> >> >> >> > rather than delving into the technicalities of acks=0/1 first.
> >> >> >> >
> >> >> >> >> Unclean recovery
> >> >> >> >
> >> >> >> > So, here you are introducing a new configuration,
> >> >> >> > unclean.recovery.strategy. The difficult thing here is that
> there
> >> is a
> >> >> >> > lot of overlap with unclean.leader.election.enable. So we have 3
> >> >> >> > different settings for unclean.recovery.strategy, plus 2
> different
> >> >> >> > settings for unclean.leader.election.enable, giving a cross
> >> product of
> >> >> >> > 6 different options. The following "unclean recovery manager"
> >> section
> >> >> >> > only applies to one fo those 6 different possibilities (I
> think?)
> >> >> >> >
> >> >> >> > I simply don't think we need so many different election types.
> >> Really
> >> >> >> > the use-cases we need are people who want NO unclean elections,
> >> people
> >> >> >> > who want "the reasonable thing" and people who want
> avaialbility at
> >> >> all
> >> >> >> > costs.
> >> >> >> >
> >> >> >> > Overall, I feel like the first half of the KIP is about the ELR,
> >> and
> >> >> >> > the second half is about reworking unclean leader election. It
> >> might
> >> >> be
> >> >> >> > better to move that second half to a separate KIP so that we can
> >> >> figure
> >> >> >> > it out fully. It should be fine to punt this until later and
> just
> >> have
> >> >> >> > the current behavior on empty ELR be waiting for the last known
> >> leader
> >> >> >> > to return. After all, that's what we do today.
> >> >> >> >
> >> >> >> >> DescribeTopicRequest
> >> >> >> >
> >> >> >> > Is the intention for AdminClient to use this RPC for
> >> >> >> > Admin#describeTopics ? If so, we need to describe all of the
> >> changes
> >> >> to
> >> >> >> > the admin client API, as well as changes to command-line tools
> like
> >> >> >> > kafka-topics.sh (if there are any). For example, you will
> probably
> >> >> need
> >> >> >> > changes to TopicDescription.java. You will also need to provide
> >> all of
> >> >> >> > the things that admin client needs -- for example,
> >> >> >> > TopicAuthorizedOperations.
> >> >> >> >
> >> >> >> > I also don't think the controller should serve this request. We
> >> want
> >> >> to
> >> >> >> > minimize load on the controller. Just like with the other
> metadata
> >> >> >> > requests like MetadataRequest, this should be served by brokers.
> >> >> >> >
> >> >> >> > It's a bit confusing why both topic ID and topic name are
> provided
> >> to
> >> >> >> > this API. Is the intention that callers should set one but not
> the
> >> >> >> > other? Or both? This needs to be clarified. Also, if we do want
> to
> >> >> >> > support lookups by UUID, that is another thing that needs to be
> >> added
> >> >> >> > to adminclient.
> >> >> >> >
> >> >> >> > In general, I feel like this should also probably be its own KIP
> >> since
> >> >> >> > it's fairly complex
> >> >> >> >
> >> >> >> > best,
> >> >> >> > Colin
> >> >> >> >
> >> >> >> >
> >> >> >> > On Thu, Aug 10, 2023, at 15:46, Calvin Liu 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
> >> >> >>
> >> >>
> >>
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Colin McCabe <cm...@apache.org>.
On Thu, Sep 14, 2023, at 22:23, Calvin Liu wrote:
> Hi Colin
> 1. I think using the new config name is more clear.
>        a. The unclean leader election is actually removed if unclean
> recovery is in use.
>        b. Using multiple values in unclean.leader.election.enable is
> confusing and it will be more confusing after people forget about this
> discussion.

Hi Calvin,

So, the proposal is that if someone sets "unclean.leader.election.enable = true" but then sets one of your new configurations, the value of unclean.leader.election.enable is ignored? That seems less clear to me, not more. Just in general, having multiple configuration keys to control the same thing confuses users. Basically, they are sitting at a giant control panel, and some of the levers do nothing.

> 2. Sorry I forgot to mention in the response that I did add the
> unclean.recovery.Enabled flag.

The configuration key should be "unclean.recovery.manager.enabled", right? Becuase we can do "unclean recovery" without the manager. Disabling the manager just means we use a different mechanism for recovery.

>        c. Maybe I underestimated the challenge of replacing the config. Any
> implementation problems ahead?

There are four levels of overrides for unclean.leader.election.enable.

1. static configuration for node.
    This goes in the configuration file, typically named server.properties

2. dynamic configuration for node default
  ConfigResource(type=BROKER, name="")

3. dynamic configuration for node
  ConfigResource(type=BROKER, name=<controller id>)

4. dynamic configuration for topic
  ConfigResource(type=TOPIC, name=<topic-name>)

How do these 4 levels of overrides interact with your new configurations? If the new configurations dominate over the old ones, it seems like this will get a lot more confusing to implement (and also to use.)

Again, I'd recommend just adding some new values to unclean.leader.election.enable. It's simple and will prevent user confusion (as well as developer confusion.)

best,
Colin


> 3. About the admin client, I mentioned 3 changes in the client. Anything
> else I missed in the KIP?
>       a. The client will switch to using the new RPC instead of
> MetadataRequest for the topics.
>       b. The TopicPartitionInfo used in TopicDescription needs to add new
> fields related to the ELR.
>       c. The outputs will add the ELR related fields.
>
> On Thu, Sep 14, 2023 at 9:19 PM Colin McCabe <cm...@apache.org> wrote:
>
>> Hi Calvin,
>>
>> Thanks for the changes.
>>
>> 1. Earlier I commented that creating "unclean.recovery.strategy " is not
>> necessary, and we can just reuse the existing
>> "unclean.leader.election.enable" configuration key. Let's discuss that.
>>
>> 2.I also don't understand why you didn't add a configuration to enable or
>> disable the Unclean Recovery Manager. This seems like a very simple way to
>> handle the staging issue which we discussed. The URM can just be turned off
>> until it is production ready. Let's discuss this.
>>
>> 3. You still need to describe the changes to AdminClient that are needed
>> to use DescribeTopicRequest.
>>
>> Keep at it. It's looking better. :)
>>
>> best,
>> Colin
>>
>>
>> On Thu, Sep 14, 2023, at 11:03, Calvin Liu wrote:
>> > Hi Colin
>> > Thanks for the comments!
>> >
>> > I did the following changes
>> >
>> >    1.
>> >
>> >    Simplified the API spec section to only include the diff.
>> >    2.
>> >
>> >    Reordered the HWM requirement section.
>> >    3.
>> >
>> >    Removed the URM implementation details to keep the necessary
>> >    characteristics to perform the unclean recovery.
>> >    1.
>> >
>> >       When to perform the unclean recovery
>> >       2.
>> >
>> >       Under different config, how the unclean recovery finds the leader.
>> >       3.
>> >
>> >       How the config unclean.leader.election.enable and
>> >       unclean.recovery.strategy are converted when users enable/disable
>> the
>> >       unclean recovery.
>> >       4.
>> >
>> >    More details about how we change admin client.
>> >    5.
>> >
>> >    API limits on the GetReplicaLogInfoRequest and DescribeTopicRequest.
>> >    6.
>> >
>> >    Two metrics added
>> >    1.
>> >
>> >       Kafka.controller.global_under_min_isr_partition_count
>> >       2.
>> >
>> >       kafka.controller.unclean_recovery_finished_count
>> >
>> >
>> > On Wed, Sep 13, 2023 at 10:46 AM Colin McCabe <cm...@apache.org>
>> wrote:
>> >
>> >> On Tue, Sep 12, 2023, at 17:21, Calvin Liu wrote:
>> >> > Hi Colin
>> >> > Thanks for the comments!
>> >> >
>> >>
>> >> Hi Calvin,
>> >>
>> >> Thanks again for the KIP.
>> >>
>> >> One meta-comment: it's usually better to just do a diff on a message
>> spec
>> >> file or java file if you're including changes to it in the KIP. This is
>> >> easier to read than looking for "new fields begin" etc. in the text, and
>> >> gracefully handles the case where existing fields were changed.
>> >>
>> >> > Rewrite the Additional High Watermark advancement requirement
>> >> > There was feedback on this section that some readers may not be
>> familiar
>> >> > with HWM and Ack=0,1,all requests. This can help them understand the
>> >> > proposal. I will rewrite this part for more readability.
>> >> >
>> >>
>> >> To be clear, I wasn't suggesting dropping either section. I agree that
>> >> they add useful background. I was just suggesting that we should discuss
>> >> the "acks" setting AFTER discussing the new high watermark advancement
>> >> conditions. We also should discuss acks=0. While it isn't conceptually
>> much
>> >> different than acks=1 here, its omission from this section is confusing.
>> >>
>> >> > Unclean recovery
>> >> >
>> >> > The plan is to replace the unclean.leader.election.enable with
>> >> > unclean.recovery.strategy. If the Unclean Recovery is enabled then it
>> >> deals
>> >> > with the three options in the unclean.recovery.strategy.
>> >> >
>> >> >
>> >> > Let’s refine the Unclean Recovery. We have already taken a lot of
>> >> > suggestions and I hope to enhance the durability of Kafka to the next
>> >> level
>> >> > with this KIP.
>> >>
>> >> I am OK with doing the unclean leader recovery improvements in this KIP.
>> >> However, I think we need to really work on the configuration settings.
>> >>
>> >> Configuration overrides are often quite messy. For example, the cases
>> >> where we have log.roll.hours and log.roll.segment.ms, the user has to
>> >> remember which one takes precedence, and it is not obvious. So, rather
>> than
>> >> creating a new configuration, why not add additional values to
>> >> "unclean.leader.election.enable"? I think this will be simpler for
>> people
>> >> to understand, and simpler in the code as well.
>> >>
>> >> What if we continued to use "unclean.leader.election.enable" but
>> extended
>> >> it so that it took a string? Then the string could have these values:
>> >>
>> >> never
>> >>     never automatically do an unclean leader election under any
>> conditions
>> >>
>> >> false / default
>> >>     only do an unclean leader election if there may be possible data
>> loss
>> >>
>> >> true / always
>> >>     always do an unclean leader election if we can't immediately elect a
>> >> leader
>> >>
>> >> It's a bit awkward that false maps to default rather than to never. But
>> >> this awkwardness exists if we use two different configuration keys as
>> well.
>> >> The reason for the awkwardness is that we simply don't want most of the
>> >> people currently setting unclean.leader.election.enable=false to get the
>> >> "never" behavior. We have to bite that bullet. Better to be clear and
>> >> explicit than hide it.
>> >>
>> >> Another thing that's a bit awkward is having two different ways to do
>> >> unclean leader election specified in the KIP. You descirbe two methods:
>> the
>> >> simple "choose the last leader" method, and the "unclean recovery
>> manager"
>> >> method. I understand why you did it this way -- "choose the last
>> leader" is
>> >> simple, and will help us deliver an implementation quickly, while the
>> URM
>> >> is preferable in the long term. My suggestion here is to separate the
>> >> decision of HOW to do unclean leader election from the decision of WHEN
>> to
>> >> do it.
>> >>
>> >> So in other words, have "unclean.leader.election.enable" specify when we
>> >> do unclean leader election, and have a new configuration like
>> >> "unclean.recovery.manager.enable" to determine if we use the URM.
>> >> Presumably the URM will take some time to get fully stable, so this can
>> >> default to false for a while, and we can flip the default to true when
>> we
>> >> feel ready.
>> >>
>> >> The URM is somewhat under-described here. I think we need a few
>> >> configurations here for it. For example, we need a configuration to
>> specify
>> >> how long it should wait for a broker to respond to its RPCs before
>> moving
>> >> on. We also need to understand how the URM interacts with
>> >> unclean.leader.election.enable=always. I assume that with "always" we
>> will
>> >> just unconditionally use the URM rather than choosing randomly. But this
>> >> should be spelled out in the KIP.
>> >>
>> >> >
>> >> > DescribeTopicRequest
>> >> >
>> >> >    1.
>> >> >    Yes, the plan is to replace the MetadataRequest with the
>> >> >    DescribeTopicRequest for the admin clients. Will check the details.
>> >>
>> >> Sounds good. But as I said, you need to specify how AdminClient
>> interacts
>> >> with the new request. This will involve adding some fields to
>> >> TopicDescription.java. And you need to specify the changes to the
>> >> kafka-topics.sh command line tool. Otherwise we cannot use the tool to
>> see
>> >> the new information.
>> >>
>> >> The new requests, DescribeTopicRequest and GetReplicaLogInfoRequest,
>> need
>> >> to have limits placed on them so that their size can't be infinite. We
>> >> don't want to propagate the current problems of MetadataRequest, where
>> >> clients can request massive responses that can mess up the JVM when
>> handled.
>> >>
>> >> Adding limits is simple for GetReplicaLogInfoRequest -- we can just say
>> >> that only 2000 partitions at a time can be requested. For
>> >> DescribeTopicRequest we can probably just limit to 20 topics or
>> something
>> >> like that, to avoid the complexity of doing pagination in this KIP.
>> >>
>> >> >    2.
>> >> >    I can let the broker load the ELR info so that they can serve the
>> >> >    DescribeTopicRequest as well.
>> >>
>> >> Yes, it's fine to add to MetadataCache. In fact, you'll be loading it
>> >> anyway once it's added to PartitionImage.
>> >>
>> >> >    3.
>> >> >    Yeah, it does not make sense to have the topic id if
>> >> >    DescribeTopicRequest is only used by the admin client.
>> >>
>> >> OK. That makes things simpler. We can always create a new API later
>> >> (hopefully not in this KIP!) to query by topic ID.
>> >>
>> >> >
>> >> >
>> >> > Metrics
>> >> >
>> >> > As for overall cluster health metrics, I think under-min-ISR is still
>> a
>> >> > useful one. ELR is more like a safety belt. When the ELR is used, the
>> >> > cluster availability has already been impacted.
>> >> >
>> >> > Maybe we can have a metric to count the partitions that sum(ISR, ELR)
>> <
>> >> min
>> >> > ISR. What do you think?
>> >>
>> >> How about:
>> >>
>> >> A.  a metric for the totoal number of under-min-isr partitions? We don't
>> >> have that in Apache Kafka at the moment.
>> >>
>> >> B. a metric for the number of unclean leader elections we did (for
>> >> simplicity, it can reset to 0 on controller restart: we expect people to
>> >> monitor the change over time anyway)
>> >>
>> >> best,
>> >> Colin
>> >>
>> >>
>> >> >
>> >> > Yeah, for the ongoing unclean recoveries, the controller can keep an
>> >> > accurate count through failover because partition registration can
>> >> indicate
>> >> > whether a recovery is needed. However, for the happened ones, unless
>> we
>> >> > want to persist the number somewhere, we can only figure it out from
>> the
>> >> > log.
>> >> >
>> >> > On Tue, Sep 12, 2023 at 3:16 PM Colin McCabe <cm...@apache.org>
>> wrote:
>> >> >
>> >> >> Also, we should have metrics that show what is going on with regard
>> to
>> >> the
>> >> >> eligible replica set. I'm not sure exactly what to suggest, but
>> >> something
>> >> >> that could identify when things are going wrong in the clsuter.
>> >> >>
>> >> >> For example, maybe a metric for partitions containing replicas that
>> are
>> >> >> ineligible to be leader? That would show a spike when a broker had an
>> >> >> unclean restart.
>> >> >>
>> >> >> Ideally, we'd also have a metric that indicates when an unclear
>> leader
>> >> >> election or a recovery happened. It's a bit tricky because the simple
>> >> >> thing, of tracking it per controller, may be a bit confusing during
>> >> >> failovers.
>> >> >>
>> >> >> best,
>> >> >> Colin
>> >> >>
>> >> >>
>> >> >> On Tue, Sep 12, 2023, at 14:25, Colin McCabe wrote:
>> >> >> > Hi Calvin,
>> >> >> >
>> >> >> > Thanks for the KIP. I think this is a great improvement.
>> >> >> >
>> >> >> >> Additional High Watermark advance requirement
>> >> >> >
>> >> >> > Typo: change "advance" to "advancement"
>> >> >> >
>> >> >> >> A bit recap of some key concepts.
>> >> >> >
>> >> >> > Typo: change "bit" to "quick"
>> >> >> >
>> >> >> >> Ack=1/all produce request. It defines when the Kafka server should
>> >> >> respond to the produce request
>> >> >> >
>> >> >> > I think this section would be clearer if we talked about the new
>> high
>> >> >> > watermark advancement requirement first, and THEN talked about its
>> >> >> > impact on acks=0, acks=1, and     acks=all.  acks=all is of course
>> the
>> >> >> > main case we care about here, so it would be good to lead with
>> that,
>> >> >> > rather than delving into the technicalities of acks=0/1 first.
>> >> >> >
>> >> >> >> Unclean recovery
>> >> >> >
>> >> >> > So, here you are introducing a new configuration,
>> >> >> > unclean.recovery.strategy. The difficult thing here is that there
>> is a
>> >> >> > lot of overlap with unclean.leader.election.enable. So we have 3
>> >> >> > different settings for unclean.recovery.strategy, plus 2 different
>> >> >> > settings for unclean.leader.election.enable, giving a cross
>> product of
>> >> >> > 6 different options. The following "unclean recovery manager"
>> section
>> >> >> > only applies to one fo those 6 different possibilities (I think?)
>> >> >> >
>> >> >> > I simply don't think we need so many different election types.
>> Really
>> >> >> > the use-cases we need are people who want NO unclean elections,
>> people
>> >> >> > who want "the reasonable thing" and people who want avaialbility at
>> >> all
>> >> >> > costs.
>> >> >> >
>> >> >> > Overall, I feel like the first half of the KIP is about the ELR,
>> and
>> >> >> > the second half is about reworking unclean leader election. It
>> might
>> >> be
>> >> >> > better to move that second half to a separate KIP so that we can
>> >> figure
>> >> >> > it out fully. It should be fine to punt this until later and just
>> have
>> >> >> > the current behavior on empty ELR be waiting for the last known
>> leader
>> >> >> > to return. After all, that's what we do today.
>> >> >> >
>> >> >> >> DescribeTopicRequest
>> >> >> >
>> >> >> > Is the intention for AdminClient to use this RPC for
>> >> >> > Admin#describeTopics ? If so, we need to describe all of the
>> changes
>> >> to
>> >> >> > the admin client API, as well as changes to command-line tools like
>> >> >> > kafka-topics.sh (if there are any). For example, you will probably
>> >> need
>> >> >> > changes to TopicDescription.java. You will also need to provide
>> all of
>> >> >> > the things that admin client needs -- for example,
>> >> >> > TopicAuthorizedOperations.
>> >> >> >
>> >> >> > I also don't think the controller should serve this request. We
>> want
>> >> to
>> >> >> > minimize load on the controller. Just like with the other metadata
>> >> >> > requests like MetadataRequest, this should be served by brokers.
>> >> >> >
>> >> >> > It's a bit confusing why both topic ID and topic name are provided
>> to
>> >> >> > this API. Is the intention that callers should set one but not the
>> >> >> > other? Or both? This needs to be clarified. Also, if we do want to
>> >> >> > support lookups by UUID, that is another thing that needs to be
>> added
>> >> >> > to adminclient.
>> >> >> >
>> >> >> > In general, I feel like this should also probably be its own KIP
>> since
>> >> >> > it's fairly complex
>> >> >> >
>> >> >> > best,
>> >> >> > Colin
>> >> >> >
>> >> >> >
>> >> >> > On Thu, Aug 10, 2023, at 15:46, Calvin Liu 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
>> >> >>
>> >>
>>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Calvin Liu <ca...@confluent.io.INVALID>.
Hi Colin
1. I think using the new config name is more clear.
       a. The unclean leader election is actually removed if unclean
recovery is in use.
       b. Using multiple values in unclean.leader.election.enable is
confusing and it will be more confusing after people forget about this
discussion.
       c. Maybe I underestimated the challenge of replacing the config. Any
implementation problems ahead?
2. Sorry I forgot to mention in the response that I did add the
unclean.recovery.Enabled flag.
3. About the admin client, I mentioned 3 changes in the client. Anything
else I missed in the KIP?
      a. The client will switch to using the new RPC instead of
MetadataRequest for the topics.
      b. The TopicPartitionInfo used in TopicDescription needs to add new
fields related to the ELR.
      c. The outputs will add the ELR related fields.

On Thu, Sep 14, 2023 at 9:19 PM Colin McCabe <cm...@apache.org> wrote:

> Hi Calvin,
>
> Thanks for the changes.
>
> 1. Earlier I commented that creating "unclean.recovery.strategy " is not
> necessary, and we can just reuse the existing
> "unclean.leader.election.enable" configuration key. Let's discuss that.
>
> 2.I also don't understand why you didn't add a configuration to enable or
> disable the Unclean Recovery Manager. This seems like a very simple way to
> handle the staging issue which we discussed. The URM can just be turned off
> until it is production ready. Let's discuss this.
>
> 3. You still need to describe the changes to AdminClient that are needed
> to use DescribeTopicRequest.
>
> Keep at it. It's looking better. :)
>
> best,
> Colin
>
>
> On Thu, Sep 14, 2023, at 11:03, Calvin Liu wrote:
> > Hi Colin
> > Thanks for the comments!
> >
> > I did the following changes
> >
> >    1.
> >
> >    Simplified the API spec section to only include the diff.
> >    2.
> >
> >    Reordered the HWM requirement section.
> >    3.
> >
> >    Removed the URM implementation details to keep the necessary
> >    characteristics to perform the unclean recovery.
> >    1.
> >
> >       When to perform the unclean recovery
> >       2.
> >
> >       Under different config, how the unclean recovery finds the leader.
> >       3.
> >
> >       How the config unclean.leader.election.enable and
> >       unclean.recovery.strategy are converted when users enable/disable
> the
> >       unclean recovery.
> >       4.
> >
> >    More details about how we change admin client.
> >    5.
> >
> >    API limits on the GetReplicaLogInfoRequest and DescribeTopicRequest.
> >    6.
> >
> >    Two metrics added
> >    1.
> >
> >       Kafka.controller.global_under_min_isr_partition_count
> >       2.
> >
> >       kafka.controller.unclean_recovery_finished_count
> >
> >
> > On Wed, Sep 13, 2023 at 10:46 AM Colin McCabe <cm...@apache.org>
> wrote:
> >
> >> On Tue, Sep 12, 2023, at 17:21, Calvin Liu wrote:
> >> > Hi Colin
> >> > Thanks for the comments!
> >> >
> >>
> >> Hi Calvin,
> >>
> >> Thanks again for the KIP.
> >>
> >> One meta-comment: it's usually better to just do a diff on a message
> spec
> >> file or java file if you're including changes to it in the KIP. This is
> >> easier to read than looking for "new fields begin" etc. in the text, and
> >> gracefully handles the case where existing fields were changed.
> >>
> >> > Rewrite the Additional High Watermark advancement requirement
> >> > There was feedback on this section that some readers may not be
> familiar
> >> > with HWM and Ack=0,1,all requests. This can help them understand the
> >> > proposal. I will rewrite this part for more readability.
> >> >
> >>
> >> To be clear, I wasn't suggesting dropping either section. I agree that
> >> they add useful background. I was just suggesting that we should discuss
> >> the "acks" setting AFTER discussing the new high watermark advancement
> >> conditions. We also should discuss acks=0. While it isn't conceptually
> much
> >> different than acks=1 here, its omission from this section is confusing.
> >>
> >> > Unclean recovery
> >> >
> >> > The plan is to replace the unclean.leader.election.enable with
> >> > unclean.recovery.strategy. If the Unclean Recovery is enabled then it
> >> deals
> >> > with the three options in the unclean.recovery.strategy.
> >> >
> >> >
> >> > Let’s refine the Unclean Recovery. We have already taken a lot of
> >> > suggestions and I hope to enhance the durability of Kafka to the next
> >> level
> >> > with this KIP.
> >>
> >> I am OK with doing the unclean leader recovery improvements in this KIP.
> >> However, I think we need to really work on the configuration settings.
> >>
> >> Configuration overrides are often quite messy. For example, the cases
> >> where we have log.roll.hours and log.roll.segment.ms, the user has to
> >> remember which one takes precedence, and it is not obvious. So, rather
> than
> >> creating a new configuration, why not add additional values to
> >> "unclean.leader.election.enable"? I think this will be simpler for
> people
> >> to understand, and simpler in the code as well.
> >>
> >> What if we continued to use "unclean.leader.election.enable" but
> extended
> >> it so that it took a string? Then the string could have these values:
> >>
> >> never
> >>     never automatically do an unclean leader election under any
> conditions
> >>
> >> false / default
> >>     only do an unclean leader election if there may be possible data
> loss
> >>
> >> true / always
> >>     always do an unclean leader election if we can't immediately elect a
> >> leader
> >>
> >> It's a bit awkward that false maps to default rather than to never. But
> >> this awkwardness exists if we use two different configuration keys as
> well.
> >> The reason for the awkwardness is that we simply don't want most of the
> >> people currently setting unclean.leader.election.enable=false to get the
> >> "never" behavior. We have to bite that bullet. Better to be clear and
> >> explicit than hide it.
> >>
> >> Another thing that's a bit awkward is having two different ways to do
> >> unclean leader election specified in the KIP. You descirbe two methods:
> the
> >> simple "choose the last leader" method, and the "unclean recovery
> manager"
> >> method. I understand why you did it this way -- "choose the last
> leader" is
> >> simple, and will help us deliver an implementation quickly, while the
> URM
> >> is preferable in the long term. My suggestion here is to separate the
> >> decision of HOW to do unclean leader election from the decision of WHEN
> to
> >> do it.
> >>
> >> So in other words, have "unclean.leader.election.enable" specify when we
> >> do unclean leader election, and have a new configuration like
> >> "unclean.recovery.manager.enable" to determine if we use the URM.
> >> Presumably the URM will take some time to get fully stable, so this can
> >> default to false for a while, and we can flip the default to true when
> we
> >> feel ready.
> >>
> >> The URM is somewhat under-described here. I think we need a few
> >> configurations here for it. For example, we need a configuration to
> specify
> >> how long it should wait for a broker to respond to its RPCs before
> moving
> >> on. We also need to understand how the URM interacts with
> >> unclean.leader.election.enable=always. I assume that with "always" we
> will
> >> just unconditionally use the URM rather than choosing randomly. But this
> >> should be spelled out in the KIP.
> >>
> >> >
> >> > DescribeTopicRequest
> >> >
> >> >    1.
> >> >    Yes, the plan is to replace the MetadataRequest with the
> >> >    DescribeTopicRequest for the admin clients. Will check the details.
> >>
> >> Sounds good. But as I said, you need to specify how AdminClient
> interacts
> >> with the new request. This will involve adding some fields to
> >> TopicDescription.java. And you need to specify the changes to the
> >> kafka-topics.sh command line tool. Otherwise we cannot use the tool to
> see
> >> the new information.
> >>
> >> The new requests, DescribeTopicRequest and GetReplicaLogInfoRequest,
> need
> >> to have limits placed on them so that their size can't be infinite. We
> >> don't want to propagate the current problems of MetadataRequest, where
> >> clients can request massive responses that can mess up the JVM when
> handled.
> >>
> >> Adding limits is simple for GetReplicaLogInfoRequest -- we can just say
> >> that only 2000 partitions at a time can be requested. For
> >> DescribeTopicRequest we can probably just limit to 20 topics or
> something
> >> like that, to avoid the complexity of doing pagination in this KIP.
> >>
> >> >    2.
> >> >    I can let the broker load the ELR info so that they can serve the
> >> >    DescribeTopicRequest as well.
> >>
> >> Yes, it's fine to add to MetadataCache. In fact, you'll be loading it
> >> anyway once it's added to PartitionImage.
> >>
> >> >    3.
> >> >    Yeah, it does not make sense to have the topic id if
> >> >    DescribeTopicRequest is only used by the admin client.
> >>
> >> OK. That makes things simpler. We can always create a new API later
> >> (hopefully not in this KIP!) to query by topic ID.
> >>
> >> >
> >> >
> >> > Metrics
> >> >
> >> > As for overall cluster health metrics, I think under-min-ISR is still
> a
> >> > useful one. ELR is more like a safety belt. When the ELR is used, the
> >> > cluster availability has already been impacted.
> >> >
> >> > Maybe we can have a metric to count the partitions that sum(ISR, ELR)
> <
> >> min
> >> > ISR. What do you think?
> >>
> >> How about:
> >>
> >> A.  a metric for the totoal number of under-min-isr partitions? We don't
> >> have that in Apache Kafka at the moment.
> >>
> >> B. a metric for the number of unclean leader elections we did (for
> >> simplicity, it can reset to 0 on controller restart: we expect people to
> >> monitor the change over time anyway)
> >>
> >> best,
> >> Colin
> >>
> >>
> >> >
> >> > Yeah, for the ongoing unclean recoveries, the controller can keep an
> >> > accurate count through failover because partition registration can
> >> indicate
> >> > whether a recovery is needed. However, for the happened ones, unless
> we
> >> > want to persist the number somewhere, we can only figure it out from
> the
> >> > log.
> >> >
> >> > On Tue, Sep 12, 2023 at 3:16 PM Colin McCabe <cm...@apache.org>
> wrote:
> >> >
> >> >> Also, we should have metrics that show what is going on with regard
> to
> >> the
> >> >> eligible replica set. I'm not sure exactly what to suggest, but
> >> something
> >> >> that could identify when things are going wrong in the clsuter.
> >> >>
> >> >> For example, maybe a metric for partitions containing replicas that
> are
> >> >> ineligible to be leader? That would show a spike when a broker had an
> >> >> unclean restart.
> >> >>
> >> >> Ideally, we'd also have a metric that indicates when an unclear
> leader
> >> >> election or a recovery happened. It's a bit tricky because the simple
> >> >> thing, of tracking it per controller, may be a bit confusing during
> >> >> failovers.
> >> >>
> >> >> best,
> >> >> Colin
> >> >>
> >> >>
> >> >> On Tue, Sep 12, 2023, at 14:25, Colin McCabe wrote:
> >> >> > Hi Calvin,
> >> >> >
> >> >> > Thanks for the KIP. I think this is a great improvement.
> >> >> >
> >> >> >> Additional High Watermark advance requirement
> >> >> >
> >> >> > Typo: change "advance" to "advancement"
> >> >> >
> >> >> >> A bit recap of some key concepts.
> >> >> >
> >> >> > Typo: change "bit" to "quick"
> >> >> >
> >> >> >> Ack=1/all produce request. It defines when the Kafka server should
> >> >> respond to the produce request
> >> >> >
> >> >> > I think this section would be clearer if we talked about the new
> high
> >> >> > watermark advancement requirement first, and THEN talked about its
> >> >> > impact on acks=0, acks=1, and     acks=all.  acks=all is of course
> the
> >> >> > main case we care about here, so it would be good to lead with
> that,
> >> >> > rather than delving into the technicalities of acks=0/1 first.
> >> >> >
> >> >> >> Unclean recovery
> >> >> >
> >> >> > So, here you are introducing a new configuration,
> >> >> > unclean.recovery.strategy. The difficult thing here is that there
> is a
> >> >> > lot of overlap with unclean.leader.election.enable. So we have 3
> >> >> > different settings for unclean.recovery.strategy, plus 2 different
> >> >> > settings for unclean.leader.election.enable, giving a cross
> product of
> >> >> > 6 different options. The following "unclean recovery manager"
> section
> >> >> > only applies to one fo those 6 different possibilities (I think?)
> >> >> >
> >> >> > I simply don't think we need so many different election types.
> Really
> >> >> > the use-cases we need are people who want NO unclean elections,
> people
> >> >> > who want "the reasonable thing" and people who want avaialbility at
> >> all
> >> >> > costs.
> >> >> >
> >> >> > Overall, I feel like the first half of the KIP is about the ELR,
> and
> >> >> > the second half is about reworking unclean leader election. It
> might
> >> be
> >> >> > better to move that second half to a separate KIP so that we can
> >> figure
> >> >> > it out fully. It should be fine to punt this until later and just
> have
> >> >> > the current behavior on empty ELR be waiting for the last known
> leader
> >> >> > to return. After all, that's what we do today.
> >> >> >
> >> >> >> DescribeTopicRequest
> >> >> >
> >> >> > Is the intention for AdminClient to use this RPC for
> >> >> > Admin#describeTopics ? If so, we need to describe all of the
> changes
> >> to
> >> >> > the admin client API, as well as changes to command-line tools like
> >> >> > kafka-topics.sh (if there are any). For example, you will probably
> >> need
> >> >> > changes to TopicDescription.java. You will also need to provide
> all of
> >> >> > the things that admin client needs -- for example,
> >> >> > TopicAuthorizedOperations.
> >> >> >
> >> >> > I also don't think the controller should serve this request. We
> want
> >> to
> >> >> > minimize load on the controller. Just like with the other metadata
> >> >> > requests like MetadataRequest, this should be served by brokers.
> >> >> >
> >> >> > It's a bit confusing why both topic ID and topic name are provided
> to
> >> >> > this API. Is the intention that callers should set one but not the
> >> >> > other? Or both? This needs to be clarified. Also, if we do want to
> >> >> > support lookups by UUID, that is another thing that needs to be
> added
> >> >> > to adminclient.
> >> >> >
> >> >> > In general, I feel like this should also probably be its own KIP
> since
> >> >> > it's fairly complex
> >> >> >
> >> >> > best,
> >> >> > Colin
> >> >> >
> >> >> >
> >> >> > On Thu, Aug 10, 2023, at 15:46, Calvin Liu 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
> >> >>
> >>
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Colin McCabe <cm...@apache.org>.
Hi Calvin,

Thanks for the changes.

1. Earlier I commented that creating "unclean.recovery.strategy " is not necessary, and we can just reuse the existing "unclean.leader.election.enable" configuration key. Let's discuss that.

2.I also don't understand why you didn't add a configuration to enable or disable the Unclean Recovery Manager. This seems like a very simple way to handle the staging issue which we discussed. The URM can just be turned off until it is production ready. Let's discuss this.

3. You still need to describe the changes to AdminClient that are needed to use DescribeTopicRequest.

Keep at it. It's looking better. :)

best,
Colin


On Thu, Sep 14, 2023, at 11:03, Calvin Liu wrote:
> Hi Colin
> Thanks for the comments!
>
> I did the following changes
>
>    1.
>
>    Simplified the API spec section to only include the diff.
>    2.
>
>    Reordered the HWM requirement section.
>    3.
>
>    Removed the URM implementation details to keep the necessary
>    characteristics to perform the unclean recovery.
>    1.
>
>       When to perform the unclean recovery
>       2.
>
>       Under different config, how the unclean recovery finds the leader.
>       3.
>
>       How the config unclean.leader.election.enable and
>       unclean.recovery.strategy are converted when users enable/disable the
>       unclean recovery.
>       4.
>
>    More details about how we change admin client.
>    5.
>
>    API limits on the GetReplicaLogInfoRequest and DescribeTopicRequest.
>    6.
>
>    Two metrics added
>    1.
>
>       Kafka.controller.global_under_min_isr_partition_count
>       2.
>
>       kafka.controller.unclean_recovery_finished_count
>
>
> On Wed, Sep 13, 2023 at 10:46 AM Colin McCabe <cm...@apache.org> wrote:
>
>> On Tue, Sep 12, 2023, at 17:21, Calvin Liu wrote:
>> > Hi Colin
>> > Thanks for the comments!
>> >
>>
>> Hi Calvin,
>>
>> Thanks again for the KIP.
>>
>> One meta-comment: it's usually better to just do a diff on a message spec
>> file or java file if you're including changes to it in the KIP. This is
>> easier to read than looking for "new fields begin" etc. in the text, and
>> gracefully handles the case where existing fields were changed.
>>
>> > Rewrite the Additional High Watermark advancement requirement
>> > There was feedback on this section that some readers may not be familiar
>> > with HWM and Ack=0,1,all requests. This can help them understand the
>> > proposal. I will rewrite this part for more readability.
>> >
>>
>> To be clear, I wasn't suggesting dropping either section. I agree that
>> they add useful background. I was just suggesting that we should discuss
>> the "acks" setting AFTER discussing the new high watermark advancement
>> conditions. We also should discuss acks=0. While it isn't conceptually much
>> different than acks=1 here, its omission from this section is confusing.
>>
>> > Unclean recovery
>> >
>> > The plan is to replace the unclean.leader.election.enable with
>> > unclean.recovery.strategy. If the Unclean Recovery is enabled then it
>> deals
>> > with the three options in the unclean.recovery.strategy.
>> >
>> >
>> > Let’s refine the Unclean Recovery. We have already taken a lot of
>> > suggestions and I hope to enhance the durability of Kafka to the next
>> level
>> > with this KIP.
>>
>> I am OK with doing the unclean leader recovery improvements in this KIP.
>> However, I think we need to really work on the configuration settings.
>>
>> Configuration overrides are often quite messy. For example, the cases
>> where we have log.roll.hours and log.roll.segment.ms, the user has to
>> remember which one takes precedence, and it is not obvious. So, rather than
>> creating a new configuration, why not add additional values to
>> "unclean.leader.election.enable"? I think this will be simpler for people
>> to understand, and simpler in the code as well.
>>
>> What if we continued to use "unclean.leader.election.enable" but extended
>> it so that it took a string? Then the string could have these values:
>>
>> never
>>     never automatically do an unclean leader election under any conditions
>>
>> false / default
>>     only do an unclean leader election if there may be possible data loss
>>
>> true / always
>>     always do an unclean leader election if we can't immediately elect a
>> leader
>>
>> It's a bit awkward that false maps to default rather than to never. But
>> this awkwardness exists if we use two different configuration keys as well.
>> The reason for the awkwardness is that we simply don't want most of the
>> people currently setting unclean.leader.election.enable=false to get the
>> "never" behavior. We have to bite that bullet. Better to be clear and
>> explicit than hide it.
>>
>> Another thing that's a bit awkward is having two different ways to do
>> unclean leader election specified in the KIP. You descirbe two methods: the
>> simple "choose the last leader" method, and the "unclean recovery manager"
>> method. I understand why you did it this way -- "choose the last leader" is
>> simple, and will help us deliver an implementation quickly, while the URM
>> is preferable in the long term. My suggestion here is to separate the
>> decision of HOW to do unclean leader election from the decision of WHEN to
>> do it.
>>
>> So in other words, have "unclean.leader.election.enable" specify when we
>> do unclean leader election, and have a new configuration like
>> "unclean.recovery.manager.enable" to determine if we use the URM.
>> Presumably the URM will take some time to get fully stable, so this can
>> default to false for a while, and we can flip the default to true when we
>> feel ready.
>>
>> The URM is somewhat under-described here. I think we need a few
>> configurations here for it. For example, we need a configuration to specify
>> how long it should wait for a broker to respond to its RPCs before moving
>> on. We also need to understand how the URM interacts with
>> unclean.leader.election.enable=always. I assume that with "always" we will
>> just unconditionally use the URM rather than choosing randomly. But this
>> should be spelled out in the KIP.
>>
>> >
>> > DescribeTopicRequest
>> >
>> >    1.
>> >    Yes, the plan is to replace the MetadataRequest with the
>> >    DescribeTopicRequest for the admin clients. Will check the details.
>>
>> Sounds good. But as I said, you need to specify how AdminClient interacts
>> with the new request. This will involve adding some fields to
>> TopicDescription.java. And you need to specify the changes to the
>> kafka-topics.sh command line tool. Otherwise we cannot use the tool to see
>> the new information.
>>
>> The new requests, DescribeTopicRequest and GetReplicaLogInfoRequest, need
>> to have limits placed on them so that their size can't be infinite. We
>> don't want to propagate the current problems of MetadataRequest, where
>> clients can request massive responses that can mess up the JVM when handled.
>>
>> Adding limits is simple for GetReplicaLogInfoRequest -- we can just say
>> that only 2000 partitions at a time can be requested. For
>> DescribeTopicRequest we can probably just limit to 20 topics or something
>> like that, to avoid the complexity of doing pagination in this KIP.
>>
>> >    2.
>> >    I can let the broker load the ELR info so that they can serve the
>> >    DescribeTopicRequest as well.
>>
>> Yes, it's fine to add to MetadataCache. In fact, you'll be loading it
>> anyway once it's added to PartitionImage.
>>
>> >    3.
>> >    Yeah, it does not make sense to have the topic id if
>> >    DescribeTopicRequest is only used by the admin client.
>>
>> OK. That makes things simpler. We can always create a new API later
>> (hopefully not in this KIP!) to query by topic ID.
>>
>> >
>> >
>> > Metrics
>> >
>> > As for overall cluster health metrics, I think under-min-ISR is still a
>> > useful one. ELR is more like a safety belt. When the ELR is used, the
>> > cluster availability has already been impacted.
>> >
>> > Maybe we can have a metric to count the partitions that sum(ISR, ELR) <
>> min
>> > ISR. What do you think?
>>
>> How about:
>>
>> A.  a metric for the totoal number of under-min-isr partitions? We don't
>> have that in Apache Kafka at the moment.
>>
>> B. a metric for the number of unclean leader elections we did (for
>> simplicity, it can reset to 0 on controller restart: we expect people to
>> monitor the change over time anyway)
>>
>> best,
>> Colin
>>
>>
>> >
>> > Yeah, for the ongoing unclean recoveries, the controller can keep an
>> > accurate count through failover because partition registration can
>> indicate
>> > whether a recovery is needed. However, for the happened ones, unless we
>> > want to persist the number somewhere, we can only figure it out from the
>> > log.
>> >
>> > On Tue, Sep 12, 2023 at 3:16 PM Colin McCabe <cm...@apache.org> wrote:
>> >
>> >> Also, we should have metrics that show what is going on with regard to
>> the
>> >> eligible replica set. I'm not sure exactly what to suggest, but
>> something
>> >> that could identify when things are going wrong in the clsuter.
>> >>
>> >> For example, maybe a metric for partitions containing replicas that are
>> >> ineligible to be leader? That would show a spike when a broker had an
>> >> unclean restart.
>> >>
>> >> Ideally, we'd also have a metric that indicates when an unclear leader
>> >> election or a recovery happened. It's a bit tricky because the simple
>> >> thing, of tracking it per controller, may be a bit confusing during
>> >> failovers.
>> >>
>> >> best,
>> >> Colin
>> >>
>> >>
>> >> On Tue, Sep 12, 2023, at 14:25, Colin McCabe wrote:
>> >> > Hi Calvin,
>> >> >
>> >> > Thanks for the KIP. I think this is a great improvement.
>> >> >
>> >> >> Additional High Watermark advance requirement
>> >> >
>> >> > Typo: change "advance" to "advancement"
>> >> >
>> >> >> A bit recap of some key concepts.
>> >> >
>> >> > Typo: change "bit" to "quick"
>> >> >
>> >> >> Ack=1/all produce request. It defines when the Kafka server should
>> >> respond to the produce request
>> >> >
>> >> > I think this section would be clearer if we talked about the new high
>> >> > watermark advancement requirement first, and THEN talked about its
>> >> > impact on acks=0, acks=1, and     acks=all.  acks=all is of course the
>> >> > main case we care about here, so it would be good to lead with that,
>> >> > rather than delving into the technicalities of acks=0/1 first.
>> >> >
>> >> >> Unclean recovery
>> >> >
>> >> > So, here you are introducing a new configuration,
>> >> > unclean.recovery.strategy. The difficult thing here is that there is a
>> >> > lot of overlap with unclean.leader.election.enable. So we have 3
>> >> > different settings for unclean.recovery.strategy, plus 2 different
>> >> > settings for unclean.leader.election.enable, giving a cross product of
>> >> > 6 different options. The following "unclean recovery manager" section
>> >> > only applies to one fo those 6 different possibilities (I think?)
>> >> >
>> >> > I simply don't think we need so many different election types. Really
>> >> > the use-cases we need are people who want NO unclean elections, people
>> >> > who want "the reasonable thing" and people who want avaialbility at
>> all
>> >> > costs.
>> >> >
>> >> > Overall, I feel like the first half of the KIP is about the ELR, and
>> >> > the second half is about reworking unclean leader election. It might
>> be
>> >> > better to move that second half to a separate KIP so that we can
>> figure
>> >> > it out fully. It should be fine to punt this until later and just have
>> >> > the current behavior on empty ELR be waiting for the last known leader
>> >> > to return. After all, that's what we do today.
>> >> >
>> >> >> DescribeTopicRequest
>> >> >
>> >> > Is the intention for AdminClient to use this RPC for
>> >> > Admin#describeTopics ? If so, we need to describe all of the changes
>> to
>> >> > the admin client API, as well as changes to command-line tools like
>> >> > kafka-topics.sh (if there are any). For example, you will probably
>> need
>> >> > changes to TopicDescription.java. You will also need to provide all of
>> >> > the things that admin client needs -- for example,
>> >> > TopicAuthorizedOperations.
>> >> >
>> >> > I also don't think the controller should serve this request. We want
>> to
>> >> > minimize load on the controller. Just like with the other metadata
>> >> > requests like MetadataRequest, this should be served by brokers.
>> >> >
>> >> > It's a bit confusing why both topic ID and topic name are provided to
>> >> > this API. Is the intention that callers should set one but not the
>> >> > other? Or both? This needs to be clarified. Also, if we do want to
>> >> > support lookups by UUID, that is another thing that needs to be added
>> >> > to adminclient.
>> >> >
>> >> > In general, I feel like this should also probably be its own KIP since
>> >> > it's fairly complex
>> >> >
>> >> > best,
>> >> > Colin
>> >> >
>> >> >
>> >> > On Thu, Aug 10, 2023, at 15:46, Calvin Liu 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
>> >>
>>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Calvin Liu <ca...@confluent.io.INVALID>.
Hi Colin
Thanks for the comments!

I did the following changes

   1.

   Simplified the API spec section to only include the diff.
   2.

   Reordered the HWM requirement section.
   3.

   Removed the URM implementation details to keep the necessary
   characteristics to perform the unclean recovery.
   1.

      When to perform the unclean recovery
      2.

      Under different config, how the unclean recovery finds the leader.
      3.

      How the config unclean.leader.election.enable and
      unclean.recovery.strategy are converted when users enable/disable the
      unclean recovery.
      4.

   More details about how we change admin client.
   5.

   API limits on the GetReplicaLogInfoRequest and DescribeTopicRequest.
   6.

   Two metrics added
   1.

      Kafka.controller.global_under_min_isr_partition_count
      2.

      kafka.controller.unclean_recovery_finished_count


On Wed, Sep 13, 2023 at 10:46 AM Colin McCabe <cm...@apache.org> wrote:

> On Tue, Sep 12, 2023, at 17:21, Calvin Liu wrote:
> > Hi Colin
> > Thanks for the comments!
> >
>
> Hi Calvin,
>
> Thanks again for the KIP.
>
> One meta-comment: it's usually better to just do a diff on a message spec
> file or java file if you're including changes to it in the KIP. This is
> easier to read than looking for "new fields begin" etc. in the text, and
> gracefully handles the case where existing fields were changed.
>
> > Rewrite the Additional High Watermark advancement requirement
> > There was feedback on this section that some readers may not be familiar
> > with HWM and Ack=0,1,all requests. This can help them understand the
> > proposal. I will rewrite this part for more readability.
> >
>
> To be clear, I wasn't suggesting dropping either section. I agree that
> they add useful background. I was just suggesting that we should discuss
> the "acks" setting AFTER discussing the new high watermark advancement
> conditions. We also should discuss acks=0. While it isn't conceptually much
> different than acks=1 here, its omission from this section is confusing.
>
> > Unclean recovery
> >
> > The plan is to replace the unclean.leader.election.enable with
> > unclean.recovery.strategy. If the Unclean Recovery is enabled then it
> deals
> > with the three options in the unclean.recovery.strategy.
> >
> >
> > Let’s refine the Unclean Recovery. We have already taken a lot of
> > suggestions and I hope to enhance the durability of Kafka to the next
> level
> > with this KIP.
>
> I am OK with doing the unclean leader recovery improvements in this KIP.
> However, I think we need to really work on the configuration settings.
>
> Configuration overrides are often quite messy. For example, the cases
> where we have log.roll.hours and log.roll.segment.ms, the user has to
> remember which one takes precedence, and it is not obvious. So, rather than
> creating a new configuration, why not add additional values to
> "unclean.leader.election.enable"? I think this will be simpler for people
> to understand, and simpler in the code as well.
>
> What if we continued to use "unclean.leader.election.enable" but extended
> it so that it took a string? Then the string could have these values:
>
> never
>     never automatically do an unclean leader election under any conditions
>
> false / default
>     only do an unclean leader election if there may be possible data loss
>
> true / always
>     always do an unclean leader election if we can't immediately elect a
> leader
>
> It's a bit awkward that false maps to default rather than to never. But
> this awkwardness exists if we use two different configuration keys as well.
> The reason for the awkwardness is that we simply don't want most of the
> people currently setting unclean.leader.election.enable=false to get the
> "never" behavior. We have to bite that bullet. Better to be clear and
> explicit than hide it.
>
> Another thing that's a bit awkward is having two different ways to do
> unclean leader election specified in the KIP. You descirbe two methods: the
> simple "choose the last leader" method, and the "unclean recovery manager"
> method. I understand why you did it this way -- "choose the last leader" is
> simple, and will help us deliver an implementation quickly, while the URM
> is preferable in the long term. My suggestion here is to separate the
> decision of HOW to do unclean leader election from the decision of WHEN to
> do it.
>
> So in other words, have "unclean.leader.election.enable" specify when we
> do unclean leader election, and have a new configuration like
> "unclean.recovery.manager.enable" to determine if we use the URM.
> Presumably the URM will take some time to get fully stable, so this can
> default to false for a while, and we can flip the default to true when we
> feel ready.
>
> The URM is somewhat under-described here. I think we need a few
> configurations here for it. For example, we need a configuration to specify
> how long it should wait for a broker to respond to its RPCs before moving
> on. We also need to understand how the URM interacts with
> unclean.leader.election.enable=always. I assume that with "always" we will
> just unconditionally use the URM rather than choosing randomly. But this
> should be spelled out in the KIP.
>
> >
> > DescribeTopicRequest
> >
> >    1.
> >    Yes, the plan is to replace the MetadataRequest with the
> >    DescribeTopicRequest for the admin clients. Will check the details.
>
> Sounds good. But as I said, you need to specify how AdminClient interacts
> with the new request. This will involve adding some fields to
> TopicDescription.java. And you need to specify the changes to the
> kafka-topics.sh command line tool. Otherwise we cannot use the tool to see
> the new information.
>
> The new requests, DescribeTopicRequest and GetReplicaLogInfoRequest, need
> to have limits placed on them so that their size can't be infinite. We
> don't want to propagate the current problems of MetadataRequest, where
> clients can request massive responses that can mess up the JVM when handled.
>
> Adding limits is simple for GetReplicaLogInfoRequest -- we can just say
> that only 2000 partitions at a time can be requested. For
> DescribeTopicRequest we can probably just limit to 20 topics or something
> like that, to avoid the complexity of doing pagination in this KIP.
>
> >    2.
> >    I can let the broker load the ELR info so that they can serve the
> >    DescribeTopicRequest as well.
>
> Yes, it's fine to add to MetadataCache. In fact, you'll be loading it
> anyway once it's added to PartitionImage.
>
> >    3.
> >    Yeah, it does not make sense to have the topic id if
> >    DescribeTopicRequest is only used by the admin client.
>
> OK. That makes things simpler. We can always create a new API later
> (hopefully not in this KIP!) to query by topic ID.
>
> >
> >
> > Metrics
> >
> > As for overall cluster health metrics, I think under-min-ISR is still a
> > useful one. ELR is more like a safety belt. When the ELR is used, the
> > cluster availability has already been impacted.
> >
> > Maybe we can have a metric to count the partitions that sum(ISR, ELR) <
> min
> > ISR. What do you think?
>
> How about:
>
> A.  a metric for the totoal number of under-min-isr partitions? We don't
> have that in Apache Kafka at the moment.
>
> B. a metric for the number of unclean leader elections we did (for
> simplicity, it can reset to 0 on controller restart: we expect people to
> monitor the change over time anyway)
>
> best,
> Colin
>
>
> >
> > Yeah, for the ongoing unclean recoveries, the controller can keep an
> > accurate count through failover because partition registration can
> indicate
> > whether a recovery is needed. However, for the happened ones, unless we
> > want to persist the number somewhere, we can only figure it out from the
> > log.
> >
> > On Tue, Sep 12, 2023 at 3:16 PM Colin McCabe <cm...@apache.org> wrote:
> >
> >> Also, we should have metrics that show what is going on with regard to
> the
> >> eligible replica set. I'm not sure exactly what to suggest, but
> something
> >> that could identify when things are going wrong in the clsuter.
> >>
> >> For example, maybe a metric for partitions containing replicas that are
> >> ineligible to be leader? That would show a spike when a broker had an
> >> unclean restart.
> >>
> >> Ideally, we'd also have a metric that indicates when an unclear leader
> >> election or a recovery happened. It's a bit tricky because the simple
> >> thing, of tracking it per controller, may be a bit confusing during
> >> failovers.
> >>
> >> best,
> >> Colin
> >>
> >>
> >> On Tue, Sep 12, 2023, at 14:25, Colin McCabe wrote:
> >> > Hi Calvin,
> >> >
> >> > Thanks for the KIP. I think this is a great improvement.
> >> >
> >> >> Additional High Watermark advance requirement
> >> >
> >> > Typo: change "advance" to "advancement"
> >> >
> >> >> A bit recap of some key concepts.
> >> >
> >> > Typo: change "bit" to "quick"
> >> >
> >> >> Ack=1/all produce request. It defines when the Kafka server should
> >> respond to the produce request
> >> >
> >> > I think this section would be clearer if we talked about the new high
> >> > watermark advancement requirement first, and THEN talked about its
> >> > impact on acks=0, acks=1, and     acks=all.  acks=all is of course the
> >> > main case we care about here, so it would be good to lead with that,
> >> > rather than delving into the technicalities of acks=0/1 first.
> >> >
> >> >> Unclean recovery
> >> >
> >> > So, here you are introducing a new configuration,
> >> > unclean.recovery.strategy. The difficult thing here is that there is a
> >> > lot of overlap with unclean.leader.election.enable. So we have 3
> >> > different settings for unclean.recovery.strategy, plus 2 different
> >> > settings for unclean.leader.election.enable, giving a cross product of
> >> > 6 different options. The following "unclean recovery manager" section
> >> > only applies to one fo those 6 different possibilities (I think?)
> >> >
> >> > I simply don't think we need so many different election types. Really
> >> > the use-cases we need are people who want NO unclean elections, people
> >> > who want "the reasonable thing" and people who want avaialbility at
> all
> >> > costs.
> >> >
> >> > Overall, I feel like the first half of the KIP is about the ELR, and
> >> > the second half is about reworking unclean leader election. It might
> be
> >> > better to move that second half to a separate KIP so that we can
> figure
> >> > it out fully. It should be fine to punt this until later and just have
> >> > the current behavior on empty ELR be waiting for the last known leader
> >> > to return. After all, that's what we do today.
> >> >
> >> >> DescribeTopicRequest
> >> >
> >> > Is the intention for AdminClient to use this RPC for
> >> > Admin#describeTopics ? If so, we need to describe all of the changes
> to
> >> > the admin client API, as well as changes to command-line tools like
> >> > kafka-topics.sh (if there are any). For example, you will probably
> need
> >> > changes to TopicDescription.java. You will also need to provide all of
> >> > the things that admin client needs -- for example,
> >> > TopicAuthorizedOperations.
> >> >
> >> > I also don't think the controller should serve this request. We want
> to
> >> > minimize load on the controller. Just like with the other metadata
> >> > requests like MetadataRequest, this should be served by brokers.
> >> >
> >> > It's a bit confusing why both topic ID and topic name are provided to
> >> > this API. Is the intention that callers should set one but not the
> >> > other? Or both? This needs to be clarified. Also, if we do want to
> >> > support lookups by UUID, that is another thing that needs to be added
> >> > to adminclient.
> >> >
> >> > In general, I feel like this should also probably be its own KIP since
> >> > it's fairly complex
> >> >
> >> > best,
> >> > Colin
> >> >
> >> >
> >> > On Thu, Aug 10, 2023, at 15:46, Calvin Liu 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
> >>
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Colin McCabe <cm...@apache.org>.
On Tue, Sep 12, 2023, at 17:21, Calvin Liu wrote:
> Hi Colin
> Thanks for the comments!
>

Hi Calvin,

Thanks again for the KIP.

One meta-comment: it's usually better to just do a diff on a message spec file or java file if you're including changes to it in the KIP. This is easier to read than looking for "new fields begin" etc. in the text, and gracefully handles the case where existing fields were changed.

> Rewrite the Additional High Watermark advancement requirement
> There was feedback on this section that some readers may not be familiar
> with HWM and Ack=0,1,all requests. This can help them understand the
> proposal. I will rewrite this part for more readability.
>

To be clear, I wasn't suggesting dropping either section. I agree that they add useful background. I was just suggesting that we should discuss the "acks" setting AFTER discussing the new high watermark advancement conditions. We also should discuss acks=0. While it isn't conceptually much different than acks=1 here, its omission from this section is confusing.

> Unclean recovery
>
> The plan is to replace the unclean.leader.election.enable with
> unclean.recovery.strategy. If the Unclean Recovery is enabled then it deals
> with the three options in the unclean.recovery.strategy.
>
>
> Let’s refine the Unclean Recovery. We have already taken a lot of
> suggestions and I hope to enhance the durability of Kafka to the next level
> with this KIP.

I am OK with doing the unclean leader recovery improvements in this KIP. However, I think we need to really work on the configuration settings.

Configuration overrides are often quite messy. For example, the cases where we have log.roll.hours and log.roll.segment.ms, the user has to remember which one takes precedence, and it is not obvious. So, rather than creating a new configuration, why not add additional values to "unclean.leader.election.enable"? I think this will be simpler for people to understand, and simpler in the code as well.

What if we continued to use "unclean.leader.election.enable" but extended it so that it took a string? Then the string could have these values:

never
    never automatically do an unclean leader election under any conditions

false / default
    only do an unclean leader election if there may be possible data loss

true / always
    always do an unclean leader election if we can't immediately elect a leader

It's a bit awkward that false maps to default rather than to never. But this awkwardness exists if we use two different configuration keys as well. The reason for the awkwardness is that we simply don't want most of the people currently setting unclean.leader.election.enable=false to get the "never" behavior. We have to bite that bullet. Better to be clear and explicit than hide it.

Another thing that's a bit awkward is having two different ways to do unclean leader election specified in the KIP. You descirbe two methods: the simple "choose the last leader" method, and the "unclean recovery manager" method. I understand why you did it this way -- "choose the last leader" is simple, and will help us deliver an implementation quickly, while the URM is preferable in the long term. My suggestion here is to separate the decision of HOW to do unclean leader election from the decision of WHEN to do it.

So in other words, have "unclean.leader.election.enable" specify when we do unclean leader election, and have a new configuration like "unclean.recovery.manager.enable" to determine if we use the URM. Presumably the URM will take some time to get fully stable, so this can default to false for a while, and we can flip the default to true when we feel ready.

The URM is somewhat under-described here. I think we need a few configurations here for it. For example, we need a configuration to specify how long it should wait for a broker to respond to its RPCs before moving on. We also need to understand how the URM interacts with unclean.leader.election.enable=always. I assume that with "always" we will just unconditionally use the URM rather than choosing randomly. But this should be spelled out in the KIP.

>
> DescribeTopicRequest
>
>    1.
>    Yes, the plan is to replace the MetadataRequest with the
>    DescribeTopicRequest for the admin clients. Will check the details.

Sounds good. But as I said, you need to specify how AdminClient interacts with the new request. This will involve adding some fields to TopicDescription.java. And you need to specify the changes to the kafka-topics.sh command line tool. Otherwise we cannot use the tool to see the new information.

The new requests, DescribeTopicRequest and GetReplicaLogInfoRequest, need to have limits placed on them so that their size can't be infinite. We don't want to propagate the current problems of MetadataRequest, where clients can request massive responses that can mess up the JVM when handled.

Adding limits is simple for GetReplicaLogInfoRequest -- we can just say that only 2000 partitions at a time can be requested. For DescribeTopicRequest we can probably just limit to 20 topics or something like that, to avoid the complexity of doing pagination in this KIP.

>    2.
>    I can let the broker load the ELR info so that they can serve the
>    DescribeTopicRequest as well.

Yes, it's fine to add to MetadataCache. In fact, you'll be loading it anyway once it's added to PartitionImage.

>    3.
>    Yeah, it does not make sense to have the topic id if
>    DescribeTopicRequest is only used by the admin client.

OK. That makes things simpler. We can always create a new API later (hopefully not in this KIP!) to query by topic ID.

>
>
> Metrics
>
> As for overall cluster health metrics, I think under-min-ISR is still a
> useful one. ELR is more like a safety belt. When the ELR is used, the
> cluster availability has already been impacted.
>
> Maybe we can have a metric to count the partitions that sum(ISR, ELR) < min
> ISR. What do you think?

How about:

A.  a metric for the totoal number of under-min-isr partitions? We don't have that in Apache Kafka at the moment.

B. a metric for the number of unclean leader elections we did (for simplicity, it can reset to 0 on controller restart: we expect people to monitor the change over time anyway)

best,
Colin


>
> Yeah, for the ongoing unclean recoveries, the controller can keep an
> accurate count through failover because partition registration can indicate
> whether a recovery is needed. However, for the happened ones, unless we
> want to persist the number somewhere, we can only figure it out from the
> log.
>
> On Tue, Sep 12, 2023 at 3:16 PM Colin McCabe <cm...@apache.org> wrote:
>
>> Also, we should have metrics that show what is going on with regard to the
>> eligible replica set. I'm not sure exactly what to suggest, but something
>> that could identify when things are going wrong in the clsuter.
>>
>> For example, maybe a metric for partitions containing replicas that are
>> ineligible to be leader? That would show a spike when a broker had an
>> unclean restart.
>>
>> Ideally, we'd also have a metric that indicates when an unclear leader
>> election or a recovery happened. It's a bit tricky because the simple
>> thing, of tracking it per controller, may be a bit confusing during
>> failovers.
>>
>> best,
>> Colin
>>
>>
>> On Tue, Sep 12, 2023, at 14:25, Colin McCabe wrote:
>> > Hi Calvin,
>> >
>> > Thanks for the KIP. I think this is a great improvement.
>> >
>> >> Additional High Watermark advance requirement
>> >
>> > Typo: change "advance" to "advancement"
>> >
>> >> A bit recap of some key concepts.
>> >
>> > Typo: change "bit" to "quick"
>> >
>> >> Ack=1/all produce request. It defines when the Kafka server should
>> respond to the produce request
>> >
>> > I think this section would be clearer if we talked about the new high
>> > watermark advancement requirement first, and THEN talked about its
>> > impact on acks=0, acks=1, and     acks=all.  acks=all is of course the
>> > main case we care about here, so it would be good to lead with that,
>> > rather than delving into the technicalities of acks=0/1 first.
>> >
>> >> Unclean recovery
>> >
>> > So, here you are introducing a new configuration,
>> > unclean.recovery.strategy. The difficult thing here is that there is a
>> > lot of overlap with unclean.leader.election.enable. So we have 3
>> > different settings for unclean.recovery.strategy, plus 2 different
>> > settings for unclean.leader.election.enable, giving a cross product of
>> > 6 different options. The following "unclean recovery manager" section
>> > only applies to one fo those 6 different possibilities (I think?)
>> >
>> > I simply don't think we need so many different election types. Really
>> > the use-cases we need are people who want NO unclean elections, people
>> > who want "the reasonable thing" and people who want avaialbility at all
>> > costs.
>> >
>> > Overall, I feel like the first half of the KIP is about the ELR, and
>> > the second half is about reworking unclean leader election. It might be
>> > better to move that second half to a separate KIP so that we can figure
>> > it out fully. It should be fine to punt this until later and just have
>> > the current behavior on empty ELR be waiting for the last known leader
>> > to return. After all, that's what we do today.
>> >
>> >> DescribeTopicRequest
>> >
>> > Is the intention for AdminClient to use this RPC for
>> > Admin#describeTopics ? If so, we need to describe all of the changes to
>> > the admin client API, as well as changes to command-line tools like
>> > kafka-topics.sh (if there are any). For example, you will probably need
>> > changes to TopicDescription.java. You will also need to provide all of
>> > the things that admin client needs -- for example,
>> > TopicAuthorizedOperations.
>> >
>> > I also don't think the controller should serve this request. We want to
>> > minimize load on the controller. Just like with the other metadata
>> > requests like MetadataRequest, this should be served by brokers.
>> >
>> > It's a bit confusing why both topic ID and topic name are provided to
>> > this API. Is the intention that callers should set one but not the
>> > other? Or both? This needs to be clarified. Also, if we do want to
>> > support lookups by UUID, that is another thing that needs to be added
>> > to adminclient.
>> >
>> > In general, I feel like this should also probably be its own KIP since
>> > it's fairly complex
>> >
>> > best,
>> > Colin
>> >
>> >
>> > On Thu, Aug 10, 2023, at 15:46, Calvin Liu 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
>>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Calvin Liu <ca...@confluent.io.INVALID>.
Hi Colin
Thanks for the comments!

Rewrite the Additional High Watermark advancement requirement
There was feedback on this section that some readers may not be familiar
with HWM and Ack=0,1,all requests. This can help them understand the
proposal. I will rewrite this part for more readability.

Unclean recovery

The plan is to replace the unclean.leader.election.enable with
unclean.recovery.strategy. If the Unclean Recovery is enabled then it deals
with the three options in the unclean.recovery.strategy.

Let’s refine the Unclean Recovery. We have already taken a lot of
suggestions and I hope to enhance the durability of Kafka to the next level
with this KIP.

DescribeTopicRequest

   1.

   Yes, the plan is to replace the MetadataRequest with the
   DescribeTopicRequest for the admin clients. Will check the details.
   2.

   I can let the broker load the ELR info so that they can serve the
   DescribeTopicRequest as well.
   3.

   Yeah, it does not make sense to have the topic id if
   DescribeTopicRequest is only used by the admin client.


Metrics

As for overall cluster health metrics, I think under-min-ISR is still a
useful one. ELR is more like a safety belt. When the ELR is used, the
cluster availability has already been impacted.

Maybe we can have a metric to count the partitions that sum(ISR, ELR) < min
ISR. What do you think?

Yeah, for the ongoing unclean recoveries, the controller can keep an
accurate count through failover because partition registration can indicate
whether a recovery is needed. However, for the happened ones, unless we
want to persist the number somewhere, we can only figure it out from the
log.

On Tue, Sep 12, 2023 at 3:16 PM Colin McCabe <cm...@apache.org> wrote:

> Also, we should have metrics that show what is going on with regard to the
> eligible replica set. I'm not sure exactly what to suggest, but something
> that could identify when things are going wrong in the clsuter.
>
> For example, maybe a metric for partitions containing replicas that are
> ineligible to be leader? That would show a spike when a broker had an
> unclean restart.
>
> Ideally, we'd also have a metric that indicates when an unclear leader
> election or a recovery happened. It's a bit tricky because the simple
> thing, of tracking it per controller, may be a bit confusing during
> failovers.
>
> best,
> Colin
>
>
> On Tue, Sep 12, 2023, at 14:25, Colin McCabe wrote:
> > Hi Calvin,
> >
> > Thanks for the KIP. I think this is a great improvement.
> >
> >> Additional High Watermark advance requirement
> >
> > Typo: change "advance" to "advancement"
> >
> >> A bit recap of some key concepts.
> >
> > Typo: change "bit" to "quick"
> >
> >> Ack=1/all produce request. It defines when the Kafka server should
> respond to the produce request
> >
> > I think this section would be clearer if we talked about the new high
> > watermark advancement requirement first, and THEN talked about its
> > impact on acks=0, acks=1, and     acks=all.  acks=all is of course the
> > main case we care about here, so it would be good to lead with that,
> > rather than delving into the technicalities of acks=0/1 first.
> >
> >> Unclean recovery
> >
> > So, here you are introducing a new configuration,
> > unclean.recovery.strategy. The difficult thing here is that there is a
> > lot of overlap with unclean.leader.election.enable. So we have 3
> > different settings for unclean.recovery.strategy, plus 2 different
> > settings for unclean.leader.election.enable, giving a cross product of
> > 6 different options. The following "unclean recovery manager" section
> > only applies to one fo those 6 different possibilities (I think?)
> >
> > I simply don't think we need so many different election types. Really
> > the use-cases we need are people who want NO unclean elections, people
> > who want "the reasonable thing" and people who want avaialbility at all
> > costs.
> >
> > Overall, I feel like the first half of the KIP is about the ELR, and
> > the second half is about reworking unclean leader election. It might be
> > better to move that second half to a separate KIP so that we can figure
> > it out fully. It should be fine to punt this until later and just have
> > the current behavior on empty ELR be waiting for the last known leader
> > to return. After all, that's what we do today.
> >
> >> DescribeTopicRequest
> >
> > Is the intention for AdminClient to use this RPC for
> > Admin#describeTopics ? If so, we need to describe all of the changes to
> > the admin client API, as well as changes to command-line tools like
> > kafka-topics.sh (if there are any). For example, you will probably need
> > changes to TopicDescription.java. You will also need to provide all of
> > the things that admin client needs -- for example,
> > TopicAuthorizedOperations.
> >
> > I also don't think the controller should serve this request. We want to
> > minimize load on the controller. Just like with the other metadata
> > requests like MetadataRequest, this should be served by brokers.
> >
> > It's a bit confusing why both topic ID and topic name are provided to
> > this API. Is the intention that callers should set one but not the
> > other? Or both? This needs to be clarified. Also, if we do want to
> > support lookups by UUID, that is another thing that needs to be added
> > to adminclient.
> >
> > In general, I feel like this should also probably be its own KIP since
> > it's fairly complex
> >
> > best,
> > Colin
> >
> >
> > On Thu, Aug 10, 2023, at 15:46, Calvin Liu 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
>

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Colin McCabe <cm...@apache.org>.
Also, we should have metrics that show what is going on with regard to the eligible replica set. I'm not sure exactly what to suggest, but something that could identify when things are going wrong in the clsuter.

For example, maybe a metric for partitions containing replicas that are ineligible to be leader? That would show a spike when a broker had an unclean restart.

Ideally, we'd also have a metric that indicates when an unclear leader election or a recovery happened. It's a bit tricky because the simple thing, of tracking it per controller, may be a bit confusing during failovers.

best,
Colin


On Tue, Sep 12, 2023, at 14:25, Colin McCabe wrote:
> Hi Calvin,
>
> Thanks for the KIP. I think this is a great improvement.
>
>> Additional High Watermark advance requirement
>
> Typo: change "advance" to "advancement"
>
>> A bit recap of some key concepts.
>
> Typo: change "bit" to "quick"
>
>> Ack=1/all produce request. It defines when the Kafka server should respond to the produce request
>
> I think this section would be clearer if we talked about the new high 
> watermark advancement requirement first, and THEN talked about its 
> impact on acks=0, acks=1, and     acks=all.  acks=all is of course the 
> main case we care about here, so it would be good to lead with that, 
> rather than delving into the technicalities of acks=0/1 first.
>
>> Unclean recovery
>
> So, here you are introducing a new configuration, 
> unclean.recovery.strategy. The difficult thing here is that there is a 
> lot of overlap with unclean.leader.election.enable. So we have 3 
> different settings for unclean.recovery.strategy, plus 2 different 
> settings for unclean.leader.election.enable, giving a cross product of 
> 6 different options. The following "unclean recovery manager" section 
> only applies to one fo those 6 different possibilities (I think?)
>
> I simply don't think we need so many different election types. Really 
> the use-cases we need are people who want NO unclean elections, people 
> who want "the reasonable thing" and people who want avaialbility at all 
> costs.
>
> Overall, I feel like the first half of the KIP is about the ELR, and 
> the second half is about reworking unclean leader election. It might be 
> better to move that second half to a separate KIP so that we can figure 
> it out fully. It should be fine to punt this until later and just have 
> the current behavior on empty ELR be waiting for the last known leader 
> to return. After all, that's what we do today.
>
>> DescribeTopicRequest
>
> Is the intention for AdminClient to use this RPC for 
> Admin#describeTopics ? If so, we need to describe all of the changes to 
> the admin client API, as well as changes to command-line tools like 
> kafka-topics.sh (if there are any). For example, you will probably need 
> changes to TopicDescription.java. You will also need to provide all of 
> the things that admin client needs -- for example, 
> TopicAuthorizedOperations.
>
> I also don't think the controller should serve this request. We want to 
> minimize load on the controller. Just like with the other metadata 
> requests like MetadataRequest, this should be served by brokers.
>
> It's a bit confusing why both topic ID and topic name are provided to 
> this API. Is the intention that callers should set one but not the 
> other? Or both? This needs to be clarified. Also, if we do want to 
> support lookups by UUID, that is another thing that needs to be added 
> to adminclient.
>
> In general, I feel like this should also probably be its own KIP since 
> it's fairly complex
>
> best,
> Colin
>
>
> On Thu, Aug 10, 2023, at 15:46, Calvin Liu 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

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

Posted by Colin McCabe <cm...@apache.org>.
Hi Calvin,

Thanks for the KIP. I think this is a great improvement.

> Additional High Watermark advance requirement

Typo: change "advance" to "advancement"

> A bit recap of some key concepts.

Typo: change "bit" to "quick"

> Ack=1/all produce request. It defines when the Kafka server should respond to the produce request

I think this section would be clearer if we talked about the new high watermark advancement requirement first, and THEN talked about its impact on acks=0, acks=1, and     acks=all.  acks=all is of course the main case we care about here, so it would be good to lead with that, rather than delving into the technicalities of acks=0/1 first.

> Unclean recovery

So, here you are introducing a new configuration, unclean.recovery.strategy. The difficult thing here is that there is a lot of overlap with unclean.leader.election.enable. So we have 3 different settings for unclean.recovery.strategy, plus 2 different settings for unclean.leader.election.enable, giving a cross product of 6 different options. The following "unclean recovery manager" section only applies to one fo those 6 different possibilities (I think?)

I simply don't think we need so many different election types. Really the use-cases we need are people who want NO unclean elections, people who want "the reasonable thing" and people who want avaialbility at all costs.

Overall, I feel like the first half of the KIP is about the ELR, and the second half is about reworking unclean leader election. It might be better to move that second half to a separate KIP so that we can figure it out fully. It should be fine to punt this until later and just have the current behavior on empty ELR be waiting for the last known leader to return. After all, that's what we do today.

> DescribeTopicRequest

Is the intention for AdminClient to use this RPC for Admin#describeTopics ? If so, we need to describe all of the changes to the admin client API, as well as changes to command-line tools like kafka-topics.sh (if there are any). For example, you will probably need changes to TopicDescription.java. You will also need to provide all of the things that admin client needs -- for example, TopicAuthorizedOperations.

I also don't think the controller should serve this request. We want to minimize load on the controller. Just like with the other metadata requests like MetadataRequest, this should be served by brokers.

It's a bit confusing why both topic ID and topic name are provided to this API. Is the intention that callers should set one but not the other? Or both? This needs to be clarified. Also, if we do want to support lookups by UUID, that is another thing that needs to be added to adminclient.

In general, I feel like this should also probably be its own KIP since it's fairly complex

best,
Colin


On Thu, Aug 10, 2023, at 15:46, Calvin Liu 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