Jun,
I see what's going on -- the leader doesn't update its HW as soon as the
follower has requested the messages, it updates when the follower
requests the _next_ messages. I.e., it infers that because the follower
requested from offset 38 that everything <= 37 is durable.
This makes sense and
Hi, Mark,
So you did the manual leader election after the cluster is stabilized (i.e,
all replicas are in sync)? Then, it may not be related to the issue that I
described.
If there is just a single leadership change, what you described shouldn't
happen by design. I modified your steps to the foll
Jun,
Yeah, I probably have an off-by-one issue in the HW description. I
think you could pick any number here and the problem remains -- could
you read through the steps I posted and see if they logically make
sense, numbers aside?
We definitely lost data in 4 partitions of the 8,000 that were e
Hi, Mark,
Hmm, the committing of a message at offset X is the equivalent of saying
that the HW is at offset X + 1. So, in your example, if the producer
publishes a new message at offset 37, this message won't be committed
(i.e., HW moves to offset 38) until the leader sees the follower fetch from
Jun,
Thanks for the reply!
I am aware the HW won't advance until the in-sync replicas have
_requested_ the messages. However, I believe the issue is that the
leader has no guarantee the replicas have _received_ the fetch response.
There is no second-phase to the commit.
So, in the particular cas
Mark,
Thanks for reporting this. First, a clarification. The HW is actually never
advanced until all in-sync followers have fetched the corresponding
message. For example, in step 2, if all follower replicas issue a fetch
request at offset 10, it serves as an indication that all replicas have
rece
Correct, we've disabled unclean leader election. There were also no log
messages from an unclean election. I believe that Kafka thinks it
performed a clean election and still lost data.
--
Mark Smith
m...@qq.is
On Thu, Nov 17, 2016, at 06:23 PM, Tauzell, Dave wrote:
> Do you have:
>
> U
Do you have:
Unclean.leader.election.enable = false ?
Dave
> On Nov 17, 2016, at 19:39, Mark Smith wrote:
>
> Hey folks,
>
> I work at Dropbox and I was doing some maintenance yesterday and it
> looks like we lost some committed data during a preferred replica
> election. As far as I understand
Hey folks,
I work at Dropbox and I was doing some maintenance yesterday and it
looks like we lost some committed data during a preferred replica
election. As far as I understand this shouldn't happen, but I have a
theory and want to run it by ya'll.
Preamble:
* Kafka 0.9.0.1
* required.acks = -1