[
https://issues.apache.org/jira/browse/KAFKA-9605?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Boyang Chen updated KAFKA-9605:
-------------------------------
Description:
In the Producer we could see network client hits fatal exception while trying
to complete the batches after Txn manager hits fatal fenced error:
{code:java}
[2020-02-24T13:23:29-08:00]
(streams-soak-trunk-eos_soak_i-02ea56d369c55eec2_streamslog) [2020-02-24
21:23:28,673] ERROR [kafka-producer-network-thread |
stream-soak-test-5e16fa60-12a3-4c4f-9900-c75f7d10859f-StreamThread-3-1_0-producer]
[Producer
clientId=stream-soak-test-5e16fa60-12a3-4c4f-9900-c75f7d10859f-StreamThread-3-1_0-producer,
transactionalId=stream-soak-test-1_0] Aborting producer batches due to fatal
error (org.apache.kafka.clients.producer.internals.Sender)
[2020-02-24T13:23:29-08:00]
(streams-soak-trunk-eos_soak_i-02ea56d369c55eec2_streamslog)
org.apache.kafka.common.errors.ProducerFencedException: Producer attempted an
operation with an old epoch. Either there is a newer producer with the same
transactionalId, or the producer's transaction has been expired by the broker.
[2020-02-24T13:23:29-08:00]
(streams-soak-trunk-eos_soak_i-02ea56d369c55eec2_streamslog) [2020-02-24
21:23:28,674] INFO
[stream-soak-test-5e16fa60-12a3-4c4f-9900-c75f7d10859f-StreamThread-3]
[Producer
clientId=stream-soak-test-5e16fa60-12a3-4c4f-9900-c75f7d10859f-StreamThread-3-0_0-producer,
transactionalId=stream-soak-test-0_0] Closing the Kafka producer with
timeoutMillis = 9223372036854775807 ms.
(org.apache.kafka.clients.producer.KafkaProducer)
[2020-02-24T13:23:29-08:00]
(streams-soak-trunk-eos_soak_i-02ea56d369c55eec2_streamslog) [2020-02-24
21:23:28,684] INFO [kafka-producer-network-thread |
stream-soak-test-5e16fa60-12a3-4c4f-9900-c75f7d10859f-StreamThread-3-1_0-producer]
[Producer
clientId=stream-soak-test-5e16fa60-12a3-4c4f-9900-c75f7d10859f-StreamThread-3-1_0-producer,
transactionalId=stream-soak-test-1_0] Resetting sequence number of batch with
current sequence 354277 for partition windowed-node-counts-0 to 354276
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2020-02-24T13:23:29-08:00]
(streams-soak-trunk-eos_soak_i-02ea56d369c55eec2_streamslog) [2020-02-24
21:23:28,684] INFO [kafka-producer-network-thread |
stream-soak-test-5e16fa60-12a3-4c4f-9900-c75f7d10859f-StreamThread-3-1_0-producer]
Resetting sequence number of batch with current sequence 354277 for partition
windowed-node-counts-0 to 354276
(org.apache.kafka.clients.producer.internals.ProducerBatch)
[2020-02-24T13:23:29-08:00]
(streams-soak-trunk-eos_soak_i-02ea56d369c55eec2_streamslog) [2020-02-24
21:23:28,685] ERROR [kafka-producer-network-thread |
stream-soak-test-5e16fa60-12a3-4c4f-9900-c75f7d10859f-StreamThread-3-1_0-producer]
[Producer
clientId=stream-soak-test-5e16fa60-12a3-4c4f-9900-c75f7d10859f-StreamThread-3-1_0-producer,
transactionalId=stream-soak-test-1_0] Uncaught error in request completion:
(org.apache.kafka.clients.NetworkClient)
[2020-02-24T13:23:29-08:00]
(streams-soak-trunk-eos_soak_i-02ea56d369c55eec2_streamslog)
java.lang.IllegalStateException: Should not reopen a batch which is already
aborted.
at
org.apache.kafka.common.record.MemoryRecordsBuilder.reopenAndRewriteProducerState(MemoryRecordsBuilder.java:295)
at
org.apache.kafka.clients.producer.internals.ProducerBatch.resetProducerState(ProducerBatch.java:395)
at
org.apache.kafka.clients.producer.internals.TransactionManager.lambda$adjustSequencesDueToFailedBatch$4(TransactionManager.java:770)
at
org.apache.kafka.clients.producer.internals.TransactionManager$TopicPartitionEntry.resetSequenceNumbers(TransactionManager.java:180)
at
org.apache.kafka.clients.producer.internals.TransactionManager.adjustSequencesDueToFailedBatch(TransactionManager.java:760)
at
org.apache.kafka.clients.producer.internals.TransactionManager.handleFailedBatch(TransactionManager.java:735)
at
org.apache.kafka.clients.producer.internals.Sender.failBatch(Sender.java:671)
at
org.apache.kafka.clients.producer.internals.Sender.failBatch(Sender.java:662)
at
org.apache.kafka.clients.producer.internals.Sender.completeBatch(Sender.java:620)
at
org.apache.kafka.clients.producer.internals.Sender.handleProduceResponse(Sender.java:554)
at
org.apache.kafka.clients.producer.internals.Sender.access$100(Sender.java:69)
at
org.apache.kafka.clients.producer.internals.Sender$1.onComplete(Sender.java:745)
at
org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:109)
at
org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:571)
at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:563)
at
org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:304)
at
org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:239)
at java.lang.Thread.run(Thread.java:748)
{code}
The proper fix is to add a check for handle failed batch in txn manager.
was:
```
[2020-02-24T13:23:29-08:00]
(streams-soak-trunk-eos_soak_i-02ea56d369c55eec2_streamslog) [2020-02-24
21:23:28,673] ERROR [kafka-producer-network-thread |
stream-soak-test-5e16fa60-12a3-4c4f-9900-c75f7d10859f-StreamThread-3-1_0-producer]
[Producer
clientId=stream-soak-test-5e16fa60-12a3-4c4f-9900-c75f7d10859f-StreamThread-3-1_0-producer,
transactionalId=stream-soak-test-1_0] Aborting producer batches due to fatal
error (org.apache.kafka.clients.producer.internals.Sender)
[2020-02-24T13:23:29-08:00]
(streams-soak-trunk-eos_soak_i-02ea56d369c55eec2_streamslog)
org.apache.kafka.common.errors.ProducerFencedException: Producer attempted an
operation with an old epoch. Either there is a newer producer with the same
transactionalId, or the producer's transaction has been expired by the broker.
[2020-02-24T13:23:29-08:00]
(streams-soak-trunk-eos_soak_i-02ea56d369c55eec2_streamslog) [2020-02-24
21:23:28,674] INFO
[stream-soak-test-5e16fa60-12a3-4c4f-9900-c75f7d10859f-StreamThread-3]
[Producer
clientId=stream-soak-test-5e16fa60-12a3-4c4f-9900-c75f7d10859f-StreamThread-3-0_0-producer,
transactionalId=stream-soak-test-0_0] Closing the Kafka producer with
timeoutMillis = 9223372036854775807 ms.
(org.apache.kafka.clients.producer.KafkaProducer)
[2020-02-24T13:23:29-08:00]
(streams-soak-trunk-eos_soak_i-02ea56d369c55eec2_streamslog) [2020-02-24
21:23:28,684] INFO [kafka-producer-network-thread |
stream-soak-test-5e16fa60-12a3-4c4f-9900-c75f7d10859f-StreamThread-3-1_0-producer]
[Producer
clientId=stream-soak-test-5e16fa60-12a3-4c4f-9900-c75f7d10859f-StreamThread-3-1_0-producer,
transactionalId=stream-soak-test-1_0] Resetting sequence number of batch with
current sequence 354277 for partition windowed-node-counts-0 to 354276
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2020-02-24T13:23:29-08:00]
(streams-soak-trunk-eos_soak_i-02ea56d369c55eec2_streamslog) [2020-02-24
21:23:28,684] INFO [kafka-producer-network-thread |
stream-soak-test-5e16fa60-12a3-4c4f-9900-c75f7d10859f-StreamThread-3-1_0-producer]
Resetting sequence number of batch with current sequence 354277 for partition
windowed-node-counts-0 to 354276
(org.apache.kafka.clients.producer.internals.ProducerBatch)
[2020-02-24T13:23:29-08:00]
(streams-soak-trunk-eos_soak_i-02ea56d369c55eec2_streamslog) [2020-02-24
21:23:28,685] ERROR [kafka-producer-network-thread |
stream-soak-test-5e16fa60-12a3-4c4f-9900-c75f7d10859f-StreamThread-3-1_0-producer]
[Producer
clientId=stream-soak-test-5e16fa60-12a3-4c4f-9900-c75f7d10859f-StreamThread-3-1_0-producer,
transactionalId=stream-soak-test-1_0] Uncaught error in request completion:
(org.apache.kafka.clients.NetworkClient)
[2020-02-24T13:23:29-08:00]
(streams-soak-trunk-eos_soak_i-02ea56d369c55eec2_streamslog)
java.lang.IllegalStateException: Should not reopen a batch which is already
aborted.
at
org.apache.kafka.common.record.MemoryRecordsBuilder.reopenAndRewriteProducerState(MemoryRecordsBuilder.java:295)
at
org.apache.kafka.clients.producer.internals.ProducerBatch.resetProducerState(ProducerBatch.java:395)
at
org.apache.kafka.clients.producer.internals.TransactionManager.lambda$adjustSequencesDueToFailedBatch$4(TransactionManager.java:770)
at
org.apache.kafka.clients.producer.internals.TransactionManager$TopicPartitionEntry.resetSequenceNumbers(TransactionManager.java:180)
at
org.apache.kafka.clients.producer.internals.TransactionManager.adjustSequencesDueToFailedBatch(TransactionManager.java:760)
at
org.apache.kafka.clients.producer.internals.TransactionManager.handleFailedBatch(TransactionManager.java:735)
at
org.apache.kafka.clients.producer.internals.Sender.failBatch(Sender.java:671)
at
org.apache.kafka.clients.producer.internals.Sender.failBatch(Sender.java:662)
at
org.apache.kafka.clients.producer.internals.Sender.completeBatch(Sender.java:620)
at
org.apache.kafka.clients.producer.internals.Sender.handleProduceResponse(Sender.java:554)
at
org.apache.kafka.clients.producer.internals.Sender.access$100(Sender.java:69)
at
org.apache.kafka.clients.producer.internals.Sender$1.onComplete(Sender.java:745)
at
org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:109)
at
org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:571)
at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:563)
at
org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:304)
at
org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:239)
at java.lang.Thread.run(Thread.java:748)```
> EOS Producer could throw illegal state if trying to complete a failed batch
> after fatal error
> ---------------------------------------------------------------------------------------------
>
> Key: KAFKA-9605
> URL: https://issues.apache.org/jira/browse/KAFKA-9605
> Project: Kafka
> Issue Type: Bug
> Affects Versions: 2.4.0, 2.5.0
> Reporter: Boyang Chen
> Assignee: Boyang Chen
> Priority: Major
> Fix For: 2.5.0
>
>
> In the Producer we could see network client hits fatal exception while trying
> to complete the batches after Txn manager hits fatal fenced error:
> {code:java}
>
> [2020-02-24T13:23:29-08:00]
> (streams-soak-trunk-eos_soak_i-02ea56d369c55eec2_streamslog) [2020-02-24
> 21:23:28,673] ERROR [kafka-producer-network-thread |
> stream-soak-test-5e16fa60-12a3-4c4f-9900-c75f7d10859f-StreamThread-3-1_0-producer]
> [Producer
> clientId=stream-soak-test-5e16fa60-12a3-4c4f-9900-c75f7d10859f-StreamThread-3-1_0-producer,
> transactionalId=stream-soak-test-1_0] Aborting producer batches due to fatal
> error (org.apache.kafka.clients.producer.internals.Sender)
> [2020-02-24T13:23:29-08:00]
> (streams-soak-trunk-eos_soak_i-02ea56d369c55eec2_streamslog)
> org.apache.kafka.common.errors.ProducerFencedException: Producer attempted an
> operation with an old epoch. Either there is a newer producer with the same
> transactionalId, or the producer's transaction has been expired by the broker.
> [2020-02-24T13:23:29-08:00]
> (streams-soak-trunk-eos_soak_i-02ea56d369c55eec2_streamslog) [2020-02-24
> 21:23:28,674] INFO
> [stream-soak-test-5e16fa60-12a3-4c4f-9900-c75f7d10859f-StreamThread-3]
> [Producer
> clientId=stream-soak-test-5e16fa60-12a3-4c4f-9900-c75f7d10859f-StreamThread-3-0_0-producer,
> transactionalId=stream-soak-test-0_0] Closing the Kafka producer with
> timeoutMillis = 9223372036854775807 ms.
> (org.apache.kafka.clients.producer.KafkaProducer)
> [2020-02-24T13:23:29-08:00]
> (streams-soak-trunk-eos_soak_i-02ea56d369c55eec2_streamslog) [2020-02-24
> 21:23:28,684] INFO [kafka-producer-network-thread |
> stream-soak-test-5e16fa60-12a3-4c4f-9900-c75f7d10859f-StreamThread-3-1_0-producer]
> [Producer
> clientId=stream-soak-test-5e16fa60-12a3-4c4f-9900-c75f7d10859f-StreamThread-3-1_0-producer,
> transactionalId=stream-soak-test-1_0] Resetting sequence number of batch
> with current sequence 354277 for partition windowed-node-counts-0 to 354276
> (org.apache.kafka.clients.producer.internals.TransactionManager)
> [2020-02-24T13:23:29-08:00]
> (streams-soak-trunk-eos_soak_i-02ea56d369c55eec2_streamslog) [2020-02-24
> 21:23:28,684] INFO [kafka-producer-network-thread |
> stream-soak-test-5e16fa60-12a3-4c4f-9900-c75f7d10859f-StreamThread-3-1_0-producer]
> Resetting sequence number of batch with current sequence 354277 for
> partition windowed-node-counts-0 to 354276
> (org.apache.kafka.clients.producer.internals.ProducerBatch)
> [2020-02-24T13:23:29-08:00]
> (streams-soak-trunk-eos_soak_i-02ea56d369c55eec2_streamslog) [2020-02-24
> 21:23:28,685] ERROR [kafka-producer-network-thread |
> stream-soak-test-5e16fa60-12a3-4c4f-9900-c75f7d10859f-StreamThread-3-1_0-producer]
> [Producer
> clientId=stream-soak-test-5e16fa60-12a3-4c4f-9900-c75f7d10859f-StreamThread-3-1_0-producer,
> transactionalId=stream-soak-test-1_0] Uncaught error in request completion:
> (org.apache.kafka.clients.NetworkClient)
> [2020-02-24T13:23:29-08:00]
> (streams-soak-trunk-eos_soak_i-02ea56d369c55eec2_streamslog)
> java.lang.IllegalStateException: Should not reopen a batch which is already
> aborted.
> at
> org.apache.kafka.common.record.MemoryRecordsBuilder.reopenAndRewriteProducerState(MemoryRecordsBuilder.java:295)
> at
> org.apache.kafka.clients.producer.internals.ProducerBatch.resetProducerState(ProducerBatch.java:395)
> at
> org.apache.kafka.clients.producer.internals.TransactionManager.lambda$adjustSequencesDueToFailedBatch$4(TransactionManager.java:770)
> at
> org.apache.kafka.clients.producer.internals.TransactionManager$TopicPartitionEntry.resetSequenceNumbers(TransactionManager.java:180)
> at
> org.apache.kafka.clients.producer.internals.TransactionManager.adjustSequencesDueToFailedBatch(TransactionManager.java:760)
> at
> org.apache.kafka.clients.producer.internals.TransactionManager.handleFailedBatch(TransactionManager.java:735)
> at
> org.apache.kafka.clients.producer.internals.Sender.failBatch(Sender.java:671)
> at
> org.apache.kafka.clients.producer.internals.Sender.failBatch(Sender.java:662)
> at
> org.apache.kafka.clients.producer.internals.Sender.completeBatch(Sender.java:620)
> at
> org.apache.kafka.clients.producer.internals.Sender.handleProduceResponse(Sender.java:554)
> at
> org.apache.kafka.clients.producer.internals.Sender.access$100(Sender.java:69)
> at
> org.apache.kafka.clients.producer.internals.Sender$1.onComplete(Sender.java:745)
> at
> org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:109)
> at
> org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:571)
> at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:563)
> at
> org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:304)
> at
> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:239)
> at java.lang.Thread.run(Thread.java:748)
> {code}
> The proper fix is to add a check for handle failed batch in txn manager.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)