> but it’s not directly tied to a quorum mechanism Correct.
> this doesn’t really differ from having only three brokers Also correct. However, practically, we often reassign failed replicas to another broker (e.g. using CruiseControl) in case of broker crash so having 3 + 1 brokers is more safe. > is there a chance that replicas might end up continuously requesting metadata from each other in a loop What "metadata" mean here? KRaft metadata? I don't fully get what scenario is on your mind yet. 2025年1月7日(火) 6:56 Ilya Starchenko <st.ilya....@gmail.com>: > Thank you very much, Haruki! > > This really clarifies things for me. I just want to confirm one more point: > it seems that a majority quorum in Kafka doesn’t actually play a role in > leader election (it might make the cluster more reliable overall, but it’s > not directly tied to a quorum mechanism). Am I correct? > > For example, I have a cluster with four brokers, where replication.factor=3 > and min.insync.replicas=2. If the partition leader fails, leadership simply > transfers to another replica from isr-set. In practice, this doesn’t really > differ from having only three brokers (because the remaining two replicas > can’t form a classic “quorum” anyway). Is that right? > > Also, is there a chance that replicas might end up continuously requesting > metadata from each other in a loop, potentially causing an > OutOfMemoryError? Have you encountered similar situations in large > clusters? > > Best regards, > Ilya Starchenko > > On 6 Jan 2025 at 10:49:14, Haruki Okada <ocadar...@gmail.com> wrote: > > > Hi Ilya, > > > > Does it require some sort of majoritary quorum to decide whose offsets > > > > are valid > > > > No. Controller just elects a new leader from ISR (in current Kafka > version, > > the leftmost ISR will be chosen though) without taking offsets into > > account. > > After an ISR becomes a new leader, other replicas truncate the offsets to > > the new leader's LEO so the log divergence doesn't happen. > > > > Also, producer's acks setting plays an important role here. > > As long as acks is set to all, produce requests don't return successful > > response until all ISRs replicate the message. So messages never lost for > > acks=all producers even when electing any leader without taking offset > into > > account. > > > > 2025年1月6日(月) 6:41 Ilya Starchenko <st.ilya....@gmail.com>: > > > > Hello, > > > > > > I’m trying to understand how replication and leader election work in > Kafka, > > > > and I have a couple of questions. > > > > > > I know that Kafka does not rely on a majority-quorum approach. Instead, > > > > there is an ISR set, which consists of replicas that the current leader > > > > considers sufficiently up-to-date (this is partially managed through > > > > replica.lag.time.max.ms). The leader election algorithm itself ( > > > > > > > > > https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/controller/PartitionStateMachine.scala#L549 > > > > ) > > > > is quite straightforward. > > > > > > As long as the leader has not “kicked out” a replica from the ISR due to > > > > excessive lag, that replica can become the new leader if the current > leader > > > > failsa and control plane is aware of which replicas belong to the ISR. > > > > > > Here’s the scenario I’m confused about: Suppose the leader(Broker #1) > dies > > > > before it can update the ISR to remove lagging replicas. Let’s say: > > > > > > - Broker #2 fully fetched the most recent messages from the leader, > > > > - Broker #3 is still lagging, > > > > - Broker #4 only pulled metadata. > > > > > > From the control plane’s perspective, since the leader did not remove > them > > > > from the ISR, all these replicas appear up-to-date enough to take over as > > > > leader—even though they each have different states. > > > > > > I know Kafka has a Replication Reconciliation process, but I’m not > entirely > > > > clear on how it determines the “source of truth” for offsets. Does it > > > > require some sort of majoritary quorum to decide whose offsets are valid > > > > (via offset metadata polling from other replicas)? Or does the replica > with > > > > the most recent offsets (and a certain epoch) automatically become the > new > > > > leader, forcing the others to reconcile by removing any records that > don’t > > > > align with the new leader’s log—even if a “majority” of brokers have > those > > > > records? > > > > > > I’d really appreciate any clarification on how this scenario is handled, > as > > > > well as any relevant documentation or code references. Thanks! > > > > > > > > > > -- > > ======================== > > Okada Haruki > > ocadar...@gmail.com > > ======================== > > > -- ======================== Okada Haruki ocadar...@gmail.com ========================