pulsar-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From GitBox <...@apache.org>
Subject [GitHub] rdhabalia edited a comment on issue #3630: Broker can not respond to client requests
Date Thu, 28 Feb 2019 01:54:19 GMT
rdhabalia edited a comment on issue #3630: Broker can not respond to client requests
URL: https://github.com/apache/pulsar/issues/3630#issuecomment-468103484
 
 
   @massakam 
   I think you are right. thread-dump shows the same deadlock condition where
   1. `pulsar-io` takes lock on `PersistentSubscription's` synchronized method and waiting
for zk response
   2. `ForkJoinPool` gets value from zk and same thread tries to access same lock taken by
`pulsar-io` thread in `PersistentSubscription`. so, `ForkJoinPool-thread` couldn't complete
and can't serve further zk-cache request. and the same time pulsar-io threads are waiting
for zk-cache respinse.
   
   
   ```
   "pulsar-io-21-2" #159 prio=5 os_prio=0 tid=0x00002af27c009000 nid=0x23ba waiting for monitor
entry [0x00002aefaa6b2000]
      java.lang.Thread.State: BLOCKED (on object monitor)
   	at org.apache.pulsar.broker.service.persistent.PersistentSubscription.addConsumer(PersistentSubscription.java:100)
   	- waiting to lock <0x000000058e35a2b0> (a org.apache.pulsar.broker.service.persistent.PersistentSubscription)
   	at org.apache.pulsar.broker.service.persistent.PersistentTopic.lambda$9(PersistentTopic.java:523)
   	at org.apache.pulsar.broker.service.persistent.PersistentTopic$$Lambda$309/503840967.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:519)
   	at org.apache.pulsar.broker.service.ServerCnx.lambda$22(ServerCnx.java:609)
   	at org.apache.pulsar.broker.service.ServerCnx$$Lambda$299/1090387422.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$298/1756865666.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$294/380312495.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.fireChannelRead(ByteToMessageDecoder.java:297)
   	at io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:413)
   	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)
   
      Locked ownable synchronizers:
   	- None
   ```
   
   ```
   "pulsar-io-21-15" #340 prio=5 os_prio=0 tid=0x00002af27c020800 nid=0x263d waiting on condition
[0x00002aefdfb09000]
      java.lang.Thread.State: WAITING (parking)
           at sun.misc.Unsafe.park(Native Method)
           - parking to wait for  <0x00000006960b8018> (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 <0x000000058afd2dd0> (a org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers)
           at org.apache.pulsar.broker.service.persistent.PersistentSubscription.addConsumer(PersistentSubscription.java:139)
           - locked <0x000000058afd2ed8> (a org.apache.pulsar.broker.service.persistent.PersistentSubscription)
           at org.apache.pulsar.broker.service.persistent.PersistentTopic.lambda$9(PersistentTopic.java:523)
           at org.apache.pulsar.broker.service.persistent.PersistentTopic$$Lambda$309/503840967.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:519)
           at org.apache.pulsar.broker.service.ServerCnx.lambda$22(ServerCnx.java:609)
           at org.apache.pulsar.broker.service.ServerCnx$$Lambda$299/1090387422.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$298/1756865666.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$294/380312495.apply(Unknown
Source
   ```
   

----------------------------------------------------------------
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