[ https://issues.apache.org/jira/browse/KAFKA-4557?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15839994#comment-15839994 ]
ASF GitHub Bot commented on KAFKA-4557: --------------------------------------- GitHub user rajinisivaram opened a pull request: https://github.com/apache/kafka/pull/2449 KAFKA-4557: Handle Producer.send correctly in expiry callbacks When iterating deque for expiring record batches, delay the actual completion of the batch until iteration is complete since callbacks invoked during expiry may send more records, modifying the deque, resulting in ConcurrentModificationException in the iterator. You can merge this pull request into a Git repository by running: $ git pull https://github.com/rajinisivaram/kafka KAFKA-4557 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/kafka/pull/2449.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #2449 ---- commit dae191b0dc14bed39b8f77589c74a2394ed4ea48 Author: Rajini Sivaram <rajinisiva...@googlemail.com> Date: 2017-01-26T16:23:49Z KAFKA-4557: Handle Producer.send correctly in expiry callbacks ---- > ConcurrentModificationException in KafkaProducer event loop > ----------------------------------------------------------- > > Key: KAFKA-4557 > URL: https://issues.apache.org/jira/browse/KAFKA-4557 > Project: Kafka > Issue Type: Bug > Components: clients > Affects Versions: 0.10.1.0 > Reporter: Sergey Alaev > Assignee: Rajini Sivaram > Priority: Critical > Labels: reliability > Fix For: 0.10.2.0 > > > Under heavy load, Kafka producer can stop publishing events. Logs below. > [2016-12-19T15:01:28.779Z] [sgs] [kafka-producer-network-thread | producer-3] > [NetworkClient] [] [<none>] [] [DEBUG]: Disconnecting from node 2 due to > request timeout. > [2016-12-19T15:01:28.793Z] [sgs] [kafka-producer-network-thread | producer-3] > [KafkaProducerClient] [] [<none>] [1B2M2Y8Asg] [WARN]: Error sending message > to Kafka > org.apache.kafka.common.errors.NetworkException: The server disconnected > before a response was received. > [2016-12-19T15:01:28.838Z] [sgs] [kafka-producer-network-thread | producer-3] > [KafkaProducerClient] [] [<none>] [1B2M2Y8Asg] [WARN]: Error sending message > to Kafka > org.apache.kafka.common.errors.NetworkException: The server disconnected > before a response was received. (#2 from 2016-12-19T15:01:28.793Z) > -------------------------------- > [2016-12-19T15:01:28.956Z] [sgs] [kafka-producer-network-thread | producer-3] > [KafkaProducerClient] [] [<none>] [1B2M2Y8Asg] [WARN]: Error sending message > to Kafka > org.apache.kafka.common.errors.TimeoutException: Expiring 46 record(s) for > events-deadletter-0 due to 30032 ms has passed since batch creation plus > linger time (#285 from 2016-12-19 > T15:01:28.793Z) > [2016-12-19T15:01:28.956Z] [sgs] [kafka-producer-network-thread | producer-3] > [SgsService] [] [<none>] [1B2M2Y8Asg] [WARN]: Error writing signal to Kafka > deadletter queue > org.apache.kafka.common.errors.TimeoutException: Expiring 46 record(s) for > events-deadletter-0 due to 30032 ms has passed since batch creation plus > linger time (#286 from 2016-12-19 > T15:01:28.793Z) > [2016-12-19T15:01:28.960Z] [sgs] [kafka-producer-network-thread | producer-3] > [Sender] [] [<none>] [1B2M2Y8Asg] [ERROR]: Uncaught error in kafka producer > I/O thread: > java.util.ConcurrentModificationException: null > at java.util.ArrayDeque$DeqIterator.next(ArrayDeque.java:643) > ~[na:1.8.0_45] > at > org.apache.kafka.clients.producer.internals.RecordAccumulator.abortExpiredBatches(RecordAccumulator.java:242) > ~[kafka-clients-0.10.1.0.jar:na] > at > org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:212) > ~[kafka-clients-0.10.1.0.jar:na] > at > org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:135) > ~[kafka-clients-0.10.1.0.jar:na] > at java.lang.Thread.run(Thread.java:745) [na:1.8.0_45] > [2016-12-19T15:01:28.981Z] [sgs] [kafka-producer-network-thread | producer-3] > [NetworkClient] [] [<none>] [1B2M2Y8Asg] [WARN]: Error while fetching > metadata with correlation id 28711 : {events-deadletter=LEADER_NOT_AVAILABLE} -- This message was sent by Atlassian JIRA (v6.3.4#6332)