Description
We've seen the following exception in our eos-beta test application recently:
[2020-06-13T00:09:14-07:00] (streams-soak-2-6-all-fixes-eos-beta_soak_i-0ae30dd12c4fb7018_streamslog) org.apache.kafka.streams.errors.StreamsException: Error encountered sending record to topic stream-soak-test-KSTREAM-AGGREGATE-STATE-STORE-0000000025-changelog for task 1_2 due to: [2020-06-13T00:09:14-07:00] (streams-soak-2-6-all-fixes-eos-beta_soak_i-0ae30dd12c4fb7018_streamslog) org.apache.kafka.common.KafkaException: Failing batch since transaction was aborted [2020-06-13T00:09:14-07:00] (streams-soak-2-6-all-fixes-eos-beta_soak_i-0ae30dd12c4fb7018_streamslog) Exception handler choose to FAIL the processing, no more records would be sent. at org.apache.kafka.streams.processor.internals.RecordCollectorImpl.recordSendError(RecordCollectorImpl.java:213) at org.apache.kafka.streams.processor.internals.RecordCollectorImpl.lambda$send$0(RecordCollectorImpl.java:185) at org.apache.kafka.clients.producer.KafkaProducer$InterceptorCallback.onCompletion(KafkaProducer.java:1347) at org.apache.kafka.clients.producer.internals.ProducerBatch.completeFutureAndFireCallbacks(ProducerBatch.java:231) at org.apache.kafka.clients.producer.internals.ProducerBatch.abort(ProducerBatch.java:159) at org.apache.kafka.clients.producer.internals.RecordAccumulator.abortUndrainedBatches(RecordAccumulator.java:781) at org.apache.kafka.clients.producer.internals.Sender.maybeSendAndPollTransactionalRequest(Sender.java:425) at org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:313) at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:240) at java.lang.Thread.run(Thread.java:748) [2020-06-13T00:09:14-07:00] (streams-soak-2-6-all-fixes-eos-beta_soak_i-0ae30dd12c4fb7018_streamslog) Caused by: org.apache.kafka.common.KafkaException: Failing batch since transaction was aborted at org.apache.kafka.clients.producer.internals.Sender.maybeSendAndPollTransactionalRequest(Sender.java:423) ... 3 more
Somewhat unclear if this is an issue with eos-beta specifically, or just eos in general. But several threads have died over the course of a few days in the eos-beta application, while none so far have died on the eos-alpha application.
It's also unclear (at least to me) whether this is definitely an issue in Streams or possibly a bug in the producer (or even the broker, although that seems unlikely)
Attachments
Issue Links
- is caused by
-
KAFKA-10186 Aborting transaction with pending data should throw non-fatal exception
- Resolved
- links to