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

Sijie Guo commented on BOOKKEEPER-620:
--------------------------------------

> LOG.info("Successfully connected to bookie: {}", addr);

add channel in this statement, and move this line down after the channel is 
assigned.

> LOG.warn("Closed before connection completed, clean up: {}, current state {}",
+                                  addr, state);

log channel object instead of addr, which channel give more information than 
addr for debugging.

> LOG.debug("Already connected with another channel, so close the new channel 
> ");

log existed channel and new connected channel, otherwise this log statement 
didn't give any useful information

> LOG.error("Could not connect to bookie: {}, current state {}", addr, state);

same thing, log channel rather than addr.


{code}
     private void closeInternal(boolean permanent) {
+        ChannelFuture closeFuture = null;
         synchronized (this) {
             if (permanent) {
                 state = ConnectionState.CLOSED;
             } else if (state != ConnectionState.CLOSED) {
                 state = ConnectionState.DISCONNECTED;
             }
+            if (channel != null) {
+                closeFuture = closeChannel(channel);
+                channel = null;
+            }
         }
-        if (channel != null) {
-            channel.close().awaitUninterruptibly();
+        if (closeFuture != null) {
+            closeFuture.awaitUninterruptibly();
         }
-        if (readTimeoutTimer != null) {
-            readTimeoutTimer.stop();
-            readTimeoutTimer = null;
+    }
{code}

I would suggest moving close out side of synchronization, just assigned the 
channel to a local variable and close this local variable out side of the 
synchronization block, we observed a deadlock on netty channel buffer.

{code}
     public void channelDisconnected(ChannelHandlerContext ctx, 
ChannelStateEvent e) throws Exception {
         LOG.info("Disconnected from bookie: " + addr);
-        errorOutOutstandingEntries();
-        Channel c = this.channel;
+        Channel c = ctx.getChannel();
         if (c != null) {
-            c.close();
+            closeChannel(c);
         }
+
+        Set<CompletionKey> addsToError = Collections.emptySet();
+        Set<CompletionKey> readsToError = Collections.emptySet();
         synchronized (this) {
-            if (state != ConnectionState.CLOSED) {
+            if (this.channel == c
+                && state != ConnectionState.CLOSED) {
                 state = ConnectionState.DISCONNECTED;
             }
+            if (state == ConnectionState.CLOSED
+                || state == ConnectionState.DISCONNECTED) {
+                addsToError = addCompletions.keySet();
+                readsToError = readCompletions.keySet();
+            }
+        }
+        for (CompletionKey key : addsToError) {
+            errorOutAddKey(key);
+        }
+        for (CompletionKey key : readsToError) {
+            errorOutReadKey(key);
         }
{code}
1) it would be better to keep comments when you moved around the code
2) it would be better to error out entries before set state back to 
DISCONNECTED. since after the channel is set to DISCONNECTED, the client would 
begin sending requests to connecting. you might error ongoing requests.

so you could do conditional check first, error out entries based on conditional 
checking and set the state back DISCONNECTED after error out entries.


                
> PerChannelBookieClient race during channel disconnect
> -----------------------------------------------------
>
>                 Key: BOOKKEEPER-620
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-620
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>            Reporter: Sijie Guo
>            Assignee: Robin Dhamankar
>             Fix For: 4.2.2, 4.3.0
>
>         Attachments: 
> 0001-BOOKKEEPER-620-PerChannelBookieClient-race-during-ch.patch, 
> 0001-BOOKKEEPER-620-PerChannelBookieClient-race-during-ch.patch, 
> 0002-BOOKKEEPER-620-PerChannelBookieClient-race-during-ch.patch, 
> BOOKKEEPER-620.diff
>
>
> channel & state are not synchronized in PerChannelBookieClient#closeInternal. 
> so it might cause state is set to CONNECTED but the netty channel is closed 
> by mistake in closeInternal.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

Reply via email to