[ 
https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14223571#comment-14223571
 ] 

Bhavesh Mistry commented on KAFKA-1642:
---------------------------------------

Here is exact steps how to reproducer it bug: (Must have demon program 
continuously running).

1)  Start with happy Situation where all borkers are up everything is running 
fine.  And verify all top -pid JAVA_PID and your kit (kafka network threads  
are taking less than 4% CPU).
2)  Shutdown network (turn off network or pull the eth0 cable)  wait for while 
and you will see that CPU spike to 325% under top  (if you have 4 producer) and 
verify your kit is showing 25% CPU consumption for for each Kafka io thread.
3) Connect back the network ( Spike will still be there but CPU after while 
come down to 100% or so ) and remain connected for while.  
4) again simulate network failure (to simulate network instability) repeat 
steps again 1 to 4 but wait for 10 or so minutes in between and you will see 
the trends of CPU spike along with above exception. 
java.lang.IllegalStateException: No entry found for node -2

Also, I see that Kafka is logging excessively when network is down (your kit 
shows it is taking more CPU Cycle  as compare  to normal)

Thanks,
Bhavesh 

> [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)

Reply via email to