You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2022/03/11 08:58:10 UTC

[GitHub] [pulsar] vinz2714 opened a new issue #14660: Broker high cpu usage under light load

vinz2714 opened a new issue #14660:
URL: https://github.com/apache/pulsar/issues/14660


   **Describe the bug**
   We have 3 bookie and 2 zookeeper and 2 broker.
   we were a 2.4.x but due to error readTimeException we upgraded to 2.9.1. Recently after restarting a consumer, consumer was connected to the pulsar but no messages were pulled. (Consumer was creating a websocket connection in pull mode)
   We observed that the broker process cpu was beyond 600- 700 without any load and hence no messages are consumed even with consumer connected. 
   Ever post restart the broker issue is still the same . We downgraded to 2.8.2 but still have the same issue.
   
   **To Reproduce**
   Steps to reproduce the behavior:
   1: install 2.8.2 
   2: create 200-300 consumer
   
   **Expected behavior**
   Consumer should be able to consume messages post connection. 
   
   **Screenshots**
   `root@ip-10-48-26-13:/opt/jstack# ./profile.sh 213679 60 10
   Collecting 6 stack traces of process 213679
   ......
   Processing 6 stack traces...
   Runnable methods breakout
   -------------------------
   rank    time    name
   0       86.89%  io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
   1       86.89%  java.lang.Thread.run(java.base@11.0.14/Thread.java:829)
   2       68.85%  io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:986)
   3       68.85%  io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
   4       50.82%  io.netty.channel.epoll.Native.epollWait(Native.java:186)
   5       50.82%  io.netty.channel.epoll.EpollEventLoop.epollWaitNoTimerChange(EpollEventLoop.java:290)
   6       50.82%  io.netty.channel.epoll.Native.epollWait(Native Method)
   7       50.82%  io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:347)
   8       50.82%  io.netty.channel.epoll.Native.epollWait(Native.java:193)
   9       18.03%  java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@11.0.14/ThreadPoolExecutor.java:628)
   10      17.49%  java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.14/ThreadPoolExecutor.java:1128)
   11      16.39%  io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:351)
   12      16.39%  io.netty.channel.epoll.Native.epollWait0(Native Method)
   13      16.39%  io.netty.channel.epoll.Native.epollWait(Native.java:177)
   14      16.39%  io.netty.channel.epoll.EpollEventLoop.epollWait(EpollEventLoop.java:286)
   15      11.48%  sun.nio.ch.SelectorImpl.lockAndDoSelect(java.base@11.0.14/SelectorImpl.java:124)
   16      11.48%  org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:332)
   17      11.48%  sun.nio.ch.EPoll.wait(java.base@11.0.14/Native Method)
   18      11.48%  org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1290)
   19      11.48%  sun.nio.ch.SelectorImpl.select(java.base@11.0.14/SelectorImpl.java:136)
   20      10.93%  org.apache.bookkeeper.mledger.util.SafeRun$1.safeRun(SafeRun.java:32)
   21      10.93%  org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36)
   22      10.66%  org.apache.bookkeeper.mledger.impl.OpReadEntry$$Lambda$1987/0x000000084090ec40.run(Unknown Source)
   23      10.66%  org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers.readEntriesComplete(PersistentDispatcherMultipleConsumers.java:477)
   24      10.66%  org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.java:489)
   25      10.66%  org.apache.bookkeeper.mledger.impl.OpReadEntry.lambda$checkReadCompletion$2(OpReadEntry.java:156)
   26      8.74%   org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.asyncReadEntries(ManagedCursorImpl.java:611)
   27      8.74%   org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.asyncReadEntriesOrWait(ManagedCursorImpl.java:762)
   28      8.74%   org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers.readMoreEntries(PersistentDispatcherMultipleConsumers.java:269)
   29      5.19%   org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.asyncReadEntries(ManagedCursorImpl.java:628)
   30      5.19%   org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.asyncReadEntries(ManagedLedgerImpl.java:1712)
   31      4.92%   java.util.concurrent.CompletableFuture.thenAccept(java.base@11.0.14/CompletableFuture.java:2108)
   32      4.92%   org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl$$Lambda$1983/0x000000084090f840.accept(Unknown Source)
   33      4.92%   org.apache.bookkeeper.mledger.impl.OpReadEntry.checkReadCompletion(OpReadEntry.java:155)
   34      4.92%   java.util.concurrent.CompletableFuture.uniAcceptNow(java.base@11.0.14/CompletableFuture.java:753)
   35      4.92%   java.util.concurrent.ThreadPoolExecutor.execute(java.base@11.0.14/ThreadPoolExecutor.java:1347)
   36      4.92%   org.apache.bookkeeper.common.util.BoundedExecutorService.execute(BoundedExecutorService.java:89)
   37      4.92%   org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.internalReadFromLedger(ManagedLedgerImpl.java:1850)
   38      4.92%   org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.lambda$asyncReadEntries$16(ManagedLedgerImpl.java:1712)
   39      4.92%   org.apache.bookkeeper.common.util.OrderedExecutor.executeOrdered(OrderedExecutor.java:484)
   40      4.92%   org.apache.bookkeeper.common.util.OrderedExecutor$1.execute(OrderedExecutor.java:341)
   41      4.92%   com.google.common.util.concurrent.ForwardingExecutorService.execute(ForwardingExecutorService.java:99)
   42      4.92%   java.util.concurrent.CompletableFuture.uniAcceptStage(java.base@11.0.14/CompletableFuture.java:731)
   43      3.55%   org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.asyncReadEntries(ManagedCursorImpl.java:627)
   44      3.28%   org.eclipse.jetty.server.ServerConnector.accept(ServerConnector.java:388)
   45      3.28%   sun.nio.ch.ServerSocketChannelImpl.accept0(java.base@11.0.14/Native Method)
   46      3.28%   org.apache.pulsar.broker.web.PulsarServerConnector.accept(PulsarServerConnector.java:56)
   47      3.28%   sun.nio.ch.ServerSocketChannelImpl.accept(java.base@11.0.14/ServerSocketChannelImpl.java:533)
   48      3.28%   org.eclipse.jetty.server.AbstractConnector$Acceptor.run(AbstractConnector.java:702)
   49      3.28%   sun.nio.ch.ServerSocketChannelImpl.accept(java.base@11.0.14/ServerSocketChannelImpl.java:285)
   50      1.91%   org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers.readMoreEntries(PersistentDispatcherMultipleConsumers.java:228)
   51      1.64%   org.apache.bookkeeper.mledger.impl.OpReadEntry.create(OpReadEntry.java:50)
   52      1.64%   org.eclipse.jetty.util.thread.strategy.EatWhatYouKill.produce(EatWhatYouKill.java:137)
   53      1.64%   java.lang.ref.Reference$ReferenceHandler.run(java.base@11.0.14/Reference.java:213)
   54      1.64%   org.eclipse.jetty.io.ManagedSelector$SelectorProducer.select(ManagedSelector.java:606)
   55      1.64%   org.eclipse.jetty.io.ManagedSelector.select(ManagedSelector.java:190)
   56      1.64%   org.eclipse.jetty.util.thread.strategy.EatWhatYouKill.run(EatWhatYouKill.java:131)
   57      1.64%   io.netty.channel.nio.SelectedSelectionKeySetSelector.select(SelectedSelectionKeySetSelector.java:68)
   58      1.64%   org.eclipse.jetty.io.ManagedSelector$SelectorProducer.produce(ManagedSelector.java:543)
   59      1.64%   org.eclipse.jetty.util.thread.strategy.EatWhatYouKill.produceTask(EatWhatYouKill.java:362)
   60      1.64%   sun.nio.ch.SelectorImpl.select(java.base@11.0.14/SelectorImpl.java:141)
   61      1.64%   org.eclipse.jetty.io.ManagedSelector.nioSelect(ManagedSelector.java:183)
   62      1.64%   io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:460)
   63      1.64%   org.eclipse.jetty.util.thread.ReservedThreadExecutor$ReservedThread.run(ReservedThreadExecutor.java:386)
   64      1.64%   org.eclipse.jetty.io.ManagedSelector$$Lambda$444/0x000000084055ec40.run(Unknown Source)
   65      1.64%   org.eclipse.jetty.util.thread.strategy.EatWhatYouKill.tryProduce(EatWhatYouKill.java:173)
   66      1.64%   io.netty.channel.nio.NioEventLoop.select(NioEventLoop.java:813)
   67      1.64%   org.eclipse.jetty.util.thread.strategy.EatWhatYouKill.doProduce(EatWhatYouKill.java:186)
   68      1.64%   sun.nio.ch.EPollSelectorImpl.doSelect(java.base@11.0.14/EPollSelectorImpl.java:120)
   69      1.64%   io.netty.util.Recycler.get(Recycler.java:157)
   70      1.64%   java.lang.ref.Reference.processPendingReferences(java.base@11.0.14/Reference.java:241)
   71      1.64%   java.lang.ref.Reference.waitForReferencePendingList(java.base@11.0.14/Native Method)
   72      0.27%   java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.14/ThreadPoolExecutor.java:1137)
   73      0.27%   java.util.concurrent.CompletableFuture.exceptionally(java.base@11.0.14/CompletableFuture.java:2307)
   74      0.27%   java.util.concurrent.ThreadPoolExecutor.getTask(java.base@11.0.14/ThreadPoolExecutor.java:1054)
   75      0.27%   java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.14/ThreadPoolExecutor.java:1114)
   Blocked/Waiting methods breakout
   --------------------------------
   rank    time    name
   0       78.01%  java.lang.Thread.run(java.base@11.0.14/Thread.java:829)
   1       73.90%  io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
   2       37.44%  java.util.concurrent.LinkedBlockingQueue.take(java.base@11.0.14/LinkedBlockingQueue.java:433)
   3       21.00%  java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(java.base@11.0.14/AbstractQueuedSynchronizer.java:2081)
   4       21.00%  java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(java.base@11.0.14/ScheduledThreadPoolExecutor.java:1170)
   5       15.45%  java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(java.base@11.0.14/ScheduledThreadPoolExecutor.java:1177)
   6       15.45%  java.util.concurrent.locks.LockSupport.park(java.base@11.0.14/LockSupport.java:194)
   7       9.67%   java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(java.base@11.0.14/ScheduledThreadPoolExecutor.java:1182)
   8       9.67%   java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(java.base@11.0.14/AbstractQueuedSynchronizer.java:2123)
   9       4.11%   java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@11.0.14/ThreadPoolExecutor.java:628)
   10      4.11%   java.util.concurrent.ThreadPoolExecutor.getTask(java.base@11.0.14/ThreadPoolExecutor.java:1054)
   11      3.20%   org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:559)
   12      2.28%   io.netty.util.HashedWheelTimer$Worker.run(HashedWheelTimer.java:496)
   13      2.28%   java.lang.Thread.sleep(java.base@11.0.14/Native Method)
   14      2.28%   io.netty.util.HashedWheelTimer$Worker.waitForNextTick(HashedWheelTimer.java:600)
   15      1.83%   java.util.concurrent.ForkJoinWorkerThread.run(java.base@11.0.14/ForkJoinWorkerThread.java:183)
   16      1.83%   java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.14/ThreadPoolExecutor.java:1128)
   17      1.83%   java.util.concurrent.ForkJoinPool.runWorker(java.base@11.0.14/ForkJoinPool.java:1628)
   18      1.37%   java.util.concurrent.locks.LockSupport.parkNanos(java.base@11.0.14/LockSupport.java:234)
   19      1.37%   java.util.concurrent.SynchronousQueue.poll(java.base@11.0.14/SynchronousQueue.java:937)
   20      1.37%   java.util.concurrent.SynchronousQueue$TransferStack.transfer(java.base@11.0.14/SynchronousQueue.java:361)
   21      1.37%   java.util.concurrent.SynchronousQueue$TransferStack.awaitFulfill(java.base@11.0.14/SynchronousQueue.java:462)
   22      1.37%   java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(java.base@11.0.14/ScheduledThreadPoolExecutor.java:899)
   23      1.37%   java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.14/ThreadPoolExecutor.java:1114)
   24      1.37%   org.eclipse.jetty.util.thread.ReservedThreadExecutor$ReservedThread.reservedWait(ReservedThreadExecutor.java:312)
   25      1.37%   org.eclipse.jetty.util.thread.ReservedThreadExecutor$ReservedThread.run(ReservedThreadExecutor.java:377)
   26      0.46%   java.lang.Object.wait(java.base@11.0.14/Native Method)
   27      0.46%   org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl$$Lambda$179/0x0000000840298840.run(Unknown Source)
   28      0.46%   org.apache.pulsar.PulsarBrokerStarter$BrokerStarter.join(PulsarBrokerStarter.java:264)
   29      0.46%   jdk.internal.misc.InnocuousThread.run(java.base@11.0.14/InnocuousThread.java:161)
   30      0.46%   org.apache.pulsar.broker.PulsarService.waitUntilClosed(PulsarService.java:999)
   31      0.46%   java.lang.ref.ReferenceQueue.remove(java.base@11.0.14/ReferenceQueue.java:155)
   32      0.46%   org.apache.pulsar.PulsarBrokerStarter.main(PulsarBrokerStarter.java:337)
   33      0.46%   jdk.internal.ref.CleanerImpl.run(java.base@11.0.14/CleanerImpl.java:148)
   34      0.46%   jdk.internal.misc.Unsafe.park(java.base@11.0.14/Native Method)
   35      0.46%   java.lang.ref.ReferenceQueue.remove(java.base@11.0.14/ReferenceQueue.java:176)
   36      0.46%   java.util.concurrent.ForkJoinPool.runWorker(java.base@11.0.14/ForkJoinPool.java:1619)
   37      0.46%   org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl.cacheEvictionTask(ManagedLedgerFactoryImpl.java:244)
   38      0.46%   java.lang.ref.Finalizer$FinalizerThread.run(java.base@11.0.14/Finalizer.java:170)
   39      0.46%   java.util.concurrent.locks.LockSupport.parkUntil(java.base@11.0.14/LockSupp`
   
   **Desktop (please complete the following information):**
   Ubuntu
   
   **Additional context**
   Add any other context about the problem here.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] vinz2714 commented on issue #14660: Broker high cpu usage under light load

Posted by GitBox <gi...@apache.org>.
vinz2714 commented on issue #14660:
URL: https://github.com/apache/pulsar/issues/14660#issuecomment-1068798217


   > > We observed that the broker process cpu was beyond 600- 700 without any load
   > 
   > @vinz2714 What does `cpu beyond 600- 700` mean?
   
   we are running the broker on 8 core CPU machine. The Java process running the broker is consuming 6-7 core cpu upto 100%.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] Shawyeok commented on issue #14660: Broker high cpu usage under light load

Posted by GitBox <gi...@apache.org>.
Shawyeok commented on issue #14660:
URL: https://github.com/apache/pulsar/issues/14660#issuecomment-1068671222


   > We observed that the broker process cpu was beyond 600- 700 without any load
   
   @vinz2714  What does `cpu beyond 600- 700` mean?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] wuxuanqicn commented on issue #14660: Broker high cpu usage under light load

Posted by GitBox <gi...@apache.org>.
wuxuanqicn commented on issue #14660:
URL: https://github.com/apache/pulsar/issues/14660#issuecomment-1066666454


   look like same with #14015 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org