Open thetumbled opened 4 months ago
PTAL, thanks. @lhotari @shoothzj @poorbarcode @BewareMyPower @hangc0276
Didn't look at his closely, fastthread.io says:
Thread pulsar-io-32-13 is in deadlock with thread pulsar-io-32-14
pulsar-io-32-13
THREAD ID : 191
STATE : BLOCKED
stackTrace:
owned by pulsar-io-32-14 Id=192
at app//org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:181)
at app//org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
at app//org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
- locked java.lang.Object@20670d0e
at app//org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
- locked org.apache.bookkeeper.client.PendingAddOp@29884c8c
at app//org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
at app//org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
at app//org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
at app//org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
at app//org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2581)
at app//org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2486)
at app//io.netty.util.concurrent.DefaultPromise.notifyListener0(DefaultPromise.java:590)
at app//io.netty.util.concurrent.DefaultPromise.notifyListeners0(DefaultPromise.java:583)
at app//io.netty.util.concurrent.DefaultPromise.notifyListenersNow(DefaultPromise.java:559)
at app//io.netty.util.concurrent.DefaultPromise.notifyListeners(DefaultPromise.java:492)
at app//io.netty.util.concurrent.DefaultPromise.setValue0(DefaultPromise.java:636)
at app//io.netty.util.concurrent.DefaultPromise.setFailure0(DefaultPromise.java:629)
at app//io.netty.util.concurrent.DefaultPromise.tryFailure(DefaultPromise.java:118)
at app//io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.fulfillConnectPromise(AbstractEpollChannel.java:675)
at app//io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.finishConnect(AbstractEpollChannel.java:694)
at app//io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.epollOutReady(AbstractEpollChannel.java:567)
at app//io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:499)
at app//io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:407)
at app//io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)
at app//io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
at app//io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
at java.base@17.0.4/java.lang.Thread.run(Thread.java:833)
Locked synchronizers: count = 0
pulsar-io-32-14
THREAD ID : 192
STATE : BLOCKED
stackTrace:
owned by pulsar-io-32-13 Id=191
at app//org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:216)
at app//org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
- locked org.apache.bookkeeper.client.PendingAddOp@7701cdc8
at app//org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
at app//org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
at app//org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
at app//org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
at app//org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2581)
at app//io.netty.util.concurrent.DefaultPromise.notifyListener0(DefaultPromise.java:590)
at app//io.netty.util.concurrent.DefaultPromise.notifyListeners0(DefaultPromise.java:583)
at app//io.netty.util.concurrent.DefaultPromise.notifyListenersNow(DefaultPromise.java:559)
at app//io.netty.util.concurrent.DefaultPromise.notifyListeners(DefaultPromise.java:492)
at app//io.netty.util.concurrent.DefaultPromise.setValue0(DefaultPromise.java:636)
at app//io.netty.util.concurrent.DefaultPromise.setFailure0(DefaultPromise.java:629)
at app//io.netty.util.concurrent.DefaultPromise.tryFailure(DefaultPromise.java:118)
at app//io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.fulfillConnectPromise(AbstractEpollChannel.java:675)
at app//io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.finishConnect(AbstractEpollChannel.java:694)
at app//io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.epollOutReady(AbstractEpollChannel.java:567)
at app//io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:499)
at app//io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:407)
at app//io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)
at app//io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
at app//io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
at java.base@17.0.4/java.lang.Thread.run(Thread.java:833)
Locked synchronizers: count = 0
Thread pulsar-io-32-37 is in deadlock with thread pulsar-io-32-38
pulsar-io-32-37
THREAD ID : 217
STATE : BLOCKED
stackTrace:
owned by pulsar-io-32-38 Id=218
at app//org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:181)
at app//org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
at app//org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
- locked java.lang.Object@4bbb3f30
at app//org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
- locked org.apache.bookkeeper.client.PendingAddOp@22fc66e4
at app//org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
at app//org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
at app//org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
at app//org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
at app//org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2581)
at app//org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2486)
at app//io.netty.util.concurrent.DefaultPromise.notifyListener0(DefaultPromise.java:590)
at app//io.netty.util.concurrent.DefaultPromise.notifyListeners0(DefaultPromise.java:583)
at app//io.netty.util.concurrent.DefaultPromise.notifyListenersNow(DefaultPromise.java:559)
at app//io.netty.util.concurrent.DefaultPromise.notifyListeners(DefaultPromise.java:492)
at app//io.netty.util.concurrent.DefaultPromise.setValue0(DefaultPromise.java:636)
at app//io.netty.util.concurrent.DefaultPromise.setFailure0(DefaultPromise.java:629)
at app//io.netty.util.concurrent.DefaultPromise.tryFailure(DefaultPromise.java:118)
at app//io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.fulfillConnectPromise(AbstractEpollChannel.java:675)
at app//io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.finishConnect(AbstractEpollChannel.java:694)
at app//io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.epollOutReady(AbstractEpollChannel.java:567)
at app//io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:499)
at app//io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:407)
at app//io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)
at app//io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
at app//io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
at java.base@17.0.4/java.lang.Thread.run(Thread.java:833)
Locked synchronizers: count = 0
pulsar-io-32-38
THREAD ID : 218
STATE : BLOCKED
stackTrace:
owned by pulsar-io-32-37 Id=217
at app//org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:216)
at app//org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
- locked org.apache.bookkeeper.client.PendingAddOp@53a4eaf1
at app//org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
at app//org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
at app//org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
at app//org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
at app//org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2581)
at app//org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2486)
at app//io.netty.util.concurrent.DefaultPromise.notifyListener0(DefaultPromise.java:590)
at app//io.netty.util.concurrent.DefaultPromise.notifyListeners0(DefaultPromise.java:583)
at app//io.netty.util.concurrent.DefaultPromise.notifyListenersNow(DefaultPromise.java:559)
at app//io.netty.util.concurrent.DefaultPromise.notifyListeners(DefaultPromise.java:492)
at app//io.netty.util.concurrent.DefaultPromise.setValue0(DefaultPromise.java:636)
at app//io.netty.util.concurrent.DefaultPromise.setFailure0(DefaultPromise.java:629)
at app//io.netty.util.concurrent.DefaultPromise.tryFailure(DefaultPromise.java:118)
at app//io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.fulfillConnectPromise(AbstractEpollChannel.java:675)
at app//io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.finishConnect(AbstractEpollChannel.java:694)
at app//io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.epollOutReady(AbstractEpollChannel.java:567)
at app//io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:499)
at app//io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:407)
at app//io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)
at app//io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
at app//io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
at java.base@17.0.4/java.lang.Thread.run(Thread.java:833)
Locked synchronizers: count = 0
It looks as if we can remove this lock, which is unnecessary but result into the dead lock. https://github.com/apache/bookkeeper/blob/1ae4be04e20eca4490b52da474d455a2414168b4/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java#L211-L215 WDYT, @horizonzy @hangc0276 @dlg99 @shoothzj
@thetumbled I think it's ok to remove this lock if others uses OrderedExecutor too. But I think this issue maybe fixed in #4285
@thetumbled I think it's ok to remove this lock if others uses OrderedExecutor too. But I think this issue maybe fixed in #4285
Sorry, I might be #4278
@thetumbled I think it's ok to remove this lock if others uses OrderedExecutor too. But I think this issue maybe fixed in #4285
Sorry, I might be #4278
I think https://github.com/apache/bookkeeper/pull/4278 can't fixed the error.
We are hitting this bug in pulsae 3.0.6+ . Any update on the fix? Can the fix PR be merged?
Pulsar 3.0.5, Bookie 4.16. The cluster has entered an abnormal state due to dead lock error.