Skip to content

Commit

Permalink
KAFKA-18476: KafkaStreams should swallow TransactionAbortedException (#…
Browse files Browse the repository at this point in the history
…18487)

TransactionAbortedException is a follow up error to a previous error,
and such a previous error would already be handled when
`producer.abortTransaction()` is called. Thus, a
TransactionAbortedException can just be silently swallowed.

Reviewers: Bill Bejeck <[email protected]>
  • Loading branch information
mjsax authored Jan 15, 2025
1 parent faef80a commit 1672a4b
Show file tree
Hide file tree
Showing 2 changed files with 32 additions and 0 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import org.apache.kafka.common.errors.SecurityDisabledException;
import org.apache.kafka.common.errors.SerializationException;
import org.apache.kafka.common.errors.TimeoutException;
import org.apache.kafka.common.errors.TransactionAbortedException;
import org.apache.kafka.common.errors.UnknownServerException;
import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.header.internals.RecordHeaders;
Expand Down Expand Up @@ -441,6 +442,11 @@ private void recordSendError(final String topic,
errorMessage += "\nWritten offsets would not be recorded and no more records would be sent since the producer is fenced, " +
"indicating the task may be migrated out";
sendException.set(new TaskMigratedException(errorMessage, productionException));
} else if (productionException instanceof TransactionAbortedException) {
// swallow silently
//
// TransactionAbortedException is only thrown after `abortTransaction()` was called,
// so it's only a followup error, and Kafka Streams is already handling the original error
} else {
final ProductionExceptionHandlerResponse response;
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import org.apache.kafka.common.errors.ProducerFencedException;
import org.apache.kafka.common.errors.SerializationException;
import org.apache.kafka.common.errors.TimeoutException;
import org.apache.kafka.common.errors.TransactionAbortedException;
import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
import org.apache.kafka.common.header.Header;
import org.apache.kafka.common.header.Headers;
Expand Down Expand Up @@ -1806,6 +1807,31 @@ public void shouldNotCallProductionExceptionHandlerOnClassCastException() {
}
}

@Test
public void shouldSwallowTransactionAbortedExceptionAndNotCallProductionExceptionHandler() {
final MockProducer<byte[], byte[]> mockProducer = new MockProducer<>(
cluster,
false,
new org.apache.kafka.clients.producer.RoundRobinPartitioner(),
new ByteArraySerializer(),
new ByteArraySerializer()
);
streamsProducer = new StreamsProducer(
mockProducer,
EXACTLY_ONCE_V2,
Time.SYSTEM,
logContext
);

final RecordCollector collector = newRecordCollector(new ProductionExceptionHandlerMock());
collector.initialize();

collector.send(topic, "key", "val", null, 0, null, stringSerializer, stringSerializer, sinkNodeName, context);
mockProducer.errorNext(new TransactionAbortedException()); // error out the send() call

collector.flush(); // need to call flush() to check for internal exceptions
}

@Test
public void shouldNotSendIfSendOfOtherTaskFailedInCallback() {
final TaskId taskId1 = new TaskId(0, 0);
Expand Down

0 comments on commit 1672a4b

Please sign in to comment.