Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Dead lock error. #4457

Open
thetumbled opened this issue Jun 27, 2024 · 6 comments · May be fixed by #4467
Open

Dead lock error. #4457

thetumbled opened this issue Jun 27, 2024 · 6 comments · May be fixed by #4467
Labels

Comments

@thetumbled
Copy link
Member

Pulsar 3.0.5, Bookie 4.16.
The cluster has entered an abnormal state due to dead lock error.

Found one Java-level deadlock:
=============================
"BookKeeperClientWorker-OrderedExecutor-10-0":
  waiting to lock monitor 0x00007f162895cf20 (object 0x0000044fc7914058, a java.lang.Object),
  which is held by "pulsar-io-32-13"

"pulsar-io-32-13":
  waiting to lock monitor 0x00007f1400d928a0 (object 0x00000403f63e7520, a org.apache.bookkeeper.client.PendingAddOp),
  which is held by "pulsar-io-32-14"

"pulsar-io-32-14":
  waiting to lock monitor 0x00007f162895cf20 (object 0x0000044fc7914058, a java.lang.Object),
  which is held by "pulsar-io-32-13"

Java stack information for the threads listed above:
===================================================
"BookKeeperClientWorker-OrderedExecutor-10-0":
        at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.getCurrentEnsemble(ReadOnlyLedgerHandle.java:342)
        - waiting to lock <0x0000044fc7914058> (a java.lang.Object)
        at org.apache.bookkeeper.client.LedgerHandle.asyncRecoveryAddEntry(LedgerHandle.java:1208)
        at org.apache.bookkeeper.client.LedgerRecoveryOp.onEntryComplete(LedgerRecoveryOp.java:204)
        at org.apache.bookkeeper.client.ListenerBasedPendingReadOp.submitCallback(ListenerBasedPendingReadOp.java:67)
        at org.apache.bookkeeper.client.PendingReadOp.readEntryComplete(PendingReadOp.java:608)
        at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadCompletion$1.readEntryComplete(PerChannelBookieClient.java:1905)
        at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadCompletion.handleReadResponse(PerChannelBookieClient.java:1987)
        at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadCompletion.handleV2Response(PerChannelBookieClient.java:1939)
        at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadV2ResponseCallback.run(PerChannelBookieClient.java:1417)
        at org.apache.bookkeeper.common.util.SingleThreadExecutor.safeRunTask(SingleThreadExecutor.java:137)
        at org.apache.bookkeeper.common.util.SingleThreadExecutor.run(SingleThreadExecutor.java:113)
        at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.lang.Thread.run([email protected]/Thread.java:833)
"pulsar-io-32-13":
        at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:181)
        - waiting to lock <0x00000403f63e7520> (a org.apache.bookkeeper.client.PendingAddOp)
        at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
        at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
        - locked <0x0000044fc7914058> (a java.lang.Object)
        at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
        - locked <0x00000401526088f8> (a org.apache.bookkeeper.client.PendingAddOp)
        at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
        at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
        at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
        at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
        at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2581)
        at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2486)
        at io.netty.util.concurrent.DefaultPromise.notifyListener0(DefaultPromise.java:590)
        at io.netty.util.concurrent.DefaultPromise.notifyListeners0(DefaultPromise.java:583)
        at io.netty.util.concurrent.DefaultPromise.notifyListenersNow(DefaultPromise.java:559)
        at io.netty.util.concurrent.DefaultPromise.notifyListeners(DefaultPromise.java:492)
        at io.netty.util.concurrent.DefaultPromise.setValue0(DefaultPromise.java:636)
        at io.netty.util.concurrent.DefaultPromise.setFailure0(DefaultPromise.java:629)
        at io.netty.util.concurrent.DefaultPromise.tryFailure(DefaultPromise.java:118)
        at io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.fulfillConnectPromise(AbstractEpollChannel.java:675)
        at io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.finishConnect(AbstractEpollChannel.java:694)
        at io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.epollOutReady(AbstractEpollChannel.java:567)
        at io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:499)
        at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:407)
        at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)
        at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
        at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.lang.Thread.run([email protected]/Thread.java:833)

"pulsar-io-32-14":
        at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:216)
        - waiting to lock <0x0000044fc7914058> (a java.lang.Object)
        at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
        - locked <0x00000403f63e7520> (a org.apache.bookkeeper.client.PendingAddOp)
        at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
        at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
        at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
        at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
        at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2581)
        at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2486)
        at io.netty.util.concurrent.DefaultPromise.notifyListener0(DefaultPromise.java:590)
        at io.netty.util.concurrent.DefaultPromise.notifyListeners0(DefaultPromise.java:583)
        at io.netty.util.concurrent.DefaultPromise.notifyListenersNow(DefaultPromise.java:559)
        at io.netty.util.concurrent.DefaultPromise.notifyListeners(DefaultPromise.java:492)
        at io.netty.util.concurrent.DefaultPromise.setValue0(DefaultPromise.java:636)
        at io.netty.util.concurrent.DefaultPromise.setFailure0(DefaultPromise.java:629)
        at io.netty.util.concurrent.DefaultPromise.tryFailure(DefaultPromise.java:118)
        at io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.fulfillConnectPromise(AbstractEpollChannel.java:675)
        at io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.finishConnect(AbstractEpollChannel.java:694)
        at io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.epollOutReady(AbstractEpollChannel.java:567)
        at io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:499)
        at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:407)
        at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)
        at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
        at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.lang.Thread.run([email protected]/Thread.java:833)


Found one Java-level deadlock:
=============================
"BookKeeperClientWorker-OrderedExecutor-11-0":
  waiting to lock monitor 0x00007f161c4b0930 (object 0x000004009e329e40, a java.lang.Object),
  which is held by "pulsar-io-32-37"

"pulsar-io-32-37":
  waiting to lock monitor 0x00007f138c94a0a0 (object 0x000004033f3d3ea0, a org.apache.bookkeeper.client.PendingAddOp),
  which is held by "pulsar-io-32-38"

"pulsar-io-32-38":
  waiting to lock monitor 0x00007f161c4b0930 (object 0x000004009e329e40, a java.lang.Object),
  which is held by "pulsar-io-32-37"

Java stack information for the threads listed above:
===================================================
"BookKeeperClientWorker-OrderedExecutor-11-0":
        at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.getCurrentEnsemble(ReadOnlyLedgerHandle.java:342)
        - waiting to lock <0x000004009e329e40> (a java.lang.Object)
        at org.apache.bookkeeper.client.LedgerHandle.asyncRecoveryAddEntry(LedgerHandle.java:1208)
        at org.apache.bookkeeper.client.LedgerRecoveryOp.onEntryComplete(LedgerRecoveryOp.java:204)
        at org.apache.bookkeeper.client.ListenerBasedPendingReadOp.submitCallback(ListenerBasedPendingReadOp.java:67)
        at org.apache.bookkeeper.client.PendingReadOp.readEntryComplete(PendingReadOp.java:608)
        at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadCompletion$1.readEntryComplete(PerChannelBookieClient.java:1905)
        at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadCompletion.handleReadResponse(PerChannelBookieClient.java:1987)
        at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadCompletion.handleV2Response(PerChannelBookieClient.java:1939)
        at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadV2ResponseCallback.run(PerChannelBookieClient.java:1417)
        at org.apache.bookkeeper.common.util.SingleThreadExecutor.safeRunTask(SingleThreadExecutor.java:137)
        at org.apache.bookkeeper.common.util.SingleThreadExecutor.run(SingleThreadExecutor.java:107)
        at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.lang.Thread.run([email protected]/Thread.java:833)
"pulsar-io-32-37":
        at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:181)
        - waiting to lock <0x000004033f3d3ea0> (a org.apache.bookkeeper.client.PendingAddOp)
        at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
        at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
        - locked <0x000004009e329e40> (a java.lang.Object)
        at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
        - locked <0x000004019c391848> (a org.apache.bookkeeper.client.PendingAddOp)
        at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
        at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
        at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
        at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
        at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2581)
        at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2486)
        at io.netty.util.concurrent.DefaultPromise.notifyListener0(DefaultPromise.java:590)
        at io.netty.util.concurrent.DefaultPromise.notifyListeners0(DefaultPromise.java:583)
        at io.netty.util.concurrent.DefaultPromise.notifyListenersNow(DefaultPromise.java:559)
        at io.netty.util.concurrent.DefaultPromise.notifyListeners(DefaultPromise.java:492)
        at io.netty.util.concurrent.DefaultPromise.setValue0(DefaultPromise.java:636)
        at io.netty.util.concurrent.DefaultPromise.setFailure0(DefaultPromise.java:629)
        at io.netty.util.concurrent.DefaultPromise.tryFailure(DefaultPromise.java:118)
        at io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.fulfillConnectPromise(AbstractEpollChannel.java:675)
        at io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.finishConnect(AbstractEpollChannel.java:694)
        at io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.epollOutReady(AbstractEpollChannel.java:567)
        at io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:499)
        at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:407)
        at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)
        at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
        at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.lang.Thread.run([email protected]/Thread.java:833)
"pulsar-io-32-38":
        at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:216)
        - waiting to lock <0x000004009e329e40> (a java.lang.Object)
        at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
        - locked <0x000004033f3d3ea0> (a org.apache.bookkeeper.client.PendingAddOp)
        at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
        at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
        at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
        at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
        at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2581)
        at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2486)
        at io.netty.util.concurrent.DefaultPromise.notifyListener0(DefaultPromise.java:590)
        at io.netty.util.concurrent.DefaultPromise.notifyListeners0(DefaultPromise.java:583)
        at io.netty.util.concurrent.DefaultPromise.notifyListenersNow(DefaultPromise.java:559)
        at io.netty.util.concurrent.DefaultPromise.notifyListeners(DefaultPromise.java:492)
        at io.netty.util.concurrent.DefaultPromise.setValue0(DefaultPromise.java:636)
        at io.netty.util.concurrent.DefaultPromise.setFailure0(DefaultPromise.java:629)
        at io.netty.util.concurrent.DefaultPromise.tryFailure(DefaultPromise.java:118)
        at io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.fulfillConnectPromise(AbstractEpollChannel.java:675)
        at io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.finishConnect(AbstractEpollChannel.java:694)
        at io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.epollOutReady(AbstractEpollChannel.java:567)
        at io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:499)
        at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:407)
        at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)
        at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
        at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.lang.Thread.run([email protected]/Thread.java:833)

Found 2 deadlocks.


====> DEADLOCKS DETECTED <====


"BookKeeperClientWorker-OrderedExecutor-10-0" Id=53 in BLOCKED on lock=java.lang.Object@20670d0e
     owned by pulsar-io-32-13 Id=191
    at app//org.apache.bookkeeper.client.ReadOnlyLedgerHandle.getCurrentEnsemble(ReadOnlyLedgerHandle.java:342)
    at app//org.apache.bookkeeper.client.LedgerHandle.asyncRecoveryAddEntry(LedgerHandle.java:1208)
    at app//org.apache.bookkeeper.client.LedgerRecoveryOp.onEntryComplete(LedgerRecoveryOp.java:204)
    at app//org.apache.bookkeeper.client.ListenerBasedPendingReadOp.submitCallback(ListenerBasedPendingReadOp.java:67)
    at app//org.apache.bookkeeper.client.PendingReadOp.readEntryComplete(PendingReadOp.java:608)
    at app//org.apache.bookkeeper.proto.PerChannelBookieClient$ReadCompletion$1.readEntryComplete(PerChannelBookieClient.java:1905)
    at app//org.apache.bookkeeper.proto.PerChannelBookieClient$ReadCompletion.handleReadResponse(PerChannelBookieClient.java:1987)
    at app//org.apache.bookkeeper.proto.PerChannelBookieClient$ReadCompletion.handleV2Response(PerChannelBookieClient.java:1939)
    at app//org.apache.bookkeeper.proto.PerChannelBookieClient$ReadV2ResponseCallback.run(PerChannelBookieClient.java:1417)
    at app//org.apache.bookkeeper.common.util.SingleThreadExecutor.safeRunTask(SingleThreadExecutor.java:137)
    at app//org.apache.bookkeeper.common.util.SingleThreadExecutor.run(SingleThreadExecutor.java:113)
    at app//io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
    at [email protected]/java.lang.Thread.run(Thread.java:833)

    Locked synchronizers: count = 0



"pulsar-io-32-13" Id=191 in BLOCKED on lock=org.apache.bookkeeper.client.PendingAddOp@7701cdc8
     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 [email protected]/java.lang.Thread.run(Thread.java:833)

    Locked synchronizers: count = 0



"pulsar-io-32-14" Id=192 in BLOCKED on lock=java.lang.Object@20670d0e
     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 [email protected]/java.lang.Thread.run(Thread.java:833)

    Locked synchronizers: count = 0

"BookKeeperClientWorker-OrderedExecutor-11-0" Id=54 in BLOCKED on lock=java.lang.Object@4bbb3f30
     owned by pulsar-io-32-37 Id=217
    at app//org.apache.bookkeeper.client.ReadOnlyLedgerHandle.getCurrentEnsemble(ReadOnlyLedgerHandle.java:342)
    at app//org.apache.bookkeeper.client.LedgerHandle.asyncRecoveryAddEntry(LedgerHandle.java:1208)
    at app//org.apache.bookkeeper.client.LedgerRecoveryOp.onEntryComplete(LedgerRecoveryOp.java:204)
    at app//org.apache.bookkeeper.client.ListenerBasedPendingReadOp.submitCallback(ListenerBasedPendingReadOp.java:67)
    at app//org.apache.bookkeeper.client.PendingReadOp.readEntryComplete(PendingReadOp.java:608)
    at app//org.apache.bookkeeper.proto.PerChannelBookieClient$ReadCompletion$1.readEntryComplete(PerChannelBookieClient.java:1905)
    at app//org.apache.bookkeeper.proto.PerChannelBookieClient$ReadCompletion.handleReadResponse(PerChannelBookieClient.java:1987)
    at app//org.apache.bookkeeper.proto.PerChannelBookieClient$ReadCompletion.handleV2Response(PerChannelBookieClient.java:1939)
    at app//org.apache.bookkeeper.proto.PerChannelBookieClient$ReadV2ResponseCallback.run(PerChannelBookieClient.java:1417)
    at app//org.apache.bookkeeper.common.util.SingleThreadExecutor.safeRunTask(SingleThreadExecutor.java:137)
    at app//org.apache.bookkeeper.common.util.SingleThreadExecutor.run(SingleThreadExecutor.java:107)
    at app//io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
    at [email protected]/java.lang.Thread.run(Thread.java:833)

    Locked synchronizers: count = 0

"pulsar-io-32-37" Id=217 in BLOCKED on lock=org.apache.bookkeeper.client.PendingAddOp@53a4eaf1
     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 [email protected]/java.lang.Thread.run(Thread.java:833)

    Locked synchronizers: count = 0

"pulsar-io-32-38" Id=218 in BLOCKED on lock=java.lang.Object@4bbb3f30
     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 [email protected]/java.lang.Thread.run(Thread.java:833)

    Locked synchronizers: count = 0



15:25:23.209 [pulsar-web-68-39] ERROR org.apache.pulsar.broker.admin.impl.BrokersBase - [admin] Fail to run health check.
java.util.concurrent.CompletionException: java.lang.IllegalStateException: Deadlocked threads detected. BookKeeperClientWorker-OrderedExecutor-10-0(tid=53), pulsar-io-32-13(tid=191), pulsar-io-32-14(tid=192), BookKeeperClientWorker-OrderedExecutor-11-0(tid=54), pulsar-io-32-37(tid=217), pulsar-io-32-38(tid=218)
        at java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:315) ~[?:?]
        at java.util.concurrent.CompletableFuture.uniAcceptNow(CompletableFuture.java:761) ~[?:?]
        at java.util.concurrent.CompletableFuture.uniAcceptStage(CompletableFuture.java:735) ~[?:?]
        at java.util.concurrent.CompletableFuture.thenAccept(CompletableFuture.java:2182) ~[?:?]
        at org.apache.pulsar.broker.admin.impl.BrokersBase.healthCheck(BrokersBase.java:376) ~[org.apache.pulsar-pulsar-broker-3.0.4.1.jar:3.0.4.1]
        at jdk.internal.reflect.GeneratedMethodAccessor120.invoke(Unknown Source) ~[?:?]
        at jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) ~[?:?]
        at java.lang.reflect.Method.invoke(Method.java:568) ~[?:?]
        at org.glassfish.jersey.server.model.internal.ResourceMethodInvocationHandlerFactory.lambda$static$0(ResourceMethodInvocationHandlerFactory.java:52) ~[org.glassfish.jersey.core-jersey-server-2.34.jar:?]
        at org.glassfish.jersey.server.model.internal.AbstractJavaResourceMethodDispatcher$1.run(AbstractJavaResourceMethodDispatcher.java:124) ~[org.glassfish.jersey.core-jersey-server-2.34.jar:?]
        at org.glassfish.jersey.server.model.internal.AbstractJavaResourceMethodDispatcher.invoke(AbstractJavaResourceMethodDispatcher.java:167) ~[org.glassfish.jersey.core-jersey-server-2.34.jar:?]
        at org.glassfish.jersey.server.model.internal.JavaResourceMethodDispatcherProvider$VoidOutInvoker.doDispatch(JavaResourceMethodDispatcherProvider.java:159) ~[org.glassfish.jersey.core-jersey-server-2.34.jar:?]



@thetumbled
Copy link
Member Author

@dlg99
Copy link
Contributor

dlg99 commented Jul 1, 2024

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 [email protected]/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 [email protected]/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 [email protected]/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 [email protected]/java.lang.Thread.run(Thread.java:833)
Locked synchronizers: count = 0

@thetumbled
Copy link
Member Author

thetumbled commented Jul 18, 2024

It looks as if we can remove this lock, which is unnecessary but result into the dead lock.

void handleBookieFailure(final Map<Integer, BookieId> failedBookies) {
// handleBookieFailure should always run in the ordered executor thread for this
// ledger, so this synchronized should be unnecessary, but putting it here now
// just in case (can be removed when we validate threads)
synchronized (metadataLock) {

WDYT, @horizonzy @hangc0276 @dlg99 @shoothzj

@shoothzj
Copy link
Member

@thetumbled I think it's ok to remove this lock if others uses OrderedExecutor too. But I think this issue maybe fixed in #4285

@shoothzj
Copy link
Member

@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

@M1eyu2018
Copy link

@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 #4278 can't fixed the error.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
Development

Successfully merging a pull request may close this issue.

4 participants