hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Lars Hofhansl (Commented) (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HBASE-5001) Improve the performance of block cache keys
Date Sun, 11 Dec 2011 02:34:41 GMT

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

Lars Hofhansl commented on HBASE-5001:
--------------------------------------

Did a quick micro-benchmark:

* hfileName + CACHE_KEY_SEPARATOR + offset -> 0.34us
* new StringBuilder(hfileName).append(CACHE_KEY_SEPARATOR).append(offset).toString() ->
0.19us
* new StringBuilder(hfileName).append(offset).toString() -> 0.183us

StringBuilder(String) reserves an extra 16 characters, so as long as HFiles are < 1PB,
this works without extra allocations, and we can save a lot.

One the other hand I have a hard time believing that fractions of microseconds per cache block
access would make a difference.

                
> 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
>            Priority: Minor
>             Fix For: 0.94.0
>
>
> 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

        

Mime
View raw message