pulsar-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From GitBox <...@apache.org>
Subject [GitHub] massakam edited a comment on issue #3630: Broker can not respond to client requests
Date Wed, 20 Feb 2019 10:50:52 GMT
massakam edited a comment on issue #3630: Broker can not respond to client requests
URL: https://github.com/apache/pulsar/issues/3630#issuecomment-465458291
 
 
   This is a thread dump acquired just before restarting the broker:
   [threaddump.txt](https://github.com/apache/pulsar/files/2883472/threaddump.txt)
   
   It seems that this thread is locking a `PersistentSubscription` object and the other `pulsar-io-xx-xx`
threads are waiting for the lock to be released.
   ```
   "pulsar-io-21-61" #1012 prio=5 os_prio=0 tid=0x00007f5cb006f000 nid=0x2627 waiting on condition
[0x00007f5acd0e9000]
      java.lang.Thread.State: WAITING (parking)
           at sun.misc.Unsafe.park(Native Method)
           - parking to wait for  <0x00007f75cbe5a950> (a java.util.concurrent.CompletableFuture$Signaller)
           at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
           at java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1693)
           at java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3323)
           at java.util.concurrent.CompletableFuture.waitingGet(CompletableFuture.java:1729)
           at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1895)
           at org.apache.pulsar.zookeeper.ZooKeeperDataCache.get(ZooKeeperDataCache.java:94)
           at org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers.isConsumersExceededOnTopic(PersistentDispatcherMultipleConsumers.java:148)
           at org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers.addConsumer(PersistentDispatcherMultipleConsumers.java:129)
           - locked <0x00007f6384d84368> (a org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers)
           at org.apache.pulsar.broker.service.persistent.PersistentSubscription.addConsumer(PersistentSubscription.java:139)
           - locked <0x00007f6373bd6600> (a org.apache.pulsar.broker.service.persistent.PersistentSubscription)
           at org.apache.pulsar.broker.service.persistent.PersistentTopic.lambda$9(PersistentTopic.java:517)
           at org.apache.pulsar.broker.service.persistent.PersistentTopic$$Lambda$229/184274928.accept(Unknown
Source)
           at java.util.concurrent.CompletableFuture.uniAccept(CompletableFuture.java:656)
           at java.util.concurrent.CompletableFuture.uniAcceptStage(CompletableFuture.java:669)
           at java.util.concurrent.CompletableFuture.thenAccept(CompletableFuture.java:1997)
           at org.apache.pulsar.broker.service.persistent.PersistentTopic.subscribe(PersistentTopic.java:513)
           at org.apache.pulsar.broker.service.ServerCnx.lambda$22(ServerCnx.java:609)
           at org.apache.pulsar.broker.service.ServerCnx$$Lambda$203/2100949060.apply(Unknown
Source)
           at java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:981)
           at java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2124)
           at org.apache.pulsar.broker.service.ServerCnx.lambda$18(ServerCnx.java:592)
           at org.apache.pulsar.broker.service.ServerCnx$$Lambda$201/788613052.apply(Unknown
Source)
           at java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:602)
           at java.util.concurrent.CompletableFuture.uniApplyStage(CompletableFuture.java:614)
           at java.util.concurrent.CompletableFuture.thenApply(CompletableFuture.java:1983)
           at org.apache.pulsar.broker.service.ServerCnx.lambda$6(ServerCnx.java:550)
           at org.apache.pulsar.broker.service.ServerCnx$$Lambda$197/133793716.apply(Unknown
Source)
           at java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:602)
           at java.util.concurrent.CompletableFuture.uniApplyStage(CompletableFuture.java:614)
           at java.util.concurrent.CompletableFuture.thenApply(CompletableFuture.java:1983)
           at org.apache.pulsar.broker.service.ServerCnx.handleSubscribe(ServerCnx.java:539)
           at org.apache.pulsar.common.api.PulsarDecoder.channelRead(PulsarDecoder.java:202)
           at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362)
           at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348)
           at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:340)
           at io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:310)
           at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:284)
           at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362)
           at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348)
           at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:340)
           at io.netty.handler.ssl.SslHandler.unwrap(SslHandler.java:1389)
           at io.netty.handler.ssl.SslHandler.decodeNonJdkCompatible(SslHandler.java:1171)
           at io.netty.handler.ssl.SslHandler.decode(SslHandler.java:1205)
           at io.netty.handler.codec.ByteToMessageDecoder.decodeRemovalReentryProtection(ByteToMessageDecoder.java:489)
           at io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:428)
           at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:265)
           at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362)
           at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348)
           at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:340)
           at io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1414)
           at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362)
           at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348)
           at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:945)
           at io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.epollInReady(AbstractEpollStreamChannel.java:806)
           at io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:404)
           at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:304)
           at io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:886)
           at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
           at java.lang.Thread.run(Thread.java:748)
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

Mime
View raw message