Hi Justine and Calvin, Thank you both for the feedback, I agree that KIP-966 should solve #2. I'll file a JIRA ticket for #1 (currently in the process of registering my account).
Best, Martin On Tue, Sep 10, 2024 at 9:36 PM Justine Olshan <jols...@confluent.io.invalid> wrote: > Hey Calvin and Martin, > > Makes sense. So KIP-966 can help with 2, but 1 (some mechanism to identify > the issue) is still required. > If you haven't already filed a JIRA ticket for this, do you mind doing so? > I think it makes sense to close this gap. > > Justine > > On Tue, Sep 10, 2024 at 1:15 PM Calvin Liu <ca...@confluent.io.invalid> > wrote: > > > Hi Martin. > > Yes, the KIP-966 does not resolve your concern for the degraded leader. > To > > fill the gap, > > 1. As you have mentioned, we need a leader degradation detection > mechanism. > > So that the controller can promote another replica to the leader. > > 2. The controller needs to know which replica is a valid candidate to be > > the leader. ELR could be a good candidate(KIP-966 currently targets 4.0). > > It is a very interesting problem, probably the community can pick this up > > after 4.0. > > > > Calvin > > > > On Tue, Sep 10, 2024 at 9:37 AM Martin Dickson > > <martin.dick...@datadoghq.com.invalid> wrote: > > > > > Hi Justine, > > > > > > Indeed we are very much interested in the improvements that will come > > from > > > KIP-966! > > > > > > However I think there is still a gap regarding the failure detection of > > the > > > leader. Please correct me if this is wrong but my understanding is that > > > with KIP-966 we'll stop advancing the HWM minISR isn't satisfied, and > > will > > > be able to fail-over leadership to any member of ELR following complete > > > failure of the leader. This gives good options for recovery following a > > > complete failure, however, if the leader remains degraded but doesn't > > > completely fail then the partition will stay online but will still be > > > unavailable for writes. (So probably a better subject for my email > would > > > have been "Single degraded brokers causing unavailable partitions", > which > > > is an issue which I think remains post KIP-966?) > > > > > > Best, > > > Martin > > > > > > > > > On Tue, Sep 10, 2024 at 5:14 PM Justine Olshan > > > <jols...@confluent.io.invalid> > > > wrote: > > > > > > > Hey Martin. > > > > > > > > I recommend you take a look at KIP-966. I think can help the use case > > you > > > > are describing. > > > > The KIP talks about failure scenarios, but I believe it will also > help > > > when > > > > the leader has issues and kicks its followers out of ISR. > > > > The goal is to better handle the "last replica standing" issue > > > > > > > > > > > > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-966%3A+Eligible+Leader+Replicas > > > > > > > > Let me know if it helps, > > > > > > > > Justine > > > > > > > > On Tue, Sep 10, 2024 at 9:00 AM Martin Dickson > > > > <martin.dick...@datadoghq.com.invalid> wrote: > > > > > > > > > Hi all, > > > > > > > > > > We have a recurring issue with single broker failures causing > offline > > > > > partitions. The issue is that when a leader is degraded, follower > > > fetches > > > > > can fail to happen in a timely manner, and all followers can fall > out > > > of > > > > > sync. If that leader then later fails then the partition will go > > > offline, > > > > > but even if it remains only partially failed then applications > might > > > > still > > > > > be impacted (for example, if the producer is using acks=all and > > > > > min.insync.replicas=2). This can all happen because of a problem > > solely > > > > > with the leader, and hence a single broker failure can cause > > > > > unavailability, even if RF=3 or higher. > > > > > > > > > > We’ve seen the issue with various kinds of failures, mostly related > > to > > > > > failing disks, e.g. due to pressure on request handler threads as a > > > > result > > > > > of produce requests waiting on a slow disk. But the easiest way for > > us > > > to > > > > > reproduce it is at the outgoing network level: Setting up a cluster > > > with > > > > > moderate levels of ingoing throughput then injecting 50% outgoing > > > packet > > > > > drop on a single broker is enough to cause the partitions to cause > > > > follower > > > > > requests to be slow and replication to lag, but not enough for that > > > > broker > > > > > to lose its connection to ZK. This triggers the degraded broker to > > > become > > > > > the only member of ISR. > > > > > > > > > > We have replica.lag.time.max.ms=10000 and > > zookeeper.session.timeout.ms > > > > > =6000 > > > > > (the pre-KIP-537 values, 1/3 of the current defaults, to control > > > produce > > > > > latency when a follower is failing). We are also able to reproduce > > the > > > > > issue in the same way on a KRaft cluster with the KRaft defaults. > > (Note > > > > > that we are not very experienced with operating KRaft as we aren’t > > > > running > > > > > it in production yet.) > > > > > > > > > > The last KIP I saw regarding this was KIP-501 > > > > > < > > > > > > > > > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-501+Avoid+out-of-sync+or+offline+partitions+when+follower+fetch+requests+are+not+processed+in+time > > > > > >, > > > > > which describes this exact problem. The proposed solution there was > > in > > > > the > > > > > first part to introduce a notion of pending requests, and second > part > > > to > > > > > relinquish leadership if pending requests are taking too long. The > > > > > discussion > > > > > thread < > > > https://lists.apache.org/thread/1kbs68dq60p31frpfsr3x1vcqlzjf60x > > > > > > > > > > for that doesn’t come to a conclusion. However it is pointed out > that > > > not > > > > > all failure modes would be solved by the pending requests approach, > > and > > > > > that whilst relinquishing leadership seems ideal there are concerns > > > about > > > > > this thrashing in certain failure modes. > > > > > > > > > > We are experimenting with a variation on KIP-501 where we add a > > > heuristic > > > > > for brokers failing this way: if the leader for a partition has > > removed > > > > > many followers from ISR in a short period of time (including the > case > > > > when > > > > > it sends a single AlterPartition request removing all followers > from > > > ISR > > > > > and thereby shrinking ISR only to itself), have the controller > ignore > > > > this > > > > > request and instead choose one of the followers to become leader. > To > > > > avoid > > > > > thrashing, rate-limit how often the controller can do this per > > > > > topic-partition. We have tested that this fixes our repro, but have > > not > > > > > productionised it (see rough draft PR > > > > > <https://github.com/DataDog/kafka/pull/15/files>). We have only > > > > > implemented > > > > > ZK-mode so far. We implemented this on the controller side out of > > > > > convenience (no API changes), but potentially the demotion decision > > > > should > > > > > be taken at the broker level instead, which should also be > possible. > > > > > > > > > > Whilst the code change is small, the proposed solution we’re > > > > investigating > > > > > isn’t very clean and we’re not totally satisfied with it. We wanted > > to > > > > get > > > > > some ideas from the community on: > > > > > 1. How are other folks handling this class of issues? > > > > > 2. Is there any interest in adding more comprehensive failing > > > > > broker detection to Kafka (particularly how this could look in > > KRaft)? > > > > > 3. Is there any interest in having a heuristic failure detection > like > > > the > > > > > one described above? > > > > > > > > > > Thanks, > > > > > Martin > > > > > > > > > > > > > > >