graysonzeng commented on PR #4171:
URL: https://github.com/apache/bookkeeper/pull/4171#issuecomment-1886240623
> Thanks for explaining that @graysonzeng. Makes sense. Just wondering if
the logic really works correctly without making the sendAddSuccessCallbacks
method synchronized. For example, on line 1831, the call to
`pendingAddOps.remove()` seems to assume that it is the same instance that
`pendingAddOps.peek()` returned on line 1814. The extensive usage of
`synchronized` isn't nice, but there doesn't seem to be away around it?
When I first repaired it, I tried to use synchronized on
sendAddSuccessCallbacks, and a deadlock occurred.
```java
"BookKeeperClientWorker-OrderedExecutor-8-0":
at
org.apache.bookkeeper.client.LedgerHandle.sendAddSuccessCallbacks(LedgerHandle.java:1811)
- waiting to lock <0x00000000bfae1120> (a
org.apache.bookkeeper.client.LedgerHandle)
at
org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:204)
- locked <0x00000000c3946f68> (a
org.apache.bookkeeper.client.PendingAddOp)
at
org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2006)
at
org.apache.bookkeeper.client.LedgerHandle.lambda$ensembleChangeLoop$10(LedgerHandle.java:1997)
at
org.apache.bookkeeper.client.LedgerHandle$$Lambda$1586/0x00007f8e4c95ac58.accept(Unknown
Source)
at
java.util.concurrent.CompletableFuture.uniWhenComplete([email protected]/CompletableFuture.java:863)
at
java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire([email protected]/CompletableFuture.java:841)
at
java.util.concurrent.CompletableFuture$Completion.run([email protected]/CompletableFuture.java:482)
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-11-12":
at
org.apache.bookkeeper.client.PendingAddOp.submitCallback(PendingAddOp.java:394)
- waiting to lock <0x00000000c3946f68> (a
org.apache.bookkeeper.client.PendingAddOp)
at
org.apache.bookkeeper.client.LedgerHandle.sendAddSuccessCallbacks(LedgerHandle.java:1835)
- locked <0x00000000bfae1120> (a
org.apache.bookkeeper.client.LedgerHandle)
at
org.apache.bookkeeper.client.PendingAddOp.sendAddSuccessCallbacks(PendingAddOp.java:390)
at
org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:307)
- locked <0x00000000c3945640> (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:46)
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)
```
Therefore I want to use sendingCallbacks instead of synchronized
> For example, on line 1831, the call to pendingAddOps.remove() seems to
assume that it is the same instance that pendingAddOps.peek() returned on line
1814.
Thanks for reminding. I replaced pendingAddOps.remove() with
pendingAddOps.remove(pendingAddOp) to ensure that the same element is processed.
> should we make changingEmsemble field volatile to fix this part of the
problem?
Great suggestion, this will ensure the visibility of changingEmsemble. I've
added it
--
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]