You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Rafael Bagmanov (JIRA)" <ji...@apache.org> on 2015/07/31 01:03:05 UTC

[jira] [Commented] (KAFKA-2143) Replicas get ahead of leader and fail

    [ https://issues.apache.org/jira/browse/KAFKA-2143?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14648444#comment-14648444 ] 

Rafael Bagmanov commented on KAFKA-2143:
----------------------------------------

I would like to add that we are experiencing this issue quite often. 
And the situation get worse when we have configs like this

replication factor: 2
unclean.leader.election.enable: true 

This leads to loss of a whole partition in case of two network glitches happening in a row. 

The failure scenario looks like this:
we have replica.lag.max.messages: 4000 so I'm guessing follower is always little bit behind leader for high volume topic (which means leader has a larger offset value than the follower)

1) Broker A (leader) has committed offset up-to 5000 
2) Broker B (follower) has committed offset up to 3000 (he is still in ISR because of  replica.lag.max.messages)
***network glitch happens***
3) Broker B becomes a leader, Broker A becomes a follower
4) Broker A (follower) asks leader for messages starting from 5000 
5) Broker A (follower)  receives message that this is invalid offset (Broker B has only 3000) and drops partition to 0
***network glitch happens***
6) Broker A becomes a leader (unclean election), Broker B becomes a follower
7) Broker B (follower) ask leader for messages starting from 3000
8) Broker B (follower) receives message that this is invalid offset (Broker A has only 0) and drops partition to 0

As a result we lost partition because of 2 network glitches. 

And if the configs are 
replication factor: 2
unclean.leader.election.enable: false (!)

the scenario repeats up to 5th step, but then Broker A got kicked out of ISR  and unclean election is not happening. 

> Replicas get ahead of leader and fail
> -------------------------------------
>
>                 Key: KAFKA-2143
>                 URL: https://issues.apache.org/jira/browse/KAFKA-2143
>             Project: Kafka
>          Issue Type: Bug
>          Components: replication
>    Affects Versions: 0.8.2.1
>            Reporter: Evan Huus
>
> On a cluster of 6 nodes, we recently saw a case where a single under-replicated partition suddenly appeared, replication lag spiked, and network IO spiked. The cluster appeared to recover eventually on its own,
> Looking at the logs, the thing which failed was partition 7 of the topic {{background_queue}}. It had an ISR of 1,4,3 and its leader at the time was 3. Here are the interesting log lines:
> On node 3 (the leader):
> {noformat}
> [2015-04-23 16:50:05,879] ERROR [Replica Manager on Broker 3]: Error when processing fetch request for partition [background_queue,7] offset 3722949957 from follower with correlation id 148185816. Possible cause: Request for offset 3722949957 but we only have log segments in the range 3648049863 to 3722949955. (kafka.server.ReplicaManager)
> [2015-04-23 16:50:05,879] ERROR [Replica Manager on Broker 3]: Error when processing fetch request for partition [background_queue,7] offset 3722949957 from follower with correlation id 156007054. Possible cause: Request for offset 3722949957 but we only have log segments in the range 3648049863 to 3722949955. (kafka.server.ReplicaManager)
> [2015-04-23 16:50:13,960] INFO Partition [background_queue,7] on broker 3: Shrinking ISR for partition [background_queue,7] from 1,4,3 to 3 (kafka.cluster.Partition)
> {noformat}
> Note that both replicas suddenly asked for an offset *ahead* of the available offsets.
> And on nodes 1 and 4 (the replicas) many occurrences of the following:
> {noformat}
> [2015-04-23 16:50:05,935] INFO Scheduling log segment 3648049863 for log background_queue-7 for deletion. (kafka.log.Log) (edited)
> {noformat}
> Based on my reading, this looks like the replicas somehow got *ahead* of the leader, asked for an invalid offset, got confused, and re-replicated the entire topic from scratch to recover (this matches our network graphs, which show 3 sending a bunch of data to 1 and 4).
> Taking a stab in the dark at the cause, there appears to be a race condition where replicas can receive a new offset before the leader has committed it and is ready to replicate?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)