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

Todd Lipcon commented on HBASE-5001:
------------------------------------

haha :) My feeling is mostly that profilers point you in the right direction, 
but it's still worth doing a more macro-benchmark to know that you're really 
making a difference with the "fixes". But I'm all for shaving off percents here 
and there - they do add up over time!

Regarding IdLock, we could probably replace it with something like TIntHashSet 
from trove4j to avoid boxing costs... though that is LGPL so we'd need to do a 
cleanroom implementation or find a different implementation of similar.
                
> Improve the performance of block cache keys
> -------------------------------------------
>
>                 Key: HBASE-5001
>                 URL: https://issues.apache.org/jira/browse/HBASE-5001
>             Project: HBase
>          Issue Type: Improvement
>    Affects Versions: 0.90.4
>            Reporter: Jean-Daniel Cryans
>            Assignee: Lars Hofhansl
>            Priority: Minor
>             Fix For: 0.92.0
>
>         Attachments: 5001-0.92.txt, 5001-v1.txt, 5001-v2.txt
>
>
> Doing a pure random read test on data that's 100% block cache, I see that we 
> are spending quite some time in getBlockCacheKey:
> {quote}
> "IPC Server handler 19 on 62023" daemon prio=10 tid=0x00007fe0501ff800 
> nid=0x6c87 runnable [0x00007fe0577f6000]
>    java.lang.Thread.State: RUNNABLE
>       at java.util.Arrays.copyOf(Arrays.java:2882)
>       at 
> java.lang.AbstractStringBuilder.expandCapacity(AbstractStringBuilder.java:100)
>       at 
> java.lang.AbstractStringBuilder.append(AbstractStringBuilder.java:390)
>       at java.lang.StringBuilder.append(StringBuilder.java:119)
>       at 
> org.apache.hadoop.hbase.io.hfile.HFile.getBlockCacheKey(HFile.java:457)
>       at 
> org.apache.hadoop.hbase.io.hfile.HFileReaderV2.readBlock(HFileReaderV2.java:249)
>       at 
> org.apache.hadoop.hbase.io.hfile.HFileBlockIndex$BlockIndexReader.seekToDataBlock(HFileBlockIndex.java:209)
>       at 
> org.apache.hadoop.hbase.io.hfile.HFileReaderV2$ScannerV2.seekTo(HFileReaderV2.java:521)
>       at 
> org.apache.hadoop.hbase.io.hfile.HFileReaderV2$ScannerV2.seekTo(HFileReaderV2.java:536)
>       at 
> org.apache.hadoop.hbase.regionserver.StoreFileScanner.seekAtOrAfter(StoreFileScanner.java:178)
>       at 
> org.apache.hadoop.hbase.regionserver.StoreFileScanner.seek(StoreFileScanner.java:111)
>       at 
> org.apache.hadoop.hbase.regionserver.StoreFileScanner.seekExactly(StoreFileScanner.java:219)
>       at 
> org.apache.hadoop.hbase.regionserver.StoreScanner.<init>(StoreScanner.java:80)
>       at 
> org.apache.hadoop.hbase.regionserver.Store.getScanner(Store.java:1689)
>       at 
> org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.<init>(HRegion.java:2857)
> {quote}
> Since the HFile name size is known and the offset is a long, it should be 
> possible to allocate exactly what we need. Maybe use byte[] as the key and 
> drop the separator too.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to