KAFKA-19283: Update transaction exception handling documentation (#19931)

Added docs on Enhancements to transactional producer error handling:

* Added standardized exception categories (`RetriableException`,
`RefreshRetriableException`, `AbortableException`,
`ApplicationRecoverableException`, `InvalidConfigurationException`,
`KafkaException`) to ensure clearer error handling patterns.
* Included a link to example template code for handling transaction
exceptions: [Transaction Client

Demo](https://github.com/apache/kafka/blob/trunk/examples/src/main/java/kafka/examples/TransactionalClientDemo.java).

Reviewers: Justine Olshan <jolshan@confluent.io>
This commit is contained in:
Kaushik Raina 2025-06-10 05:37:29 +05:30 committed by GitHub
parent b420e4092e
commit dbfda79951
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
1 changed files with 12 additions and 0 deletions

View File

@ -340,6 +340,18 @@
transactions. However, in the event of a transaction abort, the application's state and in particular the current position of the consumer must be reset explicitly so that it can
reprocess the records processed by the aborted transaction.
<p>
The error handling for transactional producer has been standardized which ensures consistent behavior and clearer error handling patterns. The exception categories are now more precisely defined:
<ol>
<li><strong>RetriableException</strong>: Temporary exceptions that are retried automatically by the client. These are handled internally and don't bubble up to the application.</li>
<li><strong>RefreshRetriableException</strong>: Exceptions requiring metadata refresh before retry. These are handled internally by the client after refreshing metadata and don't bubble up to the application.</li>
<li><strong>AbortableException</strong>: Exceptions that require transaction abort and reprocessing. These bubble up to the application, which must handle them by aborting the transaction and resetting the consumer position.</li>
<li><strong>ApplicationRecoverableException</strong>: Exceptions that bubble up to the application and require application handling. The application must implement its own recovery strategy, which must include restarting the producer.</li>
<li><strong>InvalidConfigurationException</strong>: Configuration-related exceptions that bubble up to the application and require application handling. The producer doesn't need to restart, but the application may choose to restart it.</li>
<li><strong>KafkaException</strong>: General Kafka exceptions that don't fit into the above categories. These bubble up to the application for handling.</li>
</ol>
<p>
Example template code for handling transaction exceptions link : <a href="https://github.com/apache/kafka/blob/trunk/examples/src/main/java/kafka/examples/TransactionalClientDemo.java"> Transaction Client Demo</a>
<p>
A simple policy for handling exceptions and aborted transactions is to discard and recreate the Kafka producer and consumer objects and start afresh. As part of recreating the consumer, the consumer
group will rebalance and fetch the last committed offset, which has the effect of rewinding back to the state before the transaction aborted. Alternatively, a more sophisticated application (such as the
transactional message copier) can choose not to use <code>KafkaConsumer.committed</code> to retrieve the committed offset from Kafka, and then <code>KafkaConsumer.seek</code> to rewind the current position.