Re: [DISCUSS] Single broker failures causing offline partitions

2024-09-17 Thread Martin Dickson
I created https://issues.apache.org/jira/browse/KAFKA-17562 for this. Best, Martin On Wed, Sep 11, 2024 at 5:29 PM Martin Dickson wrote: > 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

Re: [DISCUSS] Single broker failures causing offline partitions

2024-09-11 Thread Martin Dickson
Hi Haruki and Kamal, It's very interesting to see how you're handling this Haruki. - For proactive broker replacement, we are doing something similar. We monitor certain metrics, mostly related to slow disks, and automatically replace the machines if those alerts fire but Kafka state is still safe

Re: [DISCUSS] Single broker failures causing offline partitions

2024-09-11 Thread Martin Dickson
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 wrote: > Hey Calvin and Martin, > > Makes sense. So KIP

Re: [DISCUSS] Single broker failures causing offline partitions

2024-09-11 Thread Haruki Okada
Hi Kamal, > Is the leader election automated to find the replica with the highest offset and latest epoch? > And, finding the eligible replica manually, will increase the outage mitigation time That's right, but this procedure is not yet automated in our operation. Because, we found that "choosin

Re: [DISCUSS] Single broker failures causing offline partitions

2024-09-11 Thread Kamal Chandraprakash
Hi Haruki, We are also interested in this issue. > The problem is how to identify such "eligible" replicas. Is the leader election automated to find the replica with the highest offset and latest epoch? If yes, could you please open a PR for it? When a broker goes down, it might be serving lead

Re: [DISCUSS] Single broker failures causing offline partitions

2024-09-10 Thread Calvin Liu
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

Re: [DISCUSS] Single broker failures causing offline partitions

2024-09-10 Thread Martin Dickson
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'

Re: [DISCUSS] Single broker failures causing offline partitions

2024-09-10 Thread Justine Olshan
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