kafka-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Jiangjie Qin (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (KAFKA-2143) Replicas get ahead of leader and fail
Date Sat, 01 Aug 2015 04:09:04 GMT

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

Jiangjie Qin commented on KAFKA-2143:
-------------------------------------

Hmm, with current design when the follower got OffsetOutOfRangeException during fetch, and
if *leader's log end offset is smaller than follower's log end offset*, follower will not
truncate to zero but the current leader's log end offsets. That means in step (5), broker
A should start to fetch from offset 3000 instead of drop to 0 unless broker B really have
nothing.

However, if the follower's log end offsets is smaller than the leader's log end offsets but
we received offset out of range exception. We assume it was because the follower's log end
offsets is too small and behind the earliest available offset of leader. But it might not
necessarily be true, because we are fetching the latest offset again after we receive the
exception. So it is possible that when the exception was returned, follower was ahead of the
leader, but when it tries to handle the exception, since the new leader has got some new messages
appended, the follower is now behind the leader. In that case, the follower will truncate
its log to the log start offset of leader, which is the problem you described.

I think what happened is as below:

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 fetch from Broker B and got an OffsetOutOfRangeException.
4.1) Broker B got some new message and its log end offset become 6000.
4.2) Broker A tries to handle the Exception so it checks the log end offset on broker B and
found it is greater than broker A's log end offset so it truncate itself to the starting offset
of Broker B.

And the rest is pretty what you described.

By design at step (3) when Broker A become follower, it will truncate its log to its high
watermark, which should be 3000. There is a possible related issue KAFKA-2334, which might
make 3000 not the real high watermark on Broker B. And that issue triggered the offset out
of range in step (4).

So it looks to me KAFKA-2334 and the ReplicaFetcherThread bug together caused the issue. I'll
take both of the issues and see if they can be solved together.

> 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)

Mime
View raw message