[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14222684#comment-14222684 ]
Ewen Cheslack-Postava commented on KAFKA-1642: ---------------------------------------------- [~Bmis13] I've only taken a quick look at the patch, but Sender.run() is intentionally a tight loop. The body of the loop calls Sender.run(time), which in turn calls Client.poll() with a timeout that *should* keep it from actually being a tight loop. The previous patch fixed some issues in those other methods that were causing the timeout to incorrectly be 0, leading to the high CPU usage. If you're still seeing this problem, the right fix will almost definitely involve tracking down how a 0 (or very small) timeout is consistently being computed. The way I verified the previous patch was simple -- I ran the producer against a local cluster and then just disabled the network connection. Can you describe how you produce the error now? Things like whether there are already active connections to the brokers, if it's time sensitive (e.g. takes a certain amount of time to start using CPU), exactly how you simulate the network failure, and whether the issue is consistent or only happens intermittently would all be helpful details to know. The issue with it not shutting down is probably because the producer doesn't timeout messages when the leader it needs to send them to isn't available, but it waits to send any outstanding messages before shutting down. KAFKA-1788 is probably really the same issue since it's also caused by messages that never get sent and don't time out, although since that issue is specifically about the RecordAccumulator/buffer pool accounting, a fix for that issue may or may not fix the shutdown issue you're describing here. > [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network > connection is lost > --------------------------------------------------------------------------------------- > > Key: KAFKA-1642 > URL: https://issues.apache.org/jira/browse/KAFKA-1642 > Project: Kafka > Issue Type: Bug > Components: producer > Affects Versions: 0.8.2 > Reporter: Bhavesh Mistry > Assignee: Ewen Cheslack-Postava > Fix For: 0.8.2 > > Attachments: > 0001-Initial-CPU-Hish-Usage-by-Kafka-FIX-and-Also-fix-CLO.patch, > KAFKA-1642.patch, KAFKA-1642_2014-10-20_17:33:57.patch, > KAFKA-1642_2014-10-23_16:19:41.patch > > > I see my CPU spike to 100% when network connection is lost for while. It > seems network IO thread are very busy logging following error message. Is > this expected behavior ? > 2014-09-17 14:06:16.830 [kafka-producer-network-thread] ERROR > org.apache.kafka.clients.producer.internals.Sender - Uncaught error in kafka > producer I/O thread: > java.lang.IllegalStateException: No entry found for node -2 > at > org.apache.kafka.clients.ClusterConnectionStates.nodeState(ClusterConnectionStates.java:110) > at > org.apache.kafka.clients.ClusterConnectionStates.disconnected(ClusterConnectionStates.java:99) > at > org.apache.kafka.clients.NetworkClient.initiateConnect(NetworkClient.java:394) > at > org.apache.kafka.clients.NetworkClient.maybeUpdateMetadata(NetworkClient.java:380) > at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:174) > at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:175) > at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:115) > at java.lang.Thread.run(Thread.java:744) > Thanks, > Bhavesh -- This message was sent by Atlassian JIRA (v6.3.4#6332)