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 <j...@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 <j...@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.ja...@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 > <j...@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 > > > > > > > <jols...@confluent.io.invalid > > > > > > > > > > > > > > > > > wrote: > > > > > > > > > > > > > > > > > Sorry -- not MV but software version. > > > > > > > > > > > > > > > > > > On Wed, Oct 4, 2023 at 9:51 AM Justine Olshan < > > > > > jols...@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 > > > > <j...@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 > > > > > > > > > >> > <alivsh...@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 < > > > > > > cmcc...@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 > > > > > > > > > <j...@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 > < > > > > > > > > > >> cmcc...@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 < > > > > > > > > > >> > > > cmcc...@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 < > > > > > > > > > >> > > > >> cmcc...@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 < > > > > > > > > > >> > > > >> > > cmcc...@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 < > > > > > > > > > >> > > > >> > > cmcc...@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 > > > > > > > > > > > > > > > > > > > > >