[ 
https://issues.apache.org/jira/browse/BOOKKEEPER-326?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13406283#comment-13406283
 ] 

Rakesh R commented on BOOKKEEPER-326:
-------------------------------------

After analyzing the threaddump I could see, the 
DefaultChannelFuture.notifyListener() is notifying in the same thread where the 
PerChannelBookieClient.connect() is invoked. Now the 
pendingOp.operationComplete(rc, null); is executing and inturn reaches the 
ReadLastConfirmedOp.readEntryComplete() and is waiting for ReadLastConfirmedOp 
lock acquisition.
{code}
at 
org.apache.bookkeeper.client.ReadLastConfirmedOp.readEntryComplete(ReadLastConfirmedOp.java:79)
        - waiting to lock <0x00007f3dc329b958> (a 
org.apache.bookkeeper.client.ReadLastConfirmedOp)
        at 
org.apache.bookkeeper.proto.BookieClient$3.operationComplete(BookieClient.java:132)
        at 
org.apache.bookkeeper.proto.BookieClient$3.operationComplete(BookieClient.java:128)
        at 
org.apache.bookkeeper.proto.PerChannelBookieClient$1.operationComplete(PerChannelBookieClient.java:168)
        at 
org.jboss.netty.channel.DefaultChannelFuture.notifyListener(DefaultChannelFuture.java:381)
        at 
org.jboss.netty.channel.DefaultChannelFuture.addListener(DefaultChannelFuture.java:148)
        at 
org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:137)
        - locked <0x00007f3dc329b6a0> (a 
org.apache.bookkeeper.proto.PerChannelBookieClient)
        at 
org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:200)
        - locked <0x00007f3dc329b6a0> (a 
org.apache.bookkeeper.proto.PerChannelBookieClient)
{code}


I think the contract mentioned in the comment 'trick to not do operations under 
the lock' is broken, since the DefaultChannelFuture.notifyListener() is coming 
in the same thread where PerChannelBookieClient.connect() is invoked.
{code}
synchronized private void connect() {
        future.addListener(new ChannelFutureListener() {
            @Override
            public void operationComplete(ChannelFuture future) throws 
Exception {
                int rc;
                Queue<GenericCallback<Void>> oldPendingOps;

                synchronized (PerChannelBookieClient.this) {
                    .....
                    .....
                    .....
                    // trick to not do operations under the lock, take the list
                    // of pending ops and assign it to a new variable, while
                    // emptying the pending ops by just assigning it to a new
                    // list
                    oldPendingOps = pendingOps;
                    pendingOps = new ArrayDeque<GenericCallback<Void>>();
                }
                for (GenericCallback<Void> pendingOp : oldPendingOps) {
                    pendingOp.operationComplete(rc, null);
                }
            }
        });
}
{code}


-Rakesh
                
> 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

        

Reply via email to