Skip to content

Commit b1ea280

Browse files
authored
KAFKA-19250 : txnProducer.abortTransaction() API should not return abortable exception (#19656)
## Problem When an `txnProducer.abortTransaction()`operation encounters a `TRANSACTION_ABORTABLE` error, it currently tries to transition to `ABORTABLE_ERROR` state. This can create an infinite retry loop since: 1. The abort operation fails with `TRANSACTION_ABORTABLE` 2. We transition to `ABORTABLE_ERROR` state 3. The application recieves instance of TransactionAbortableException and it retries the abort 4. The cycle repeats ## Solution For `txnProducer.abortTransaction()`API, convert `TRANSACTION_ABORTABLE` errors to fatal errors (`KafkaException`) during abort operations to ensure clean transaction termination. This prevents retry loops by: 1. Treating abort failures as fatal errors at application layer 2. Ensuring the transaction can be cleanly terminated 3. Providing clear error messages to the application ## Changes - Modified `EndTxnHandler.handleResponse()` to convert `TRANSACTION_ABORTABLE` errors to `KafkaException` during abort operations - Set TransactionManager state to FATAL - Updated test `testAbortableErrorIsConvertedToFatalErrorDuringAbort` to verify this behavior ## Testing - Added test case verifying that abort operations convert `TRANSACTION_ABORTABLE` errors to `KafkaException` - Verified that Commit API with TRANSACTION_ABORTABLE error should set TM to Abortable state - Verified that Abort API with TRANSACTION_ABORTABLE error should convert to Fatal error i.e. KafkaException ## Impact At application layer, this change improves transaction reliability by preventing infinite retry loops during abort operations. Reviewers: Justine Olshan <[email protected]>
1 parent 8c71ab0 commit b1ea280

File tree

2 files changed

+61
-1
lines changed

2 files changed

+61
-1
lines changed

clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1738,7 +1738,7 @@ boolean isEndTxn() {
17381738
public void handleResponse(AbstractResponse response) {
17391739
EndTxnResponse endTxnResponse = (EndTxnResponse) response;
17401740
Errors error = endTxnResponse.error();
1741-
1741+
boolean isAbort = !builder.data.committed();
17421742
if (error == Errors.NONE) {
17431743
// For End Txn version 5+, the broker includes the producerId and producerEpoch in the EndTxnResponse.
17441744
// For versions lower than 5, the producer Id and epoch are set to -1 by default.
@@ -1771,6 +1771,11 @@ public void handleResponse(AbstractResponse response) {
17711771
fatalError(error.exception());
17721772
} else if (error == Errors.UNKNOWN_PRODUCER_ID) {
17731773
abortableErrorIfPossible(error.exception());
1774+
} else if (isAbort && error.exception() instanceof TransactionAbortableException) {
1775+
// When aborting a transaction, we must convert TRANSACTION_ABORTABLE errors to KafkaException
1776+
// because if an abort operation itself encounters an abortable error, retrying the abort would create a cycle.
1777+
// Instead, we treat this as fatal error at the application layer to ensure the transaction can be cleanly terminated.
1778+
fatalError(new KafkaException("Failed to abort transaction", error.exception()));
17741779
} else if (error == Errors.TRANSACTION_ABORTABLE) {
17751780
abortableError(error.exception());
17761781
} else {

clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java

Lines changed: 55 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -3236,6 +3236,61 @@ public void testTransactionAbortableExceptionIsAnAbortableError() throws Excepti
32363236

32373237
txnManager.beginTransaction();
32383238
}
3239+
3240+
@Test
3241+
public void testAbortableErrorIsConvertedToFatalErrorDuringAbort() throws Exception {
3242+
3243+
// Initialize and begin transaction
3244+
TransactionManager transactionManager = new TransactionManager(logContext, "testAbortableErrorIsConvertedToFatalErrorDuringAbort", 60000, 100, apiVersions, false);
3245+
setupWithTransactionState(transactionManager);
3246+
doInitTransactions(transactionManager, new ProducerIdAndEpoch(1L, (short) 0));
3247+
transactionManager.beginTransaction();
3248+
3249+
// Add partition and send record
3250+
TopicPartition tp = new TopicPartition("test", 0);
3251+
addPartitionToTxn(sender, transactionManager, tp);
3252+
appendToAccumulator(tp);
3253+
3254+
// Send record and get response
3255+
sender.runOnce();
3256+
sendIdempotentProducerResponse(0, tp, Errors.NONE, 0);
3257+
sender.runOnce();
3258+
3259+
// Commit API with TRANSACTION_ABORTABLE error should set TM to Abortable state
3260+
client.prepareResponse(new EndTxnResponse(new EndTxnResponseData()
3261+
.setErrorCode(Errors.TRANSACTION_ABORTABLE.code())));
3262+
3263+
// Attempt to commit transaction
3264+
TransactionalRequestResult commitResult = transactionManager.beginCommit();
3265+
sender.runOnce();
3266+
try {
3267+
commitResult.await(1000, TimeUnit.MILLISECONDS);
3268+
fail("Expected abortable error to be thrown for commit");
3269+
} catch (KafkaException e) {
3270+
assertTrue(transactionManager.hasAbortableError());
3271+
assertEquals(commitResult.error().getClass(), TransactionAbortableException.class);
3272+
}
3273+
3274+
// Abort API with TRANSACTION_ABORTABLE error should convert to Fatal error i.e. KafkaException
3275+
client.prepareResponse(new EndTxnResponse(new EndTxnResponseData()
3276+
.setErrorCode(Errors.TRANSACTION_ABORTABLE.code())));
3277+
3278+
// Attempt to abort transaction
3279+
TransactionalRequestResult abortResult = transactionManager.beginAbort();
3280+
sender.runOnce();
3281+
3282+
// Verify the error is converted to KafkaException (not TransactionAbortableException)
3283+
try {
3284+
abortResult.await(1000, TimeUnit.MILLISECONDS);
3285+
fail("Expected KafkaException to be thrown");
3286+
} catch (KafkaException e) {
3287+
// Verify TM is in FATAL_ERROR state
3288+
assertTrue(transactionManager.hasFatalError());
3289+
assertFalse(e instanceof TransactionAbortableException);
3290+
assertEquals(abortResult.error().getClass(), KafkaException.class);
3291+
}
3292+
}
3293+
32393294
@Test
32403295
public void testProducerBatchRetriesWhenPartitionLeaderChanges() throws Exception {
32413296
Metrics m = new Metrics();

0 commit comments

Comments
 (0)