[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14226751#comment-14226751 ]
Bhavesh Mistry commented on KAFKA-1642: --------------------------------------- [~ewencp], Even setting long following parameter, states of system does get impacted does not matter what reconnect.backoff.ms and retry.backoff.ms is set to. Once Node state is removed, the time out is set to 0. Please see the following logs. # 15 minutes reconnect.backoff.ms=900000 retry.backoff.ms=900000 {code} 2014-11-26 11:01:27.898 Kafka Drop message topic=.rawlog org.apache.kafka.common.errors.TimeoutException: Failed to update metadata after 60000 ms. 2014-11-26 11:02:27.903 Kafka Drop message topic=.rawlog org.apache.kafka.common.errors.TimeoutException: Failed to update metadata after 60000 ms. 2014-11-26 11:03:27.903 Kafka Drop message topic=.rawlog org.apache.kafka.common.errors.TimeoutException: Failed to update metadata after 60000 ms. 2014-11-26 11:04:27.903 Kafka Drop message topic=.rawlog org.apache.kafka.common.errors.TimeoutException: Failed to update metadata after 60000 ms. 2014-11-26 11:05:27.904 Kafka Drop message topic=.rawlog org.apache.kafka.common.errors.TimeoutException: Failed to update metadata after 60000 ms. 2014-11-26 11:06:27.905 Kafka Drop message topic=.rawlog org.apache.kafka.common.errors.TimeoutException: Failed to update metadata after 60000 ms. 2014-11-26 11:07:27.906 Kafka Drop message topic=.rawlog org.apache.kafka.common.errors.TimeoutException: Failed to update metadata after 60000 ms. 2014-11-26 11:08:27.908 Kafka Drop message topic=.rawlog org.apache.kafka.common.errors.TimeoutException: Failed to update metadata after 60000 ms. 2014-11-26 11:09:27.908 Kafka Drop message topic=.rawlog org.apache.kafka.common.errors.TimeoutException: Failed to update metadata after 60000 ms. 2014-11-26 11:10:27.909 Kafka Drop message topic=.rawlog org.apache.kafka.common.errors.TimeoutException: Failed to update metadata after 60000 ms. 2014-11-26 11:11:27.909 Kafka Drop message topic=.rawlog org.apache.kafka.common.errors.TimeoutException: Failed to update metadata after 60000 ms. 2014-11-26 11:12:27.910 Kafka Drop message topic=.rawlog org.apache.kafka.common.errors.TimeoutException: Failed to update metadata after 60000 ms. 2014-11-26 11:13:27.911 Kafka Drop message topic=.rawlog org.apache.kafka.common.errors.TimeoutException: Failed to update metadata after 60000 ms. 2014-11-26 11:14:27.912 Kafka Drop message topic=.rawlog org.apache.kafka.common.errors.TimeoutException: Failed to update metadata after 60000 ms. 2014-11-26 11:15:27.914 Kafka Drop message topic=.rawlog org.apache.kafka.common.errors.TimeoutException: Failed to update metadata after 60000 ms. 2014-11-26 11:00:27.613 [kafka-producer-network-thread | heartbeat] ERROR org.apache.kafka.clients.producer.internals.Sender - Uncaught error in kafka producer I/O thread: 2014-11-26 11:00:27.613 [kafka-producer-network-thread | rawlog] ERROR org.apache.kafka.clients.producer.internals.Sender - Uncaught error in kafka producer I/O thread: java.lang.IllegalStateException: No entry found for node -1 at org.apache.kafka.clients.ClusterConnectionStates.nodeState(ClusterConnectionStates.java:131) at org.apache.kafka.clients.ClusterConnectionStates.disconnected(ClusterConnectionStates.java:120) at org.apache.kafka.clients.NetworkClient.initiateConnect(NetworkClient.java:407) at org.apache.kafka.clients.NetworkClient.maybeUpdateMetadata(NetworkClient.java:393) at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:187) at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:184) at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:115) at java.lang.Thread.run(Thread.java:744) java.lang.IllegalStateException: No entry found for node -3 at org.apache.kafka.clients.ClusterConnectionStates.nodeState(ClusterConnectionStates.java:131) at org.apache.kafka.clients.ClusterConnectionStates.disconnected(ClusterConnectionStates.java:120) at org.apache.kafka.clients.NetworkClient.initiateConnect(NetworkClient.java:407) at org.apache.kafka.clients.NetworkClient.maybeUpdateMetadata(NetworkClient.java:393) at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:187) at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:184) at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:115) at java.lang.Thread.run(Thread.java:744) 2014-11-26 11:00:27.613 [kafka-producer-network-thread | heartbeat] ERROR org.apache.kafka.clients.producer.internals.Sender - Uncaught error in kafka producer I/O thread: 2014-11-26 11:00:27.613 [kafka-producer-network-thread | error] ERROR org.apache.kafka.clients.producer.internals.Sender - Uncaught error in kafka producer I/O thread: 2014-11-26 11:00:27.613 [kafka-producer-network-thread | event] ERROR org.apache.kafka.clients.producer.internals.Sender - Uncaught error in kafka producer I/O thread: java.lang.IllegalStateException: No entry found for node -1 at org.apache.kafka.clients.ClusterConnectionStates.nodeState(ClusterConnectionStates.java:131) at org.apache.kafka.clients.ClusterConnectionStates.disconnected(ClusterConnectionStates.java:120) at org.apache.kafka.clients.NetworkClient.initiateConnect(NetworkClient.java:407) at org.apache.kafka.clients.NetworkClient.maybeUpdateMetadata(NetworkClient.java:393) at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:187) at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:184) at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:115) at java.lang.Thread.run(Thread.java:744) java.lang.IllegalStateException: No entry found for node -1 at org.apache.kafka.clients.ClusterConnectionStates.nodeState(ClusterConnectionStates.java:131) at org.apache.kafka.clients.ClusterConnectionStates.disconnected(ClusterConnectionStates.java:120) at org.apache.kafka.clients.NetworkClient.initiateConnect(NetworkClient.java:407) at org.apache.kafka.clients.NetworkClient.maybeUpdateMetadata(NetworkClient.java:393) at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:187) at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:184) at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:115) at java.lang.Thread.run(Thread.java:744) 2014-11-26 11:00:27.613 [kafka-producer-network-thread | error] ERROR org.apache.kafka.clients.producer.internals.Sender - Uncaught error in kafka producer I/O thread: java.lang.IllegalStateException: No entry found for node -1 at org.apache.kafka.clients.ClusterConnectionStates.nodeState(ClusterConnectionStates.java:131) at org.apache.kafka.clients.ClusterConnectionStates.disconnected(ClusterConnectionStates.java:120) at org.apache.kafka.clients.NetworkClient.initiateConnect(NetworkClient.java:407) at org.apache.kafka.clients.NetworkClient.maybeUpdateMetadata(NetworkClient.java:393) at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:187) at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:184) at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:115) at java.lang.Thread.run(Thread.java:744) 2014-11-26 11:00:27.613 [kafka-producer-network-thread | rawlog] ERROR org.apache.kafka.clients.producer.internals.Sender - Uncaught error in kafka producer I/O thread: java.lang.IllegalStateException: No entry found for node -1 at org.apache.kafka.clients.ClusterConnectionStates.nodeState(ClusterConnectionStates.java:131) at org.apache.kafka.clients.ClusterConnectionStates.disconnected(ClusterConnectionStates.java:120) at org.apache.kafka.clients.NetworkClient.initiateConnect(NetworkClient.java:407) at org.apache.kafka.clients.NetworkClient.maybeUpdateMetadata(NetworkClient.java:393) at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:187) at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:184) at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:115) at java.lang.Thread.run(Thread.java:744) 2014-11-26 11:00:27.613 [kafka-producer-network-thread | error] ERROR org.apache.kafka.clients.producer.internals.Sender - Uncaught error in kafka producer I/O thread: java.lang.IllegalStateException: No entry found for node -3 at org.apache.kafka.clients.ClusterConnectionStates.nodeState(ClusterConnectionStates.java:131) at org.apache.kafka.clients.ClusterConnectionStates.disconnected(ClusterConnectionStates.java:120) at org.apache.kafka.clients.NetworkClient.initiateConnect(NetworkClient.java:407) at org.apache.kafka.clients.NetworkClient.maybeUpdateMetadata(NetworkClient.java:393) at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:187) at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:184) at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:115) at java.lang.Thread.run(Thread.java:744) {code} > [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.1.1, 0.8.2 > Reporter: Bhavesh Mistry > Assignee: Ewen Cheslack-Postava > Priority: Blocker > 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)