confluentinc / librdkafka

The Apache Kafka C/C++ library
Other
7.36k stars 3.11k forks source link

Idempotent producer bug: "Local: Inconsistent state: Unable to reconstruct MessageSet" #4736

Closed StephanDollberg closed 2 weeks ago

StephanDollberg commented 3 weeks ago

Description

Hi,

I am using rdkafka_performance for some load tests against a redpanda cluster and running into the following idempotent producer bug:

% FATAL ERROR CALLBACK: rdkafka#producer-1: Local: Inconsistent state: Unable to reconstruct MessageSet (currently with 9 message(s)) with msgid range 86821..86835: last message added has msgid 86829: unable to guarantee consistency
%0|1716987609.390|FATAL|rdkafka#producer-1| [thrd:172.31.7.143:9093/bootstrap]: Fatal error: Local: Inconsistent state: Unable to reconstruct MessageSet (currently with 9 message(s)) with msgid range 86821..86835: last message added has msgid 86829: unable to guarantee consistency
%4|1716987609.390|TERMINATE|rdkafka#producer-1| [thrd:app]: Producer terminating with 100000 messages (102400000 bytes) still in queue or transit: use flush() to wait for outstanding message delivery

As per the comment here: https://github.com/confluentinc/librdkafka/blob/master/src/rdkafka_msgset_writer.c#L912 this seems to indicate a client bug.

How to reproduce

Unfortunately I don't have an easy reproducer but effectively I am running 10 parallel rdkafka_performance instances like so (it seems to require some broker load to trigger) :

rdkafka_performance -P -c '10000000' -s '1024' -t foo_topic -b IP:PORT -X enable.idempotence=true -X max.in.flight=5 -X acks=all -X linger.ms=1 -X batch.size=16384

Note that only enable.idempotence=true is really required. The other options can be omitted in my case as well.

I have bisected the issue back to this commit introduced between v2.2.0 and v2.3.0:

Can't immediately see what's wrong with that. Possibly it just exposes a bug that was already existing. Note 2.3.0 also introduced another suspiciously looking commit but that doesn't seem to be the issue.

From the logs (see below) it seems to get triggered by a "Broker: Not leader for partition" request error.

Checklist

IMPORTANT: We will close issues where the checklist has not been completed.

Please provide the following information:

emasab commented 3 weeks ago

Hi @StephanDollberg , thanks for the report, can you reproduce it with AK too?

Does it happen after that fix in 2.3.0? The log has version 2.2.0

StephanDollberg commented 3 weeks ago

Hi @StephanDollberg , thanks for the report, can you reproduce it with AK too?

I haven't tried that yet.

Does it happen after that fix in 2.3.0? The log has version 2.2.0

Sorry for not being clear, the issue still persist on latest master. It got introduced in the commit pointed out above -https://github.com/confluentinc/librdkafka/commit/6dc7c71e6a4f1895fa4a95b25769840702cec79d (which in git speak is 2.2+something but the first released version is 2.3.0)

emasab commented 3 weeks ago

I think it can happen in some particular cases because of this condition

https://github.com/confluentinc/librdkafka/blob/2587cac70f83fced42c51f921bed325a434f5bc7/src/rdkafka_msgset_writer.c#L877

and the backoff that is not uniform across all messages of the same idempotent batch here:

https://github.com/confluentinc/librdkafka/blob/6dc7c71e6a4f1895fa4a95b25769840702cec79d/src/rdkafka_partition.c#L925

I'm going to check about reproducing and fixing this case