horizonzy opened a new pull request, #4285:
URL: https://github.com/apache/bookkeeper/pull/4285

   Fixes #4278, you need to go through #4278 for the context.
   
   When a connection is broken, it will trigger 
PerChannelBookieClient#channelInactive. 
https://github.com/apache/bookkeeper/blob/8516d0a0f1dbd352d3ef6455c2d41c94661ce1ce/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java#L1292-L1312
   
   Point 1. line_1303, it will drain all completionObjects, and complete with 
BookieHandleNotAvailableException.
   
   Point 2. line_1310, set PerChannelBookieClient#channel = null.
   
   When PerChannelBookieClient#addEntry. 
https://github.com/apache/bookkeeper/blob/8516d0a0f1dbd352d3ef6455c2d41c94661ce1ce/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java#L776-L854
   
   Point 3. line_840, it will put  completionKeyValue to completionObjects.
   
   Point 4. line_852, if the channel is not null, invoke writeAndFlush.
   
   There is a race condition between PerChannelBookieClient#channelInactive and 
PerChannelBookieClient#addEntry.
   
   There is the timeline.
   
   Point 1 -> Point 3 -> Point 4 -> Point 2.
   
   It will write and flush the AddRequest to the netty channel. In the 
bookkeeper, there is a weakness in PerChannelBookieClient#writeAndFlush. 
https://github.com/apache/bookkeeper/blob/8516d0a0f1dbd352d3ef6455c2d41c94661ce1ce/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java#L1176-L1279
   
   At line_1230, we define a promise for the netty write and flush, if the 
write and flush failed, we only record the metrics at line_1211, not remove the 
completionKey from completionObjects. The completionKey will leak in the 
completionObjects.
   If the PerChannelBookieClient#addEntryTimeoutNanos is disabled, the 
timeoutCheck won't work, so the completionKey exists in the completionObjects 
forever.
   
   In #4278, the PerChannelBookieClient#addEntryTimeoutNanos is enabled, So you 
will see that 5 seconds(default timeout) passed after bk1 died, the step 9 
trigger PendingAddOp timeout, then cause the issue in #4278.
   
   So, as long as we remove the completionKey from completionObjects when write 
and flush AddRequest failed, we can solve the problem.
   
   
   


-- 
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: commits-unsubscr...@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to