[jira] [Comment Edited] (KAFKA-7925) Constant 100% cpu usage by all kafka brokers

2019-03-05 Thread Abhi (JIRA)


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

Abhi edited comment on KAFKA-7925 at 3/5/19 10:38 AM:
--

Hi, I deployed the patch in my setup but now I am getting below exception when 
trying to publish messages on 40 topics using same producer. This exception is 
received on client side. I did not see any error or warnings around this time 
(2019-03-05 04:16:25) in server logs.

[2019-03-05 04:16:25,146] ERROR Uncaught exception in thread 
'kafka-producer-network-thread | test_prod': 
(org.apache.kafka.common.utils.KafkaThread)
common.util.ApplicationDeath: 
org.apache.kafka.common.errors.UnknownServerException: The server experienced 
an unexpected error when processing the request.
at 
xxx.kafka.test.KafkaTestProducer$ProducerCallback.onCompletion(KafkaTestProducer.java:475)
at 
org.apache.kafka.clients.producer.KafkaProducer$InterceptorCallback.onCompletion(KafkaProducer.java:1304)
at 
org.apache.kafka.clients.producer.internals.ProducerBatch.completeFutureAndFireCallbacks(ProducerBatch.java:230)
at 
org.apache.kafka.clients.producer.internals.ProducerBatch.done(ProducerBatch.java:196)
at 
org.apache.kafka.clients.producer.internals.Sender.failBatch(Sender.java:717)
at 
org.apache.kafka.clients.producer.internals.Sender.failBatch(Sender.java:685)
at 
org.apache.kafka.clients.producer.internals.Sender.completeBatch(Sender.java:635)
at 
org.apache.kafka.clients.producer.internals.Sender.handleProduceResponse(Sender.java:557)
at 
org.apache.kafka.clients.producer.internals.Sender.access$100(Sender.java:74)
at 
org.apache.kafka.clients.producer.internals.Sender$1.onComplete(Sender.java:786)
at 
org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:109)
at 
org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:557)
at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:549)
at 
org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:311)
at 
org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:235)
at java.base/java.lang.Thread.run(Thread.java:834)
Caused by: org.apache.kafka.common.errors.UnknownServerException: The server 
experienced an unexpected error when processing the request.
[2019-03-05 04:17:25,144] DEBUG [Producer clientId=test_prod] Exception 
occurred during message send: (org.apache.kafka.clients.producer.KafkaProducer)

On a side note, after deploying this patch, I also observed lot of connection 
disconnects:

[2019-03-05 01:23:39,553] DEBUG [SocketServer brokerId=1] Connection with 
/10.219.26.10 disconnected (org.apache.kafka.common.network.Selector)
java.io.EOFException
at 
org.apache.kafka.common.network.NetworkReceive.readFrom(NetworkReceive.java:96)
at 
org.apache.kafka.common.network.KafkaChannel.receive(KafkaChannel.java:424)
at 
org.apache.kafka.common.network.KafkaChannel.read(KafkaChannel.java:385)
at 
org.apache.kafka.common.network.Selector.attemptRead(Selector.java:640)
at 
org.apache.kafka.common.network.Selector.pollSelectionKeys(Selector.java:561)
at org.apache.kafka.common.network.Selector.poll(Selector.java:472)
at kafka.network.Processor.poll(SocketServer.scala:830)
at kafka.network.Processor.run(SocketServer.scala:730)
at java.base/java.lang.Thread.run(Thread.java:834)
[2019-03-05 01:23:39,553] DEBUG [SocketServer brokerId=1] Connection with 
/10.219.26.10 disconnected (org.apache.kafka.common.network.Selector)
java.io.EOFException
at 
org.apache.kafka.common.network.NetworkReceive.readFrom(NetworkReceive.java:96)
at 
org.apache.kafka.common.network.KafkaChannel.receive(KafkaChannel.java:424)
at 
org.apache.kafka.common.network.KafkaChannel.read(KafkaChannel.java:385)
at 
org.apache.kafka.common.network.Selector.attemptRead(Selector.java:640)
at 
org.apache.kafka.common.network.Selector.pollSelectionKeys(Selector.java:561)
at org.apache.kafka.common.network.Selector.poll(Selector.java:472)
at kafka.network.Processor.poll(SocketServer.scala:830)
at kafka.network.Processor.run(SocketServer.scala:730)
at java.base/java.lang.Thread.run(Thread.java:834)

[2019-03-05 01:27:57,386] DEBUG [Controller id=1, targetBrokerId=4] Connection 
with mwkafka-prod-02.dr.xxx.com/10.218.247.23 disconnected 
(org.apache.kafka.common.network.Selector)
java.io.EOFException
at 
org.apache.kafka.common.network.NetworkReceive.readFrom(NetworkReceive.java:96)
at 
org.apache.kafka.common.network.KafkaChannel.receive(KafkaChannel.java:424)
at 
org.apache.kafka.common.network.KafkaChannel.read(KafkaChannel.java:385)
at 
org.apache.kafka.common.net

[jira] [Comment Edited] (KAFKA-7925) Constant 100% cpu usage by all kafka brokers

2019-03-05 Thread Abhi (JIRA)


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

Abhi edited comment on KAFKA-7925 at 3/5/19 10:35 AM:
--

Hi, I deployed the patch in my setup but now I am getting below exception when 
trying to publish messages. This exception is received on client side. I did 
not see any error or warnings around this time (2019-03-05 04:16:25) in server 
logs.

[2019-03-05 04:16:25,146] ERROR Uncaught exception in thread 
'kafka-producer-network-thread | test_prod': 
(org.apache.kafka.common.utils.KafkaThread)
common.util.ApplicationDeath: 
org.apache.kafka.common.errors.UnknownServerException: The server experienced 
an unexpected error when processing the request.
at 
xxx.kafka.test.KafkaTestProducer$ProducerCallback.onCompletion(KafkaTestProducer.java:475)
at 
org.apache.kafka.clients.producer.KafkaProducer$InterceptorCallback.onCompletion(KafkaProducer.java:1304)
at 
org.apache.kafka.clients.producer.internals.ProducerBatch.completeFutureAndFireCallbacks(ProducerBatch.java:230)
at 
org.apache.kafka.clients.producer.internals.ProducerBatch.done(ProducerBatch.java:196)
at 
org.apache.kafka.clients.producer.internals.Sender.failBatch(Sender.java:717)
at 
org.apache.kafka.clients.producer.internals.Sender.failBatch(Sender.java:685)
at 
org.apache.kafka.clients.producer.internals.Sender.completeBatch(Sender.java:635)
at 
org.apache.kafka.clients.producer.internals.Sender.handleProduceResponse(Sender.java:557)
at 
org.apache.kafka.clients.producer.internals.Sender.access$100(Sender.java:74)
at 
org.apache.kafka.clients.producer.internals.Sender$1.onComplete(Sender.java:786)
at 
org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:109)
at 
org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:557)
at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:549)
at 
org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:311)
at 
org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:235)
at java.base/java.lang.Thread.run(Thread.java:834)
Caused by: org.apache.kafka.common.errors.UnknownServerException: The server 
experienced an unexpected error when processing the request.
[2019-03-05 04:17:25,144] DEBUG [Producer clientId=test_prod] Exception 
occurred during message send: (org.apache.kafka.clients.producer.KafkaProducer)

On a side note, after deploying this patch, I also observed lot of connection 
disconnects:

[2019-03-05 01:23:39,553] DEBUG [SocketServer brokerId=1] Connection with 
/10.219.26.10 disconnected (org.apache.kafka.common.network.Selector)
java.io.EOFException
at 
org.apache.kafka.common.network.NetworkReceive.readFrom(NetworkReceive.java:96)
at 
org.apache.kafka.common.network.KafkaChannel.receive(KafkaChannel.java:424)
at 
org.apache.kafka.common.network.KafkaChannel.read(KafkaChannel.java:385)
at 
org.apache.kafka.common.network.Selector.attemptRead(Selector.java:640)
at 
org.apache.kafka.common.network.Selector.pollSelectionKeys(Selector.java:561)
at org.apache.kafka.common.network.Selector.poll(Selector.java:472)
at kafka.network.Processor.poll(SocketServer.scala:830)
at kafka.network.Processor.run(SocketServer.scala:730)
at java.base/java.lang.Thread.run(Thread.java:834)
[2019-03-05 01:23:39,553] DEBUG [SocketServer brokerId=1] Connection with 
/10.219.26.10 disconnected (org.apache.kafka.common.network.Selector)
java.io.EOFException
at 
org.apache.kafka.common.network.NetworkReceive.readFrom(NetworkReceive.java:96)
at 
org.apache.kafka.common.network.KafkaChannel.receive(KafkaChannel.java:424)
at 
org.apache.kafka.common.network.KafkaChannel.read(KafkaChannel.java:385)
at 
org.apache.kafka.common.network.Selector.attemptRead(Selector.java:640)
at 
org.apache.kafka.common.network.Selector.pollSelectionKeys(Selector.java:561)
at org.apache.kafka.common.network.Selector.poll(Selector.java:472)
at kafka.network.Processor.poll(SocketServer.scala:830)
at kafka.network.Processor.run(SocketServer.scala:730)
at java.base/java.lang.Thread.run(Thread.java:834)

[2019-03-05 01:27:57,386] DEBUG [Controller id=1, targetBrokerId=4] Connection 
with mwkafka-prod-02.dr.xxx.com/10.218.247.23 disconnected 
(org.apache.kafka.common.network.Selector)
java.io.EOFException
at 
org.apache.kafka.common.network.NetworkReceive.readFrom(NetworkReceive.java:96)
at 
org.apache.kafka.common.network.KafkaChannel.receive(KafkaChannel.java:424)
at 
org.apache.kafka.common.network.KafkaChannel.read(KafkaChannel.java:385)
at 
org.apache.kafka.common.network.Selector.attemptRead(Selecto

[jira] [Comment Edited] (KAFKA-7925) Constant 100% cpu usage by all kafka brokers

2019-03-05 Thread Abhi (JIRA)


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

Abhi edited comment on KAFKA-7925 at 3/5/19 1:54 PM:
-

[~rsivaram]
Did you run a clean build using the instructions Manikumar posted above? We 
want to make sure that all the jars you are running with came from that build 
to avoid NoClassDefFoundError. The client exceptions could be related to that.
>> All brokers were using JARs from [~omkreddy]'s build (using the 
>> instructions). There was no NoClassDefFoundError at the startup. Note that 
>> only one broker saw these exceptions and they all use same configuration and 
>> jars

>>Were all the brokers running with the build including the PR for a day? And 
>>during this time, were the clients always failing? Are the clients also 
>>running with the build including the PR?
Yes the brokers ran fine with PR build. The client doesn't always fail. When I 
run producer with 1 or 2 topics for some time, it works as expected but when i 
go to 40 topics, it fails after sending messages to first 15 topics (this 
happens sequentially).

The NoClassDefFoundError  exceptions went away with a restart of that 
particular broker.

I will give this another go just to make sure no other issue is affecting the 
observations.



was (Author: xabhi):
[~rsivaram]
Did you run a clean build using the instructions Manikumar posted above? We 
want to make sure that all the jars you are running with came from that build 
to avoid NoClassDefFoundError. The client exceptions could be related to that.
>> All brokers were using JARs from [~omkreddy]'s build (using the 
>> instructions). There was no NoClassDefFoundError at the startup. Note that 
>> only one broker saw these exceptions and they all use same configuration and 
>> jars

>>Were all the brokers running with the build including the PR for a day? And 
>>during this time, were the clients always failing? Are the clients also 
>>running with the build including the PR?
Yes the brokers ran fine with PR build. The client doesn't always fail. When I 
run producer with 1 or 2 topics for some time, it works as expected but when i 
go to 40 topics, it fails after sending messages to first 15 topics (this 
happens sequentially).

I can give this another go if you want.


> Constant 100% cpu usage by all kafka brokers
> 
>
> Key: KAFKA-7925
> URL: https://issues.apache.org/jira/browse/KAFKA-7925
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 2.1.0, 2.1.1
> Environment: Java 11, Kafka v2.1.0, Kafka v2.1.1
>Reporter: Abhi
>Priority: Critical
> Attachments: threadump20190212.txt
>
>
> Hi,
> I am seeing constant 100% cpu usage on all brokers in our kafka cluster even 
> without any clients connected to any broker.
> This is a bug that we have seen multiple times in our kafka setup that is not 
> yet open to clients. It is becoming a blocker for our deployment now.
> I am seeing lot of connections to other brokers in CLOSE_WAIT state (see 
> below). In thread usage, I am seeing these threads 
> 'kafka-network-thread-6-ListenerName(SASL_PLAINTEXT)-SASL_PLAINTEXT-0,kafka-network-thread-6-ListenerName(SASL_PLAINTEXT)-SASL_PLAINTEXT-1,kafka-network-thread-6-ListenerName(SASL_PLAINTEXT)-SASL_PLAINTEXT-2'
>  taking up more than 90% of the cpu time in a 60s interval.
> I have attached a thread dump of one of the brokers in the cluster.
> *Java version:*
> openjdk 11.0.2 2019-01-15
> OpenJDK Runtime Environment 18.9 (build 11.0.2+9)
> OpenJDK 64-Bit Server VM 18.9 (build 11.0.2+9, mixed mode)
> *Kafka verison:* v2.1.0
>  
> *connections:*
> java 144319 kafkagod 88u IPv4 3063266 0t0 TCP *:35395 (LISTEN)
> java 144319 kafkagod 89u IPv4 3063267 0t0 TCP *:9144 (LISTEN)
> java 144319 kafkagod 104u IPv4 3064219 0t0 TCP 
> mwkafka-prod-02.tbd:47292->mwkafka-zk-prod-05.tbd:2181 (ESTABLISHED)
> java 144319 kafkagod 2003u IPv4 3055115 0t0 TCP *:9092 (LISTEN)
> java 144319 kafkagod 2013u IPv4 7220110 0t0 TCP 
> mwkafka-prod-02.tbd:60724->mwkafka-zk-prod-04.dr:2181 (ESTABLISHED)
> java 144319 kafkagod 2020u IPv4 30012904 0t0 TCP 
> mwkafka-prod-02.tbd:38988->mwkafka-prod-02.nyc:9092 (ESTABLISHED)
> java 144319 kafkagod 2021u IPv4 30012961 0t0 TCP 
> mwkafka-prod-02.tbd:58420->mwkafka-prod-01.nyc:9092 (ESTABLISHED)
> java 144319 kafkagod 2027u IPv4 30015723 0t0 TCP 
> mwkafka-prod-02.tbd:58398->mwkafka-prod-01.nyc:9092 (ESTABLISHED)
> java 144319 kafkagod 2028u IPv4 30015630 0t0 TCP 
> mwkafka-prod-02.tbd:36248->mwkafka-prod-02.dr:9092 (ESTABLISHED)
> java 144319 kafkagod 2030u IPv4 30015726 0t0 TCP 
> mwkafka-prod-02.tbd:39012->mwkafka-prod-02.nyc:9092 (ESTABLISHED)
> java 144319 kafkagod 2031u IPv4 30013619 0t0 TCP 
> mwkafka-prod-02.tbd:38986->mwkafka-prod-