You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@bookkeeper.apache.org by GitBox <gi...@apache.org> on 2018/08/08 06:48:02 UTC

[GitHub] dlg99 commented on a change in pull request #1585: Issue #1584: LedgerHandleAdv should expose asyncAddEntry variant that takes ByteBuf (LedgerHandle exposes it as public)

dlg99 commented on a change in pull request #1585: Issue #1584: LedgerHandleAdv should expose asyncAddEntry variant that takes ByteBuf (LedgerHandle exposes it as public)
URL: https://github.com/apache/bookkeeper/pull/1585#discussion_r208472061
 
 

 ##########
 File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java
 ##########
 @@ -1067,7 +1067,6 @@ public void asyncAddEntry(final byte[] data, final int offset, final int length,
     }
 
     public void asyncAddEntry(ByteBuf data, final AddCallback cb, final Object ctx) {
-        data.retain();
 
 Review comment:
   This thing grew a bit out of "let's make method public" change ;)
   
   I'll start with DigestManager:
   it handles V2 and V3 protocol.
   For V2 protocol it returns ByteBufList which wraps received ByteBuf data and needs it until ByteBufList is released.
   ByteBufList (aka toSend object in PendingAddOp) is released when callback is triggered. On ByteBufList release it callse .release() on all its parts, including ByteBuf data.
   
   For V3 protocol currently it copies content into new unpooled ByteBuf, original data is no longer needed and no longer used so it should be released. This keeps behavior consistent for v2 and v3 in regards of ownership expectations of data (as in: computeDigestAndPackageForSending gets ownership of 1 refCnt of the passed ByteBuf and releases it when data is no longer needed or passes to a component that will do it).
   
   This matches netty's "general rule of thumb is that the party who accesses a reference-counted object lastly is responsible for the destruction of the reference-counted object."
   
   PendingAddOp keeps reference to payload.
   Looking closer at it, there is no need to retain data unless we decide to i.e. log it after callback triggered and toSend was released. I'll actually remove it and re-run bookie tests locally, probably will push the change.
   in this case payload's release in PendingAddOp should happen in recycle() only if payload was never used (hasRun == false). 
   
   If I understood correctly, retain() call was added without accompanying release to fix failing test(s), i.e. DeferredSyncTest. This works with i.e. buffers produced by Unpooled.wrappedBuffer(..) but not when one tries to use Pooled allocator where unreleased reference will lead to a leak. I approached it by changing the test.
   
   I understand your concerns about dlog. I propose the following:
   
   1. Let's agree on the ownership rules for the ByteBufs and fix the client to use these rules consistently.
   
   2. Let's review dlog's code. Any API calls in write path that pass ByteBuf directly are suspects, as well as parts where .slice() or .duplicate() is called. Tests caught one place where ref count had to be incremented. The rule of thumb is if ByteBuf is used after being passed somewhere, it has to be retained.
   I am not familiar with dlog's codebase so I'll need help. We can add asserts on recCnt to existing tests, similar to ones in the new tests I added.
   Existing tests caught few cases of reusing released ByteBuf (EnvelopedEntryWriter.getBuffer() ) and I did not find any other suspects after brief look at the dlog's code.
   We can try enabling paranoid leak detection in unit tests, but at least initially it will catch false positives related to i.e. keeping references like
   static final ByteBuf DATA = Unpooled.wrappedBuffer("foobar".getBytes()); 
   and never releasing them at the end of the test. These won't be hard to fix but tedious.
   
   3. We can add a warning to release notes. 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services