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

chenxu edited comment on HBASE-22802 at 8/7/19 1:07 AM:
--------------------------------------------------------

{quote}so we need to read ByteBuffer one by one for a MultiByteBuff ?
{quote}
yes, we can declare a method like read(FileChannel channel, long offset) in 
ByteBuff
{quote}When to release the ByteBuff ?
{quote}
I think the ByteBuff will be released in HFileScannerImpl#close, is that right?
 and there is no need to share the same refCnt with BucketEntry like what 
ByteBufferIOEngine to do (code in BucketEntry#wrapAsCacheable)


was (Author: javaman_chen):
bq. so we need to read ByteBuffer one by one for a MultiByteBuff ?
yes, we can declare a method like read(FileChannel channel, long offset) in 
ByteBuff
bq. When to release the ByteBuff ?
I think the ByteBuff will be released it in HFileScannerImpl#close, is that 
right?
and there is no need to share the same refCnt with BucketEntry like what 
ByteBufferIOEngine to do (code in BucketEntry#wrapAsCacheable)

> Avoid temp ByteBuffer allocation in FileIOEngine#read
> -----------------------------------------------------
>
>                 Key: HBASE-22802
>                 URL: https://issues.apache.org/jira/browse/HBASE-22802
>             Project: HBase
>          Issue Type: Improvement
>          Components: BucketCache
>            Reporter: chenxu
>            Priority: Major
>         Attachments: profile_mem_alloc.png
>
>
> a temp ByteBuffer was allocated each time FileIOEngine#read was called
> {code:java}
> public Cacheable read(BucketEntry be) throws IOException {
>   long offset = be.offset();
>   int length = be.getLength();
>   Preconditions.checkArgument(length >= 0, "Length of read can not be less 
> than 0.");
>   ByteBuffer dstBuffer = ByteBuffer.allocate(length);
>   ...
> }
> {code}
> we can avoid this by use of ByteBuffAllocator#allocate(length) after 
> HBASE-21879



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)

Reply via email to