You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Jaikiran Pai <ja...@gmail.com> on 2015/09/17 20:13:35 UTC

Re: Unreasonably high CPU from Kafka (0.8.2.1)

Sending this to the dev list since the Kafka dev team might have more 
inputs on this one. Can someone please take a look at the issue noted 
below and whether the suggested change makes sense?

-Jaikiran
On Tuesday 15 September 2015 12:03 AM, Jaikiran Pai wrote:
> We have been using Kafka for a while now in one of dev projects. 
> Currently we have just 1 broker and 1 zookeeper instance. Almost every 
> day, Kafka "stalls" and we end up cleaning up the data/log folder of 
> Kafka and zookeeper and bring it up afresh. We haven't been able to 
> narrow down the issue yet.
>
> However, keeping aside that part for a while, we have been noticing 
> that even when the system/application is completely idle, the Kafka 
> process seems to take up unreasonably high CPU (10-15% constantly 
> shown in top command). We have taken multiple thread dumps and each of 
> them have this:
>
> "kafka-socket-acceptor" #24 prio=5 os_prio=0 tid=0x00007f62685d9000 
> nid=0x2d47 runnable [0x00007f6231464000]
>    java.lang.Thread.State: RUNNABLE
>     at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>     at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>     at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
>     at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>     - locked <0x00000000ca77a458> (a sun.nio.ch.Util$2)
>     - locked <0x00000000ca77a440> (a 
> java.util.Collections$UnmodifiableSet)
>     - locked <0x00000000ca774550> (a sun.nio.ch.EPollSelectorImpl)
>     at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>     at kafka.network.Acceptor.run(SocketServer.scala:215)
>     at java.lang.Thread.run(Thread.java:745)
>
> "kafka-network-thread-9092-2" #23 prio=5 os_prio=0 
> tid=0x00007f62685d6800 nid=0x2d46 runnable [0x00007f6231565000]
>    java.lang.Thread.State: RUNNABLE
>     at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>     at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>     at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
>     at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>     - locked <0x00000000ca77d050> (a sun.nio.ch.Util$2)
>     - locked <0x00000000ca77d038> (a 
> java.util.Collections$UnmodifiableSet)
>     - locked <0x00000000ca7745e0> (a sun.nio.ch.EPollSelectorImpl)
>     at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>     at kafka.network.Processor.run(SocketServer.scala:320)
>     at java.lang.Thread.run(Thread.java:745)
>
> "kafka-network-thread-9092-1" #22 prio=5 os_prio=0 
> tid=0x00007f62685c7800 nid=0x2d45 runnable [0x00007f6231666000]
>    java.lang.Thread.State: RUNNABLE
>     at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>     at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>     at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
>     at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>     - locked <0x00000000ca77e590> (a sun.nio.ch.Util$2)
>     - locked <0x00000000ca77e578> (a 
> java.util.Collections$UnmodifiableSet)
>     - locked <0x00000000ca7746b8> (a sun.nio.ch.EPollSelectorImpl)
>     at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>     at kafka.network.Processor.run(SocketServer.scala:320)
>     at java.lang.Thread.run(Thread.java:745)
>
> "kafka-network-thread-9092-0" #21 prio=5 os_prio=0 
> tid=0x00007f62685b9000 nid=0x2d44 runnable [0x00007f6231767000]
>    java.lang.Thread.State: RUNNABLE
>     at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>     at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>     at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
>     at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>     - locked <0x00000000ca77fbd0> (a sun.nio.ch.Util$2)
>     - locked <0x00000000ca77fbb8> (a 
> java.util.Collections$UnmodifiableSet)
>     - locked <0x00000000ca774790> (a sun.nio.ch.EPollSelectorImpl)
>     at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>     at kafka.network.Processor.run(SocketServer.scala:320)
>     at java.lang.Thread.run(Thread.java:745)
>
>
>
>
> Looking at the code of 0.8.2.1, this piece of code looks like 
> https://github.com/apache/kafka/blob/0.8.2.1/core/src/main/scala/kafka/network/SocketServer.scala#L314:
>
> while(isRunning) {
> ...
>     val ready = selector.select(300)
>     ...
>     if(ready > 0) {
>         ...
>     }
> ...
> }
>
> This looks like a (always) "busy" while loop when selector.select 
> returns 0. Could a sleep for a few milli. seconds help in this case? 
> Similar code is present in the Acceptor in that same file, which does 
> this exact thing. Would adding some small sleep in there help with 
> reducing the CPU usage when things are idle?
>
> -Jaikiran
>
>


Re: Unreasonably high CPU from Kafka (0.8.2.1)

Posted by Jaikiran Pai <ja...@gmail.com>.
Thanks Brock for that tip!

I ran the ps -L -p command and this is trimmed version of it:

PID   LWP TTY          TIME CMD
19524 19550 ?        01:11:38 java
19524 19551 ?        01:33:55 java
19524 19552 ?        01:38:32 java
19524 19557 ?        01:30:12 java

19524 is the process id and these 4 are the top consuming entries in 
that complete output. Checking their hex equivalents in the thread 
dumps, I see these 4 corresponding stacktraces:

"kafka-network-thread-9092-2" #23 prio=5 os_prio=0 
tid=0x00007f0b485c6000 nid=0x4c60 runnable [0x00007f0b307c1000]
    java.lang.Thread.State: RUNNABLE
     at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
     at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
     at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
     at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
     - locked <0x00000000ca7b8c58> (a sun.nio.ch.Util$2)
     - locked <0x00000000ca7b8c40> (a java.util.Collections$UnmodifiableSet)
     - locked <0x00000000ca7adb70> (a sun.nio.ch.EPollSelectorImpl)
     at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
     at kafka.network.Processor.run(SocketServer.scala:320)
     at java.lang.Thread.run(Thread.java:745)

"kafka-network-thread-9092-1" #22 prio=5 os_prio=0 
tid=0x00007f0b485b7800 nid=0x4c5f runnable [0x00007f0b308c2000]
    java.lang.Thread.State: RUNNABLE
     at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
     at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
     at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
     at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
     - locked <0x00000000ca7b9e00> (a sun.nio.ch.Util$2)
     - locked <0x00000000ca7b9de8> (a java.util.Collections$UnmodifiableSet)
     - locked <0x00000000ca7adc00> (a sun.nio.ch.EPollSelectorImpl)
     at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
     at kafka.network.Processor.run(SocketServer.scala:320)
     at java.lang.Thread.run(Thread.java:745)

"kafka-network-thread-9092-0" #21 prio=5 os_prio=0 
tid=0x00007f0b485a8800 nid=0x4c5e runnable [0x00007f0b309c3000]
    java.lang.Thread.State: RUNNABLE
     at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
     at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
     at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
     at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
     - locked <0x00000000ca7bb5e0> (a sun.nio.ch.Util$2)
     - locked <0x00000000ca7bb5c8> (a java.util.Collections$UnmodifiableSet)
     - locked <0x00000000ca7adc90> (a sun.nio.ch.EPollSelectorImpl)
     at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
     at kafka.network.Processor.run(SocketServer.scala:320)
     at java.lang.Thread.run(Thread.java:745)

"request-expiration-task" #28 prio=5 os_prio=0 tid=0x00007f0b48622800 
nid=0x4c65 runnable [0x00007f0b302bc000]
    java.lang.Thread.State: TIMED_WAITING (parking)
     at sun.misc.Unsafe.park(Native Method)
     - parking to wait for  <0x00000000ca7bb6e8> (a 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
     at 
java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
     at 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
     at java.util.concurrent.DelayQueue.poll(DelayQueue.java:273)
     at 
kafka.server.RequestPurgatory$ExpiredRequestReaper.pollExpired(RequestPurgatory.scala:317)
     at 
kafka.server.RequestPurgatory$ExpiredRequestReaper.run(RequestPurgatory.scala:273)
     at java.lang.Thread.run(Thread.java:745)


I took this thread dump and the ps listing a few minutes back, when that 
dev machine on which Kafka is running was idle (FYI - there are remote 
consumers to the system just "waiting" for new messages, but there's no 
message production to the topics)

-Jaikiran





On Friday 18 September 2015 04:34 AM, Brock Noland wrote:
> If you are on Linux, you can use the following approach.
>
> 1) Get the pid
> 2) Find the largest consumer of CPU via:
>
> ps -L -p <pid>
>
> 3) Convert the LWP column to hex and then look for that value in the
> thread dump on the "nid=0x.." field of each thread.
>
> Cheers!
> Brock
>
> On Thu, Sep 17, 2015 at 3:58 PM, Jaikiran Pai <ja...@gmail.com> wrote:
>> That's a good point, I missed that. Is there any other way to track down why
>> it would consume this high CPU when it's idle? I can send the thread dumps
>> (taken at intervals of a few seconds of each other), but each of them has
>> not much interesting (IMO) than these specific repeated stacktraces.
>>
>> -Jaikiran
>>
>> On Friday 18 September 2015 04:00 AM, Jun Rao wrote:
>>> The selector is probably not the issue. If there is no incoming traffic,
>>> selector.select(300) won't return until after 300ms.
>>>
>>> Thanks,
>>>
>>> Jun
>>>
>>> On Thu, Sep 17, 2015 at 1:13 PM, Jaikiran Pai <ja...@gmail.com>
>>> wrote:
>>>
>>>> Sending this to the dev list since the Kafka dev team might have more
>>>> inputs on this one. Can someone please take a look at the issue noted
>>>> below
>>>> and whether the suggested change makes sense?
>>>>
>>>> -Jaikiran
>>>>
>>>> On Tuesday 15 September 2015 12:03 AM, Jaikiran Pai wrote:
>>>>
>>>>> We have been using Kafka for a while now in one of dev projects.
>>>>> Currently we have just 1 broker and 1 zookeeper instance. Almost every
>>>>> day,
>>>>> Kafka "stalls" and we end up cleaning up the data/log folder of Kafka
>>>>> and
>>>>> zookeeper and bring it up afresh. We haven't been able to narrow down
>>>>> the
>>>>> issue yet.
>>>>>
>>>>> However, keeping aside that part for a while, we have been noticing that
>>>>> even when the system/application is completely idle, the Kafka process
>>>>> seems to take up unreasonably high CPU (10-15% constantly shown in top
>>>>> command). We have taken multiple thread dumps and each of them have
>>>>> this:
>>>>>
>>>>> "kafka-socket-acceptor" #24 prio=5 os_prio=0 tid=0x00007f62685d9000
>>>>> nid=0x2d47 runnable [0x00007f6231464000]
>>>>>      java.lang.Thread.State: RUNNABLE
>>>>>       at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>>       at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>>>>>       at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
>>>>>       at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>>>>>       - locked <0x00000000ca77a458> (a sun.nio.ch.Util$2)
>>>>>       - locked <0x00000000ca77a440> (a
>>>>> java.util.Collections$UnmodifiableSet)
>>>>>       - locked <0x00000000ca774550> (a sun.nio.ch.EPollSelectorImpl)
>>>>>       at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>>>>>       at kafka.network.Acceptor.run(SocketServer.scala:215)
>>>>>       at java.lang.Thread.run(Thread.java:745)
>>>>>
>>>>> "kafka-network-thread-9092-2" #23 prio=5 os_prio=0
>>>>> tid=0x00007f62685d6800
>>>>> nid=0x2d46 runnable [0x00007f6231565000]
>>>>>      java.lang.Thread.State: RUNNABLE
>>>>>       at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>>       at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>>>>>       at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
>>>>>       at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>>>>>       - locked <0x00000000ca77d050> (a sun.nio.ch.Util$2)
>>>>>       - locked <0x00000000ca77d038> (a
>>>>> java.util.Collections$UnmodifiableSet)
>>>>>       - locked <0x00000000ca7745e0> (a sun.nio.ch.EPollSelectorImpl)
>>>>>       at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>>>>>       at kafka.network.Processor.run(SocketServer.scala:320)
>>>>>       at java.lang.Thread.run(Thread.java:745)
>>>>>
>>>>> "kafka-network-thread-9092-1" #22 prio=5 os_prio=0
>>>>> tid=0x00007f62685c7800
>>>>> nid=0x2d45 runnable [0x00007f6231666000]
>>>>>      java.lang.Thread.State: RUNNABLE
>>>>>       at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>>       at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>>>>>       at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
>>>>>       at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>>>>>       - locked <0x00000000ca77e590> (a sun.nio.ch.Util$2)
>>>>>       - locked <0x00000000ca77e578> (a
>>>>> java.util.Collections$UnmodifiableSet)
>>>>>       - locked <0x00000000ca7746b8> (a sun.nio.ch.EPollSelectorImpl)
>>>>>       at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>>>>>       at kafka.network.Processor.run(SocketServer.scala:320)
>>>>>       at java.lang.Thread.run(Thread.java:745)
>>>>>
>>>>> "kafka-network-thread-9092-0" #21 prio=5 os_prio=0
>>>>> tid=0x00007f62685b9000
>>>>> nid=0x2d44 runnable [0x00007f6231767000]
>>>>>      java.lang.Thread.State: RUNNABLE
>>>>>       at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>>       at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>>>>>       at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
>>>>>       at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>>>>>       - locked <0x00000000ca77fbd0> (a sun.nio.ch.Util$2)
>>>>>       - locked <0x00000000ca77fbb8> (a
>>>>> java.util.Collections$UnmodifiableSet)
>>>>>       - locked <0x00000000ca774790> (a sun.nio.ch.EPollSelectorImpl)
>>>>>       at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>>>>>       at kafka.network.Processor.run(SocketServer.scala:320)
>>>>>       at java.lang.Thread.run(Thread.java:745)
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> Looking at the code of 0.8.2.1, this piece of code looks like
>>>>>
>>>>> https://github.com/apache/kafka/blob/0.8.2.1/core/src/main/scala/kafka/network/SocketServer.scala#L314
>>>>> :
>>>>>
>>>>> while(isRunning) {
>>>>> ...
>>>>>       val ready = selector.select(300)
>>>>>       ...
>>>>>       if(ready > 0) {
>>>>>           ...
>>>>>       }
>>>>> ...
>>>>> }
>>>>>
>>>>> This looks like a (always) "busy" while loop when selector.select
>>>>> returns
>>>>> 0. Could a sleep for a few milli. seconds help in this case? Similar
>>>>> code
>>>>> is present in the Acceptor in that same file, which does this exact
>>>>> thing.
>>>>> Would adding some small sleep in there help with reducing the CPU usage
>>>>> when things are idle?
>>>>>
>>>>> -Jaikiran
>>>>>
>>>>>
>>>>>


Re: Unreasonably high CPU from Kafka (0.8.2.1)

Posted by Brock Noland <br...@apache.org>.
If you are on Linux, you can use the following approach.

1) Get the pid
2) Find the largest consumer of CPU via:

ps -L -p <pid>

3) Convert the LWP column to hex and then look for that value in the
thread dump on the "nid=0x.." field of each thread.

Cheers!
Brock

On Thu, Sep 17, 2015 at 3:58 PM, Jaikiran Pai <ja...@gmail.com> wrote:
> That's a good point, I missed that. Is there any other way to track down why
> it would consume this high CPU when it's idle? I can send the thread dumps
> (taken at intervals of a few seconds of each other), but each of them has
> not much interesting (IMO) than these specific repeated stacktraces.
>
> -Jaikiran
>
> On Friday 18 September 2015 04:00 AM, Jun Rao wrote:
>>
>> The selector is probably not the issue. If there is no incoming traffic,
>> selector.select(300) won't return until after 300ms.
>>
>> Thanks,
>>
>> Jun
>>
>> On Thu, Sep 17, 2015 at 1:13 PM, Jaikiran Pai <ja...@gmail.com>
>> wrote:
>>
>>> Sending this to the dev list since the Kafka dev team might have more
>>> inputs on this one. Can someone please take a look at the issue noted
>>> below
>>> and whether the suggested change makes sense?
>>>
>>> -Jaikiran
>>>
>>> On Tuesday 15 September 2015 12:03 AM, Jaikiran Pai wrote:
>>>
>>>> We have been using Kafka for a while now in one of dev projects.
>>>> Currently we have just 1 broker and 1 zookeeper instance. Almost every
>>>> day,
>>>> Kafka "stalls" and we end up cleaning up the data/log folder of Kafka
>>>> and
>>>> zookeeper and bring it up afresh. We haven't been able to narrow down
>>>> the
>>>> issue yet.
>>>>
>>>> However, keeping aside that part for a while, we have been noticing that
>>>> even when the system/application is completely idle, the Kafka process
>>>> seems to take up unreasonably high CPU (10-15% constantly shown in top
>>>> command). We have taken multiple thread dumps and each of them have
>>>> this:
>>>>
>>>> "kafka-socket-acceptor" #24 prio=5 os_prio=0 tid=0x00007f62685d9000
>>>> nid=0x2d47 runnable [0x00007f6231464000]
>>>>     java.lang.Thread.State: RUNNABLE
>>>>      at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>      at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>>>>      at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
>>>>      at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>>>>      - locked <0x00000000ca77a458> (a sun.nio.ch.Util$2)
>>>>      - locked <0x00000000ca77a440> (a
>>>> java.util.Collections$UnmodifiableSet)
>>>>      - locked <0x00000000ca774550> (a sun.nio.ch.EPollSelectorImpl)
>>>>      at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>>>>      at kafka.network.Acceptor.run(SocketServer.scala:215)
>>>>      at java.lang.Thread.run(Thread.java:745)
>>>>
>>>> "kafka-network-thread-9092-2" #23 prio=5 os_prio=0
>>>> tid=0x00007f62685d6800
>>>> nid=0x2d46 runnable [0x00007f6231565000]
>>>>     java.lang.Thread.State: RUNNABLE
>>>>      at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>      at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>>>>      at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
>>>>      at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>>>>      - locked <0x00000000ca77d050> (a sun.nio.ch.Util$2)
>>>>      - locked <0x00000000ca77d038> (a
>>>> java.util.Collections$UnmodifiableSet)
>>>>      - locked <0x00000000ca7745e0> (a sun.nio.ch.EPollSelectorImpl)
>>>>      at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>>>>      at kafka.network.Processor.run(SocketServer.scala:320)
>>>>      at java.lang.Thread.run(Thread.java:745)
>>>>
>>>> "kafka-network-thread-9092-1" #22 prio=5 os_prio=0
>>>> tid=0x00007f62685c7800
>>>> nid=0x2d45 runnable [0x00007f6231666000]
>>>>     java.lang.Thread.State: RUNNABLE
>>>>      at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>      at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>>>>      at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
>>>>      at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>>>>      - locked <0x00000000ca77e590> (a sun.nio.ch.Util$2)
>>>>      - locked <0x00000000ca77e578> (a
>>>> java.util.Collections$UnmodifiableSet)
>>>>      - locked <0x00000000ca7746b8> (a sun.nio.ch.EPollSelectorImpl)
>>>>      at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>>>>      at kafka.network.Processor.run(SocketServer.scala:320)
>>>>      at java.lang.Thread.run(Thread.java:745)
>>>>
>>>> "kafka-network-thread-9092-0" #21 prio=5 os_prio=0
>>>> tid=0x00007f62685b9000
>>>> nid=0x2d44 runnable [0x00007f6231767000]
>>>>     java.lang.Thread.State: RUNNABLE
>>>>      at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>      at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>>>>      at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
>>>>      at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>>>>      - locked <0x00000000ca77fbd0> (a sun.nio.ch.Util$2)
>>>>      - locked <0x00000000ca77fbb8> (a
>>>> java.util.Collections$UnmodifiableSet)
>>>>      - locked <0x00000000ca774790> (a sun.nio.ch.EPollSelectorImpl)
>>>>      at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>>>>      at kafka.network.Processor.run(SocketServer.scala:320)
>>>>      at java.lang.Thread.run(Thread.java:745)
>>>>
>>>>
>>>>
>>>>
>>>> Looking at the code of 0.8.2.1, this piece of code looks like
>>>>
>>>> https://github.com/apache/kafka/blob/0.8.2.1/core/src/main/scala/kafka/network/SocketServer.scala#L314
>>>> :
>>>>
>>>> while(isRunning) {
>>>> ...
>>>>      val ready = selector.select(300)
>>>>      ...
>>>>      if(ready > 0) {
>>>>          ...
>>>>      }
>>>> ...
>>>> }
>>>>
>>>> This looks like a (always) "busy" while loop when selector.select
>>>> returns
>>>> 0. Could a sleep for a few milli. seconds help in this case? Similar
>>>> code
>>>> is present in the Acceptor in that same file, which does this exact
>>>> thing.
>>>> Would adding some small sleep in there help with reducing the CPU usage
>>>> when things are idle?
>>>>
>>>> -Jaikiran
>>>>
>>>>
>>>>
>

Re: Unreasonably high CPU from Kafka (0.8.2.1)

Posted by Jun Rao <ju...@confluent.io>.
You can enable profiling in tools like jvisualvm and see which methods
other select() show up high.

Thanks,

Jun

On Thu, Sep 17, 2015 at 5:58 PM, Jaikiran Pai <ja...@gmail.com>
wrote:

> That's a good point, I missed that. Is there any other way to track down
> why it would consume this high CPU when it's idle? I can send the thread
> dumps (taken at intervals of a few seconds of each other), but each of them
> has not much interesting (IMO) than these specific repeated stacktraces.
>
> -Jaikiran
>
> On Friday 18 September 2015 04:00 AM, Jun Rao wrote:
>
>> The selector is probably not the issue. If there is no incoming traffic,
>> selector.select(300) won't return until after 300ms.
>>
>> Thanks,
>>
>> Jun
>>
>> On Thu, Sep 17, 2015 at 1:13 PM, Jaikiran Pai <ja...@gmail.com>
>> wrote:
>>
>> Sending this to the dev list since the Kafka dev team might have more
>>> inputs on this one. Can someone please take a look at the issue noted
>>> below
>>> and whether the suggested change makes sense?
>>>
>>> -Jaikiran
>>>
>>> On Tuesday 15 September 2015 12:03 AM, Jaikiran Pai wrote:
>>>
>>> We have been using Kafka for a while now in one of dev projects.
>>>> Currently we have just 1 broker and 1 zookeeper instance. Almost every
>>>> day,
>>>> Kafka "stalls" and we end up cleaning up the data/log folder of Kafka
>>>> and
>>>> zookeeper and bring it up afresh. We haven't been able to narrow down
>>>> the
>>>> issue yet.
>>>>
>>>> However, keeping aside that part for a while, we have been noticing that
>>>> even when the system/application is completely idle, the Kafka process
>>>> seems to take up unreasonably high CPU (10-15% constantly shown in top
>>>> command). We have taken multiple thread dumps and each of them have
>>>> this:
>>>>
>>>> "kafka-socket-acceptor" #24 prio=5 os_prio=0 tid=0x00007f62685d9000
>>>> nid=0x2d47 runnable [0x00007f6231464000]
>>>>     java.lang.Thread.State: RUNNABLE
>>>>      at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>      at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>>>>      at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
>>>>      at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>>>>      - locked <0x00000000ca77a458> (a sun.nio.ch.Util$2)
>>>>      - locked <0x00000000ca77a440> (a
>>>> java.util.Collections$UnmodifiableSet)
>>>>      - locked <0x00000000ca774550> (a sun.nio.ch.EPollSelectorImpl)
>>>>      at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>>>>      at kafka.network.Acceptor.run(SocketServer.scala:215)
>>>>      at java.lang.Thread.run(Thread.java:745)
>>>>
>>>> "kafka-network-thread-9092-2" #23 prio=5 os_prio=0
>>>> tid=0x00007f62685d6800
>>>> nid=0x2d46 runnable [0x00007f6231565000]
>>>>     java.lang.Thread.State: RUNNABLE
>>>>      at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>      at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>>>>      at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
>>>>      at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>>>>      - locked <0x00000000ca77d050> (a sun.nio.ch.Util$2)
>>>>      - locked <0x00000000ca77d038> (a
>>>> java.util.Collections$UnmodifiableSet)
>>>>      - locked <0x00000000ca7745e0> (a sun.nio.ch.EPollSelectorImpl)
>>>>      at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>>>>      at kafka.network.Processor.run(SocketServer.scala:320)
>>>>      at java.lang.Thread.run(Thread.java:745)
>>>>
>>>> "kafka-network-thread-9092-1" #22 prio=5 os_prio=0
>>>> tid=0x00007f62685c7800
>>>> nid=0x2d45 runnable [0x00007f6231666000]
>>>>     java.lang.Thread.State: RUNNABLE
>>>>      at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>      at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>>>>      at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
>>>>      at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>>>>      - locked <0x00000000ca77e590> (a sun.nio.ch.Util$2)
>>>>      - locked <0x00000000ca77e578> (a
>>>> java.util.Collections$UnmodifiableSet)
>>>>      - locked <0x00000000ca7746b8> (a sun.nio.ch.EPollSelectorImpl)
>>>>      at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>>>>      at kafka.network.Processor.run(SocketServer.scala:320)
>>>>      at java.lang.Thread.run(Thread.java:745)
>>>>
>>>> "kafka-network-thread-9092-0" #21 prio=5 os_prio=0
>>>> tid=0x00007f62685b9000
>>>> nid=0x2d44 runnable [0x00007f6231767000]
>>>>     java.lang.Thread.State: RUNNABLE
>>>>      at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>      at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>>>>      at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
>>>>      at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>>>>      - locked <0x00000000ca77fbd0> (a sun.nio.ch.Util$2)
>>>>      - locked <0x00000000ca77fbb8> (a
>>>> java.util.Collections$UnmodifiableSet)
>>>>      - locked <0x00000000ca774790> (a sun.nio.ch.EPollSelectorImpl)
>>>>      at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>>>>      at kafka.network.Processor.run(SocketServer.scala:320)
>>>>      at java.lang.Thread.run(Thread.java:745)
>>>>
>>>>
>>>>
>>>>
>>>> Looking at the code of 0.8.2.1, this piece of code looks like
>>>>
>>>> https://github.com/apache/kafka/blob/0.8.2.1/core/src/main/scala/kafka/network/SocketServer.scala#L314
>>>> :
>>>>
>>>> while(isRunning) {
>>>> ...
>>>>      val ready = selector.select(300)
>>>>      ...
>>>>      if(ready > 0) {
>>>>          ...
>>>>      }
>>>> ...
>>>> }
>>>>
>>>> This looks like a (always) "busy" while loop when selector.select
>>>> returns
>>>> 0. Could a sleep for a few milli. seconds help in this case? Similar
>>>> code
>>>> is present in the Acceptor in that same file, which does this exact
>>>> thing.
>>>> Would adding some small sleep in there help with reducing the CPU usage
>>>> when things are idle?
>>>>
>>>> -Jaikiran
>>>>
>>>>
>>>>
>>>>
>

Re: Unreasonably high CPU from Kafka (0.8.2.1)

Posted by Jaikiran Pai <ja...@gmail.com>.
That's a good point, I missed that. Is there any other way to track down 
why it would consume this high CPU when it's idle? I can send the thread 
dumps (taken at intervals of a few seconds of each other), but each of 
them has not much interesting (IMO) than these specific repeated 
stacktraces.

-Jaikiran
On Friday 18 September 2015 04:00 AM, Jun Rao wrote:
> The selector is probably not the issue. If there is no incoming traffic,
> selector.select(300) won't return until after 300ms.
>
> Thanks,
>
> Jun
>
> On Thu, Sep 17, 2015 at 1:13 PM, Jaikiran Pai <ja...@gmail.com>
> wrote:
>
>> Sending this to the dev list since the Kafka dev team might have more
>> inputs on this one. Can someone please take a look at the issue noted below
>> and whether the suggested change makes sense?
>>
>> -Jaikiran
>>
>> On Tuesday 15 September 2015 12:03 AM, Jaikiran Pai wrote:
>>
>>> We have been using Kafka for a while now in one of dev projects.
>>> Currently we have just 1 broker and 1 zookeeper instance. Almost every day,
>>> Kafka "stalls" and we end up cleaning up the data/log folder of Kafka and
>>> zookeeper and bring it up afresh. We haven't been able to narrow down the
>>> issue yet.
>>>
>>> However, keeping aside that part for a while, we have been noticing that
>>> even when the system/application is completely idle, the Kafka process
>>> seems to take up unreasonably high CPU (10-15% constantly shown in top
>>> command). We have taken multiple thread dumps and each of them have this:
>>>
>>> "kafka-socket-acceptor" #24 prio=5 os_prio=0 tid=0x00007f62685d9000
>>> nid=0x2d47 runnable [0x00007f6231464000]
>>>     java.lang.Thread.State: RUNNABLE
>>>      at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>      at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>>>      at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
>>>      at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>>>      - locked <0x00000000ca77a458> (a sun.nio.ch.Util$2)
>>>      - locked <0x00000000ca77a440> (a
>>> java.util.Collections$UnmodifiableSet)
>>>      - locked <0x00000000ca774550> (a sun.nio.ch.EPollSelectorImpl)
>>>      at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>>>      at kafka.network.Acceptor.run(SocketServer.scala:215)
>>>      at java.lang.Thread.run(Thread.java:745)
>>>
>>> "kafka-network-thread-9092-2" #23 prio=5 os_prio=0 tid=0x00007f62685d6800
>>> nid=0x2d46 runnable [0x00007f6231565000]
>>>     java.lang.Thread.State: RUNNABLE
>>>      at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>      at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>>>      at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
>>>      at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>>>      - locked <0x00000000ca77d050> (a sun.nio.ch.Util$2)
>>>      - locked <0x00000000ca77d038> (a
>>> java.util.Collections$UnmodifiableSet)
>>>      - locked <0x00000000ca7745e0> (a sun.nio.ch.EPollSelectorImpl)
>>>      at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>>>      at kafka.network.Processor.run(SocketServer.scala:320)
>>>      at java.lang.Thread.run(Thread.java:745)
>>>
>>> "kafka-network-thread-9092-1" #22 prio=5 os_prio=0 tid=0x00007f62685c7800
>>> nid=0x2d45 runnable [0x00007f6231666000]
>>>     java.lang.Thread.State: RUNNABLE
>>>      at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>      at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>>>      at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
>>>      at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>>>      - locked <0x00000000ca77e590> (a sun.nio.ch.Util$2)
>>>      - locked <0x00000000ca77e578> (a
>>> java.util.Collections$UnmodifiableSet)
>>>      - locked <0x00000000ca7746b8> (a sun.nio.ch.EPollSelectorImpl)
>>>      at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>>>      at kafka.network.Processor.run(SocketServer.scala:320)
>>>      at java.lang.Thread.run(Thread.java:745)
>>>
>>> "kafka-network-thread-9092-0" #21 prio=5 os_prio=0 tid=0x00007f62685b9000
>>> nid=0x2d44 runnable [0x00007f6231767000]
>>>     java.lang.Thread.State: RUNNABLE
>>>      at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>      at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>>>      at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
>>>      at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>>>      - locked <0x00000000ca77fbd0> (a sun.nio.ch.Util$2)
>>>      - locked <0x00000000ca77fbb8> (a
>>> java.util.Collections$UnmodifiableSet)
>>>      - locked <0x00000000ca774790> (a sun.nio.ch.EPollSelectorImpl)
>>>      at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>>>      at kafka.network.Processor.run(SocketServer.scala:320)
>>>      at java.lang.Thread.run(Thread.java:745)
>>>
>>>
>>>
>>>
>>> Looking at the code of 0.8.2.1, this piece of code looks like
>>> https://github.com/apache/kafka/blob/0.8.2.1/core/src/main/scala/kafka/network/SocketServer.scala#L314
>>> :
>>>
>>> while(isRunning) {
>>> ...
>>>      val ready = selector.select(300)
>>>      ...
>>>      if(ready > 0) {
>>>          ...
>>>      }
>>> ...
>>> }
>>>
>>> This looks like a (always) "busy" while loop when selector.select returns
>>> 0. Could a sleep for a few milli. seconds help in this case? Similar code
>>> is present in the Acceptor in that same file, which does this exact thing.
>>> Would adding some small sleep in there help with reducing the CPU usage
>>> when things are idle?
>>>
>>> -Jaikiran
>>>
>>>
>>>


Re: Unreasonably high CPU from Kafka (0.8.2.1)

Posted by Jun Rao <ju...@confluent.io>.
The selector is probably not the issue. If there is no incoming traffic,
selector.select(300) won't return until after 300ms.

Thanks,

Jun

On Thu, Sep 17, 2015 at 1:13 PM, Jaikiran Pai <ja...@gmail.com>
wrote:

> Sending this to the dev list since the Kafka dev team might have more
> inputs on this one. Can someone please take a look at the issue noted below
> and whether the suggested change makes sense?
>
> -Jaikiran
>
> On Tuesday 15 September 2015 12:03 AM, Jaikiran Pai wrote:
>
>> We have been using Kafka for a while now in one of dev projects.
>> Currently we have just 1 broker and 1 zookeeper instance. Almost every day,
>> Kafka "stalls" and we end up cleaning up the data/log folder of Kafka and
>> zookeeper and bring it up afresh. We haven't been able to narrow down the
>> issue yet.
>>
>> However, keeping aside that part for a while, we have been noticing that
>> even when the system/application is completely idle, the Kafka process
>> seems to take up unreasonably high CPU (10-15% constantly shown in top
>> command). We have taken multiple thread dumps and each of them have this:
>>
>> "kafka-socket-acceptor" #24 prio=5 os_prio=0 tid=0x00007f62685d9000
>> nid=0x2d47 runnable [0x00007f6231464000]
>>    java.lang.Thread.State: RUNNABLE
>>     at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>     at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>>     at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
>>     at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>>     - locked <0x00000000ca77a458> (a sun.nio.ch.Util$2)
>>     - locked <0x00000000ca77a440> (a
>> java.util.Collections$UnmodifiableSet)
>>     - locked <0x00000000ca774550> (a sun.nio.ch.EPollSelectorImpl)
>>     at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>>     at kafka.network.Acceptor.run(SocketServer.scala:215)
>>     at java.lang.Thread.run(Thread.java:745)
>>
>> "kafka-network-thread-9092-2" #23 prio=5 os_prio=0 tid=0x00007f62685d6800
>> nid=0x2d46 runnable [0x00007f6231565000]
>>    java.lang.Thread.State: RUNNABLE
>>     at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>     at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>>     at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
>>     at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>>     - locked <0x00000000ca77d050> (a sun.nio.ch.Util$2)
>>     - locked <0x00000000ca77d038> (a
>> java.util.Collections$UnmodifiableSet)
>>     - locked <0x00000000ca7745e0> (a sun.nio.ch.EPollSelectorImpl)
>>     at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>>     at kafka.network.Processor.run(SocketServer.scala:320)
>>     at java.lang.Thread.run(Thread.java:745)
>>
>> "kafka-network-thread-9092-1" #22 prio=5 os_prio=0 tid=0x00007f62685c7800
>> nid=0x2d45 runnable [0x00007f6231666000]
>>    java.lang.Thread.State: RUNNABLE
>>     at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>     at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>>     at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
>>     at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>>     - locked <0x00000000ca77e590> (a sun.nio.ch.Util$2)
>>     - locked <0x00000000ca77e578> (a
>> java.util.Collections$UnmodifiableSet)
>>     - locked <0x00000000ca7746b8> (a sun.nio.ch.EPollSelectorImpl)
>>     at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>>     at kafka.network.Processor.run(SocketServer.scala:320)
>>     at java.lang.Thread.run(Thread.java:745)
>>
>> "kafka-network-thread-9092-0" #21 prio=5 os_prio=0 tid=0x00007f62685b9000
>> nid=0x2d44 runnable [0x00007f6231767000]
>>    java.lang.Thread.State: RUNNABLE
>>     at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>     at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>>     at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
>>     at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>>     - locked <0x00000000ca77fbd0> (a sun.nio.ch.Util$2)
>>     - locked <0x00000000ca77fbb8> (a
>> java.util.Collections$UnmodifiableSet)
>>     - locked <0x00000000ca774790> (a sun.nio.ch.EPollSelectorImpl)
>>     at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>>     at kafka.network.Processor.run(SocketServer.scala:320)
>>     at java.lang.Thread.run(Thread.java:745)
>>
>>
>>
>>
>> Looking at the code of 0.8.2.1, this piece of code looks like
>> https://github.com/apache/kafka/blob/0.8.2.1/core/src/main/scala/kafka/network/SocketServer.scala#L314
>> :
>>
>> while(isRunning) {
>> ...
>>     val ready = selector.select(300)
>>     ...
>>     if(ready > 0) {
>>         ...
>>     }
>> ...
>> }
>>
>> This looks like a (always) "busy" while loop when selector.select returns
>> 0. Could a sleep for a few milli. seconds help in this case? Similar code
>> is present in the Acceptor in that same file, which does this exact thing.
>> Would adding some small sleep in there help with reducing the CPU usage
>> when things are idle?
>>
>> -Jaikiran
>>
>>
>>
>

Re: Unreasonably high CPU from Kafka (0.8.2.1)

Posted by Jun Rao <ju...@confluent.io>.
The selector is probably not the issue. If there is no incoming traffic,
selector.select(300) won't return until after 300ms.

Thanks,

Jun

On Thu, Sep 17, 2015 at 1:13 PM, Jaikiran Pai <ja...@gmail.com>
wrote:

> Sending this to the dev list since the Kafka dev team might have more
> inputs on this one. Can someone please take a look at the issue noted below
> and whether the suggested change makes sense?
>
> -Jaikiran
>
> On Tuesday 15 September 2015 12:03 AM, Jaikiran Pai wrote:
>
>> We have been using Kafka for a while now in one of dev projects.
>> Currently we have just 1 broker and 1 zookeeper instance. Almost every day,
>> Kafka "stalls" and we end up cleaning up the data/log folder of Kafka and
>> zookeeper and bring it up afresh. We haven't been able to narrow down the
>> issue yet.
>>
>> However, keeping aside that part for a while, we have been noticing that
>> even when the system/application is completely idle, the Kafka process
>> seems to take up unreasonably high CPU (10-15% constantly shown in top
>> command). We have taken multiple thread dumps and each of them have this:
>>
>> "kafka-socket-acceptor" #24 prio=5 os_prio=0 tid=0x00007f62685d9000
>> nid=0x2d47 runnable [0x00007f6231464000]
>>    java.lang.Thread.State: RUNNABLE
>>     at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>     at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>>     at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
>>     at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>>     - locked <0x00000000ca77a458> (a sun.nio.ch.Util$2)
>>     - locked <0x00000000ca77a440> (a
>> java.util.Collections$UnmodifiableSet)
>>     - locked <0x00000000ca774550> (a sun.nio.ch.EPollSelectorImpl)
>>     at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>>     at kafka.network.Acceptor.run(SocketServer.scala:215)
>>     at java.lang.Thread.run(Thread.java:745)
>>
>> "kafka-network-thread-9092-2" #23 prio=5 os_prio=0 tid=0x00007f62685d6800
>> nid=0x2d46 runnable [0x00007f6231565000]
>>    java.lang.Thread.State: RUNNABLE
>>     at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>     at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>>     at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
>>     at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>>     - locked <0x00000000ca77d050> (a sun.nio.ch.Util$2)
>>     - locked <0x00000000ca77d038> (a
>> java.util.Collections$UnmodifiableSet)
>>     - locked <0x00000000ca7745e0> (a sun.nio.ch.EPollSelectorImpl)
>>     at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>>     at kafka.network.Processor.run(SocketServer.scala:320)
>>     at java.lang.Thread.run(Thread.java:745)
>>
>> "kafka-network-thread-9092-1" #22 prio=5 os_prio=0 tid=0x00007f62685c7800
>> nid=0x2d45 runnable [0x00007f6231666000]
>>    java.lang.Thread.State: RUNNABLE
>>     at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>     at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>>     at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
>>     at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>>     - locked <0x00000000ca77e590> (a sun.nio.ch.Util$2)
>>     - locked <0x00000000ca77e578> (a
>> java.util.Collections$UnmodifiableSet)
>>     - locked <0x00000000ca7746b8> (a sun.nio.ch.EPollSelectorImpl)
>>     at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>>     at kafka.network.Processor.run(SocketServer.scala:320)
>>     at java.lang.Thread.run(Thread.java:745)
>>
>> "kafka-network-thread-9092-0" #21 prio=5 os_prio=0 tid=0x00007f62685b9000
>> nid=0x2d44 runnable [0x00007f6231767000]
>>    java.lang.Thread.State: RUNNABLE
>>     at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>     at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>>     at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
>>     at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>>     - locked <0x00000000ca77fbd0> (a sun.nio.ch.Util$2)
>>     - locked <0x00000000ca77fbb8> (a
>> java.util.Collections$UnmodifiableSet)
>>     - locked <0x00000000ca774790> (a sun.nio.ch.EPollSelectorImpl)
>>     at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>>     at kafka.network.Processor.run(SocketServer.scala:320)
>>     at java.lang.Thread.run(Thread.java:745)
>>
>>
>>
>>
>> Looking at the code of 0.8.2.1, this piece of code looks like
>> https://github.com/apache/kafka/blob/0.8.2.1/core/src/main/scala/kafka/network/SocketServer.scala#L314
>> :
>>
>> while(isRunning) {
>> ...
>>     val ready = selector.select(300)
>>     ...
>>     if(ready > 0) {
>>         ...
>>     }
>> ...
>> }
>>
>> This looks like a (always) "busy" while loop when selector.select returns
>> 0. Could a sleep for a few milli. seconds help in this case? Similar code
>> is present in the Acceptor in that same file, which does this exact thing.
>> Would adding some small sleep in there help with reducing the CPU usage
>> when things are idle?
>>
>> -Jaikiran
>>
>>
>>
>