You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@kafka.apache.org by Pete Wright <pw...@rubiconproject.com> on 2015/06/19 01:08:02 UTC

noticing blocking of kafka-network-thread

Hi There,
while investigating higher than expected CPU utilization on one of our
Kafka brokers we noticed multiple instances of the kafka-network-thread
running in a BLOCKED state, all of whom are waiting for a single thread
to release a lock.

Here is an example from a stack trace:

(blocked thread)
"kafka-network-thread-9092-10" - Thread t@50
   java.lang.Thread.State: BLOCKED
        at sun.nio.ch.FileChannelImpl.size(FileChannelImpl.java:300)
        - waiting to lock <2bb6fb92> (a java.lang.Object) owned by
"kafka-network-thread-9092-13" t@53
        at sun.nio.ch.FileChannelImpl.transferTo(FileChannelImpl.java:565)
        at kafka.log.FileMessageSet.writeTo(FileMessageSet.scala:147)
        at kafka.api.PartitionDataSend.writeTo(FetchResponse.scala:70)
        at kafka.network.MultiSend.writeTo(Transmission.scala:101)
        at kafka.api.TopicDataSend.writeTo(FetchResponse.scala:125)
        at kafka.network.MultiSend.writeTo(Transmission.scala:101)
        at kafka.api.FetchResponseSend.writeTo(FetchResponse.scala:231)
        at kafka.network.Processor.write(SocketServer.scala:472)
        at kafka.network.Processor.run(SocketServer.scala:342)
        at java.lang.Thread.run(Thread.java:745)



(thread 53)
"kafka-network-thread-9092-13" - Thread t@53
   java.lang.Thread.State: RUNNABLE
        at sun.nio.ch.FileDispatcherImpl.size0(Native Method)
        at sun.nio.ch.FileDispatcherImpl.size(FileDispatcherImpl.java:84)
        at sun.nio.ch.FileChannelImpl.size(FileChannelImpl.java:309)
        - locked <2bb6fb92> (a java.lang.Object)
        at sun.nio.ch.FileChannelImpl.transferTo(FileChannelImpl.java:565)
        at kafka.log.FileMessageSet.writeTo(FileMessageSet.scala:147)
        at kafka.api.PartitionDataSend.writeTo(FetchResponse.scala:70)
        at kafka.network.MultiSend.writeTo(Transmission.scala:101)
        at kafka.api.TopicDataSend.writeTo(FetchResponse.scala:125)
        at kafka.network.MultiSend.writeTo(Transmission.scala:101)
        at kafka.api.FetchResponseSend.writeTo(FetchResponse.scala:231)
        at kafka.network.Processor.write(SocketServer.scala:472)
        at kafka.network.Processor.run(SocketServer.scala:342)
        at java.lang.Thread.run(Thread.java:745)

   Locked ownable synchronizers:
        - None



What is interesting is that:
1) we have seen the broker with high CPU utilization float around our
cluster.  I.e node 0 will have high load for a time, then node 2, etc.

2) when we do a thread dump on normally operating brokers we find no
BLOCKING threads

3) all nodes we capture thread dumps on seem to all get blocked by one
thread.

Has anyone else seen this, and if not - is there an easy way to
determine which object these threads are seeing contention on?

Thanks in advance!
-pete


-- 
Pete Wright
Lead Systems Architect
Rubicon Project
pwright@rubiconproject.com
310.309.9298