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 <j...@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 >> > <alivsh...@confluent.io.invalid> wrote: >> > >> > > Hi Calvin, >> > > >> > > Thanks for the KIP. The new ELR protocol looks good to me. I have >> some >> > > questions about unclean recovery, specifically in "balanced" mode: >> > > >> > > 1. The KIP mentions that the controller would trigger unclear recovery >> > when >> > > the leader is fenced, but my understanding is that when a leader is >> > fenced, >> > > it would get into ELR. Would it be more precise to say that an >> unclear >> > > leader election is triggered when the last member of ELR gets unfenced >> > and >> > > registers with unclean shutdown? >> > > 2. For balanced mode, we need replies from at least LastKnownELR, in >> > which >> > > case, does it make sense to start unclean recovery if some of the >> > > LastKnownELR are fenced? >> > > 3. "The URM takes the partition info to initiate an unclear recovery >> task >> > > ..." the parameters are topic-partition and replica ids -- what are >> > those? >> > > Would those be just the whole replica assignment or just LastKnownELR? >> > > >> > > -Artem >> > > >> > > On Thu, Aug 10, 2023 at 3:47 PM Calvin Liu <ca...@confluent.io.invalid >> > >> > > wrote: >> > > >> > > > Hi everyone, >> > > > I'd like to discuss a series of enhancement to the replication >> > protocol. >> > > > >> > > > A partition replica can experience local data loss in unclean >> shutdown >> > > > scenarios where unflushed data in the OS page cache is lost - such >> as >> > an >> > > > availability zone power outage or a server error. The Kafka >> replication >> > > > protocol is designed to handle these situations by removing such >> > replicas >> > > > from the ISR and only re-adding them once they have caught up and >> > > therefore >> > > > recovered any lost data. This prevents replicas that lost an >> arbitrary >> > > log >> > > > suffix, which included committed data, from being elected leader. >> > > > However, there is a "last replica standing" state which when >> combined >> > > with >> > > > a data loss unclean shutdown event can turn a local data loss >> scenario >> > > into >> > > > a global data loss scenario, i.e., committed data can be removed >> from >> > all >> > > > replicas. When the last replica in the ISR experiences an unclean >> > > shutdown >> > > > and loses committed data, it will be reelected leader after >> starting up >> > > > again, causing rejoining followers to truncate their logs and >> thereby >> > > > removing the last copies of the committed records which the leader >> lost >> > > > initially. >> > > > >> > > > The new KIP will maximize the protection and provides MinISR-1 >> > tolerance >> > > to >> > > > data loss unclean shutdown events. >> > > > >> > > > >> > > > >> > > >> > >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-966%3A+Eligible+Leader+Replicas >> > > > >> > > >> > >> >