Hi, let me share some of my experience about this: - Monitoring: * request-handler idle ratio is an important metric to check the ability of a broker to handle requests, so setting alert for this might be a good idea * of course, alerting on UnderMinISR is important because if broker lost the ability to handle fetch requests from followers, the broker might make followers out-of-sync * also, in our environment, typical cause of half-dead broker happens when there's broke disk so we monitor disk health as well - Tools: * LinkedIn's CruiseControl might be a good option (we use CruiseControl for self-healing from broker failure and it works quite well. Though we don't use it for healing half-dead broker, it provides many customization point so you may use it) - Lessons learned: * the possible worst situation is half-dead broker makes some partitions UnderMinISR, which means producers can't produce messages anymore (assuming acks=all and min.insync.replicas > 1), and stopping the broker makes the partition offline. - In this case, we might need to stop the broker once accepting temporary offline, and perform unclean leader election. The thing we should care here is to elect the last replica which was caught-up to the (half-dead) leader before it becomes out-of-sync. If so, we can expect no message loss even with unclean leader election. * We had to do this manually before, but from 4.0.0, we can just use unclean leader election thanks to https://cwiki.apache.org/confluence/display/KAFKA/KIP-966%3A+Eligible+Leader+Replicas in my understanding
2025年6月26日(木) 9:27 Mohamed Saliha A <a.mohamedsalih...@gmail.com>: > Hi all, > > Just following up on this thread. > > I would appreciate any insights or guidance the community can share on this > topic whenever you have a chance. > > Thank you very much for your time and support. > > Best regards, > > Saliha > > > On Mon, 23 Jun 2025 at 12:19 PM, Mohamed Saliha A < > a.mohamedsalih...@gmail.com> wrote: > > > Hello Kafka Community, > > > > I’d like to consult the community on best practices for handling and > > preventing what’s sometimes called a "half-dead" Kafka broker scenario > in a > > self-hosted OSS Kafka environment. > > > > Specifically, I’m referring to situations where a broker appears healthy > > from a cluster perspective (i.e., still part of the ISR) but is no longer > > able to properly serve traffic, causing disruption to producers or > > consumers. > > > > I understand that some managed services like AWS MSK implement additional > > mechanisms (e.g., their "healing" state) to detect and handle such > brokers, > > but I’d like to know how self-hosted OSS Kafka operators typically manage > > this risk. > > > > Some key questions: > > > > - > > > > Are there recommended monitoring patterns to detect a "half-dead" > > broker more proactively? > > - > > > > Are there any community-recommended configurations, scripts, or tools > > to automatically remove or restart such brokers? > > - > > > > Any lessons learned or operational best practices from other > > self-hosted users? > > > > I would greatly appreciate any guidance ,Thank you in advance! > > > > Best regards, > > > > Saliha > > > -- ======================== Okada Haruki ocadar...@gmail.com ========================