librdkafka icon indicating copy to clipboard operation
librdkafka copied to clipboard

Fix: Avoid unnecessary producer epoch bumps

Open marcin-krystianc opened this issue 10 months ago • 6 comments

Fixes: #4953

When the producer is idempotent and max.in.flight.requests.per.connection is set to a value between 2 and 5, it's normal to receive OUT_OF_ORDER_SEQUENCE_NUMBER produce responses for requests R2 through R5 when the R1 failed for any other reason.

Bumping the producer epoch in this scenario violates the "exactly-once" guarantees. Therefore, we believe that it's unnecessary to bump the producer's epoch; re-enqueuing the messages is sufficient.

The same "retry, but don't bump producer epoch" behavior is implemented in the Java client: https://github.com/apache/kafka/blob/a6a588fbed9982598377060c63f94ee6184b4295/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java#L1015-L1016

marcin-krystianc avatar Feb 19 '25 12:02 marcin-krystianc

:tada: All Contributor License Agreements have been signed. Ready to merge.
:white_check_mark: marcin-krystianc
Please push an empty commit if you would like to re-run the checks to verify CLA status for all contributors.

Thanks for the contribution @marcin-krystianc I just need to change a test that is expecting a different behaviour. It's to note that the exactly-once guarantees are only partial for the idempotent producer, see this comment.

emasab avatar May 16 '25 15:05 emasab

/sem-approve

emasab avatar May 16 '25 16:05 emasab

/sem-approve

emasab avatar May 16 '25 21:05 emasab

Thanks for the contribution @marcin-krystianc I just need to change a test that is expecting a different behaviour. It's to note that the exactly-once guarantees are only partial for the idempotent producer, see this comment.

Thanks. Are there any other tests worth adding? I'm happy to do it, but I need some guidance.

marcin-krystianc avatar May 19 '25 10:05 marcin-krystianc

@marcin-krystianc at the moment I can't think of other tests related to this error that are different from the existing ones. In theory it has a meaning and in practice it's done by Java client as well so it seems like a safer behaviour. It doesn't happen with the transactional producer as the transaction is aborted with the code before this change. I request a second review as well.

btw, if you remove the sleeps in test 0144 it fails with a fatal error (r=0). I verified that it's because of the mocked error codes that should be returned after the error caused by the disconnection and not because of some problem with the new logic. Anyway we have some other soak and transactional tests that we'll run against real brokers.

Thanks again!

emasab avatar May 19 '25 14:05 emasab

Hi @emasab , do you have a rough timeline for when you anticipate being able to merge this change?

marcin-krystianc avatar Aug 05 '25 15:08 marcin-krystianc