Skip to content

Commit

Permalink
HBASE-27180 Fix multiple possible buffer leaks (#4597)
Browse files Browse the repository at this point in the history
* Fix multiple possible buffer leaks

Motivation:

When using ByteBuf you need to be very careful about releasing it as otherwise you might leak data. There were various places in the code-base where such a leak could happen.

Modifications:

- Fix possible buffer leaks
- Ensure we call touch(...) so its easier to debug buffer leaks

Result:

Fix buffer leaks

* Formatting

* Revert some changes as requested

* revert touch

* Also release checksum and header buffers

Signed-off-by: Duo Zhang <zhangduo@apache.org>
  • Loading branch information
normanmaurer committed Jul 8, 2022
1 parent 22618da commit 2197b38
Show file tree
Hide file tree
Showing 3 changed files with 10 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -429,6 +429,12 @@ private void flushBuffer(CompletableFuture<Long> future, ByteBuf dataBuf,
future.completeExceptionally(new IOException("stream already broken"));
// it's the one we have just pushed or just a no-op
waitingAckQueue.removeFirst();

checksumBuf.release();
headerBuf.release();

// This method takes ownership of the dataBuf so we need release it before returning.
dataBuf.release();
return;
}
// TODO: we should perhaps measure time taken per DN here;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -662,7 +662,8 @@ public void flush(ChannelHandlerContext ctx) throws Exception {
}

@Override
public void close(ChannelHandlerContext ctx, ChannelPromise promise) throws Exception {
public void handlerRemoved(ChannelHandlerContext ctx) throws Exception {
// Release buffer on removal.
cBuf.release();
cBuf = null;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -185,6 +185,8 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception
public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
if (!(msg instanceof ByteBuf)) {
ctx.fireChannelRead(msg);
} else {
((ByteBuf) msg).release();
}
}
});
Expand Down

0 comments on commit 2197b38

Please sign in to comment.