From dbfda79951d136a6fbf0076b42d55e42aea9bf84 Mon Sep 17 00:00:00 2001 From: Kaushik Raina <103954755+k-raina@users.noreply.github.com> Date: Tue, 10 Jun 2025 05:37:29 +0530 Subject: [PATCH] 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 --- docs/design.html | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/docs/design.html b/docs/design.html index 6d15ba11ec9..c58ead07c9a 100644 --- a/docs/design.html +++ b/docs/design.html @@ -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.

+ 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: +

    +
  1. RetriableException: Temporary exceptions that are retried automatically by the client. These are handled internally and don't bubble up to the application.
  2. +
  3. RefreshRetriableException: Exceptions requiring metadata refresh before retry. These are handled internally by the client after refreshing metadata and don't bubble up to the application.
  4. +
  5. AbortableException: 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.
  6. +
  7. ApplicationRecoverableException: 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.
  8. +
  9. InvalidConfigurationException: 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.
  10. +
  11. KafkaException: General Kafka exceptions that don't fit into the above categories. These bubble up to the application for handling.
  12. +
+

+ Example template code for handling transaction exceptions link : Transaction Client Demo +

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 KafkaConsumer.committed to retrieve the committed offset from Kafka, and then KafkaConsumer.seek to rewind the current position.