[
https://issues.apache.org/jira/browse/BOOKKEEPER-326?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13406296#comment-13406296
]
Vinay commented on BOOKKEEPER-326:
----------------------------------
Hi Rakesh,
I did some investigation on netty behaviour while invoking the listeners.
Here it is:
1. The race is between following 2 lines of PerChannelBookieClient.
{code}
ChannelFuture future = bootstrap.connect(addr);
future.addListener(new ChannelFutureListener() {code}
2. bootstrap.connect(addr) will put a connection request to one queue which
will be processed by another thread and set the connection result.
If the result of connection is available before adding the listener to
future in future.addListener(..),
then listener will be invoked in same thread immediately still holding the
locks.
Overall scenario
--------------------------
# Ledger is created and entries are writter to 3 bookies.
# one of bookie is killed. (say this bookie is not the first bookie in the
ensemble)
# New bookie started.
# now openLedger() call is made to recover the ledger. as part of this
readLastConfirmed request will be added each of the bookies with callback
ReadLastConfirmedOp.readEntryComplete which is synchronized.
# First callback came from first bookie which is alive in separate thread and
entered ReadLastConfirmedOp.readEntryComplete() and processing.
# Another Callback came for the failed bookie from connect() method by holding
the lock of PerChannelBookieClient instance of failed bookie, and trying to
invoke the same callback, but BLOCKED. ( Here to invoke the listener in same
thread, need to wait before future.addListener(..) by putting debug point)
# As part of first call back, doRecoveryRead() will put one PendingReadOp
request for asyncread. If this PendingReadOp selects same
PerChannelBookieClient of failed bookie for read, then it will enter to
deadlock. (To reproduce, bookieIndex variable can be changed to index of the
failed bookie in PendingReadOp.sendRead(..))
( Can reuse the existing test BookieFailureTest.testLedgerOpenAfterBKCrashed())
> DeadLock during ledger recovery
> --------------------------------
>
> Key: BOOKKEEPER-326
> URL: https://issues.apache.org/jira/browse/BOOKKEEPER-326
> Project: Bookkeeper
> Issue Type: Bug
> Affects Versions: 4.1.0
> Reporter: Vinay
> Attachments: BK_DeadLock.log
>
>
> Deadlock found during ledger recovery. please find the attached thread dump.
--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators:
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira