graysonzeng commented on PR #4171:
URL: https://github.com/apache/bookkeeper/pull/4171#issuecomment-1891453952
Thanks for your reply @lhotari, I have some test for #4175 , Unfortunately
it will also cause deadlock.
``` java
"pulsar-io-3-4":
at
org.apache.bookkeeper.client.PendingAddOp.submitCallback(PendingAddOp.java:394)
- waiting to lock <0x000000077f2becf0> (a
org.apache.bookkeeper.client.PendingAddOp)
at
org.apache.bookkeeper.client.LedgerHandle.sendAddSuccessCallbacks(LedgerHandle.java:1838)
- locked <0x000000077e6c0d58> (a
java.util.concurrent.ConcurrentLinkedQueue)
at
org.apache.bookkeeper.client.PendingAddOp.sendAddSuccessCallbacks(PendingAddOp.java:390)
at
org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:307)
- locked <0x000000077f2bae98> (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:2548)
at
org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
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.setFailure(DefaultPromise.java:110)
at
io.netty.channel.DefaultChannelPromise.setFailure(DefaultChannelPromise.java:89)
at io.netty.bootstrap.Bootstrap.doResolveAndConnect0(Bootstrap.java:228)
at io.netty.bootstrap.Bootstrap.access$000(Bootstrap.java:47)
at io.netty.bootstrap.Bootstrap$1.operationComplete(Bootstrap.java:189)
at io.netty.bootstrap.Bootstrap$1.operationComplete(Bootstrap.java:175)
at
io.netty.util.concurrent.DefaultPromise.notifyListener0(DefaultPromise.java:590)
at
io.netty.util.concurrent.DefaultPromise.notifyListenersNow(DefaultPromise.java:557)
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.setSuccess0(DefaultPromise.java:625)
at
io.netty.util.concurrent.DefaultPromise.trySuccess(DefaultPromise.java:105)
at
io.netty.channel.DefaultChannelPromise.trySuccess(DefaultChannelPromise.java:84)
at
io.netty.channel.AbstractChannel$AbstractUnsafe.safeSetSuccess(AbstractChannel.java:990)
at
io.netty.channel.AbstractChannel$AbstractUnsafe.register0(AbstractChannel.java:516)
at
io.netty.channel.AbstractChannel$AbstractUnsafe.access$200(AbstractChannel.java:429)
at
io.netty.channel.AbstractChannel$AbstractUnsafe$1.run(AbstractChannel.java:486)
at
io.netty.util.concurrent.AbstractEventExecutor.runTask(AbstractEventExecutor.java:173)
at
io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:166)
at
io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:470)
at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:413)
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)
"BookKeeperClientWorker-OrderedExecutor-7-0":
at
org.apache.bookkeeper.client.LedgerHandle.sendAddSuccessCallbacks(LedgerHandle.java:1819)
- waiting to lock <0x000000077e6c0d58> (a
java.util.concurrent.ConcurrentLinkedQueue)
at
org.apache.bookkeeper.client.PendingAddOp.sendAddSuccessCallbacks(PendingAddOp.java:390)
at
org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:307)
- locked <0x000000077f2becf0> (a
org.apache.bookkeeper.client.PendingAddOp)
at
org.apache.bookkeeper.proto.PerChannelBookieClient$AddCompletion.writeComplete(PerChannelBookieClient.java:2183)
at
org.apache.bookkeeper.proto.PerChannelBookieClient$AddCompletion.handleResponse(PerChannelBookieClient.java:2240)
at
org.apache.bookkeeper.proto.PerChannelBookieClient$AddCompletion.handleV2Response(PerChannelBookieClient.java:2219)
at
org.apache.bookkeeper.proto.PerChannelBookieClient$ReadV2ResponseCallback.run(PerChannelBookieClient.java:1397)
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)
Found 1 deadlock.
```
From the stack we can see that this happens within sendAddSuccessCallbacks
```
void sendAddSuccessCallbacks() {
// thread A has acquired the lock of pendingAddOp instance A , and
is trying to acquire it <----
synchronized (pendingAddOps) {
PendingAddOp pendingAddOp;
while ((pendingAddOp = pendingAddOps.peek()) != null
&& !changingEnsemble) {
if (!pendingAddOp.completed) {
if (LOG.isDebugEnabled()) {
LOG.debug("pending add not completed: {}",
pendingAddOp);
}
return;
}
pendingAddOps.remove();
explicitLacFlushPolicy.updatePiggyBackedLac(lastAddConfirmed);
pendingAddsSequenceHead = pendingAddOp.entryId;
if (!writeFlags.contains(WriteFlag.DEFERRED_SYNC)) {
this.lastAddConfirmed = pendingAddsSequenceHead;
}
// thread A has acquired the lock of pendingAddOp instance B
and
// the `pendingAddOps` lock of this LedgerHandle instance,
and is trying to acquire it <----
pendingAddOp.submitCallback(BKException.Code.OK);
}
}
}
```
thread A -> pendingAddOp lock A (locked) -> LedgerHandle lock L (locked) ->
pendingAddOp lock B (waiting)
thread B -> pendingAddOp lock B (locked) -> LedgerHandle lock L (waiting)
Here is the entire stack file
[deadLockThreadDump.txt](https://github.com/apache/bookkeeper/files/13936030/deadLockThreadDump.txt)
> @hangc0276 True, but that would be a significant change that could at
least cause performance regressions.
Suppose we are not willing to use OrderedExecutor because of performance
regressions, are there any other better suggestions? i'm looking forward to it
@lhotari
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]