-
Notifications
You must be signed in to change notification settings - Fork 3.2k
Description
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:
- PR: [KIP-580] Exponential Backoff with Mock Broker Changes to Automate Testing. #4422
- commit: 6dc7c71
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:
- librdkafka version (release number or git tag): latest (built from source)
- Apache Kafka version: Redpanda dev
- librdkafka client configuration:
enable.idempotence=true
- Operating system: ubuntu 20.04
- Provide logs (with
debug=..
as necessary) from librdkafka: debug=topic,broker (msg is too much data): https://gist.githubusercontent.com/StephanDollberg/40363a2567a98c0ad1469994df964412/raw/072343ca4c6fceb4a1c30876a951bad736df8b06/rdkafka_0_0_0.log - Provide broker log excerpts: Nothing interesting in the broker logs
- Critical issue