hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Lars Hofhansl (JIRA)" <j...@apache.org>
Subject [jira] [Comment Edited] (HBASE-15392) Single Cell Get reads two HFileBlocks
Date Thu, 10 Mar 2016 07:03:40 GMT

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

Lars Hofhansl edited comment on HBASE-15392 at 3/10/16 7:03 AM:
----------------------------------------------------------------

We're optimizing the wrong thing here. Let's fix it for Gets.
[~anoopsamjohn], I was surprised by the cost of SEEKs as well. I've spent some time to make
that better, but at the end we have to reset the pointers for every HFile we're touching.
That means SEEKing in the right HFile block of every HFile. Then there are a bunch of compares
in KeyValueHeap and the HFileReader classes. Note that is true whether we are reading a new
block from disk or from the block cache, we're doing a bunch of work for every SEEK.

The case I missed is that moreRowsMayExistAfter is not called unless we're getting a XXX_SEEK
code. My bad. Let's fix that part. For Gets for sure, we should fix it.

For Scans, unless we have a degenerate case (a very small scan that fits into one HFile block),
we might load an extra block if the StopRow falls towards the end of a block, which is far
outweighed by the time we've saved during the scan. That block will a read of the same file
and it will be the _next_ block to read (so not a random read, but a sequential read). That
is a fair trade-off. IMHO. HBase is usually CPU bound (which is in part what I am trying to
fix)

Maybe we can disable the next row optimization for small scans as well.

Anyway... If you want to remove optimize() or change it significantly, show me end-to-end
perf numbers (like I have done in HBASE-13109). :)

Areas where it might be degenerate are the following:
* Gets.. I missed that.
* Large Cells, such that only a few (maybe just one) fit into into an HFileBlock.
* Very small scans (that just read a single block, or very few block)



was (Author: lhofhansl):
We're optimizing the wrong thing here. Let's fix it for Gets.
[~anoopsamjohn], I was surprised by the cost of SEEKs as well. I've spent some time to make
that better, but at the we have reset the pointers for every HFile we're touching. That means
SEEKing in the right HFile block of every HFile. Then there are a bunch of compares in KeyValueHeap
and the HFileReader classes. Note that is true whether we are reading a new block from disk
or from the block cache, we're doing a bunch of work for every SEEK.

The case I missed is that moreRowsMayExistAfter is not called unless we're getting a XXX_SEEK
code. My bad. Let's fix that part. For Gets for sure, we should fix it.

For Scans, unless we have a degenerate case (a very small scan that fits into one HFile block),
we might load an extra block if the StopRow falls towards the end of a block, which is far
outweighed by the time we've saved during the scan. That block will a read of the same file
and it will be the _next_ block to read (so not a random read, but a sequential read). That
is a fair trade-off. IMHO.
(Maybe we can disable the next row optimization for small scans as well.)

Anyway... If you want to remove optimize() or change it significantly, show me end-to-end
perf numbers (like I have done in HBASE-13109). :)

Areas where it might be degenerate are the following:
* Gets.. I missed that.
* Large Cells, such that only a few (maybe just one) fit into into an HFileBlock.
* Very small scans (that just read a single block, or very few block)


> Single Cell Get reads two HFileBlocks
> -------------------------------------
>
>                 Key: HBASE-15392
>                 URL: https://issues.apache.org/jira/browse/HBASE-15392
>             Project: HBase
>          Issue Type: Sub-task
>          Components: BucketCache
>            Reporter: stack
>            Assignee: stack
>         Attachments: 15392-0.98-looksee.txt, 15392.wip.patch, 15392v2.wip.patch, 15392v3.wip.patch,
15392v4.patch, HBASE-15392_suggest.patch, gc.png, gc.png, io.png, no_optimize.patch, no_optimize.patch,
reads.png, reads.png, two_seeks.txt
>
>
> As found by Daniel "SystemTap" Pol, a simple Get results in our reading two HFileBlocks,
the one that contains the wanted Cell, and the block that follows.
> Here is a bit of custom logging that logs a stack trace on each HFileBlock read so you
can see the call stack responsible:
> {code}
> 2016-03-03 22:20:30,191 INFO  [B.defaultRpcServer.handler=20,queue=2,port=16020] regionserver.StoreScanner:
START LOOP
> 2016-03-03 22:20:30,192 INFO  [B.defaultRpcServer.handler=20,queue=2,port=16020] regionserver.StoreScanner:
QCODE SEEK_NEXT_COL
> 2016-03-03 22:20:30,192 INFO  [B.defaultRpcServer.handler=20,queue=2,port=16020] hfile.HFileBlockIndex:
STARTED WHILE
> 2016-03-03 22:20:30,192 INFO  [B.defaultRpcServer.handler=20,queue=2,port=16020] hfile.CombinedBlockCache:
OUT OF L2
> 2016-03-03 22:20:30,192 TRACE [B.defaultRpcServer.handler=20,queue=2,port=16020] bucket.BucketCache:
Read offset=31409152, len=2103
> 2016-03-03 22:20:30,192 TRACE [B.defaultRpcServer.handler=20,queue=2,port=16020] bucket.FileIOEngine:
offset=31409152, length=2103
> 2016-03-03 22:20:30,193 TRACE [B.defaultRpcServer.handler=20,queue=2,port=16020] hfile.HFileReaderImpl:
From Cache [blockType=DATA, fileOffset=2055421, headerSize=33, onDiskSizeWithoutHeader=2024,
uncompressedSizeWithoutHeader=2020, prevBlockOffset=2053364, isUseHBaseChecksum=true, checksumType=CRC32C,
bytesPerChecksum=16384, onDiskDataSizeWithHeader=2053, getOnDiskSizeWithHeader=2057, totalChecksumBytes=4,
isUnpacked=true, buf=[org.apache.hadoop.hbase.nio.SingleByteBuff@e19fbd54], dataBeginsWith=\x00\x00\x00)\x00\x00\x01`\x00\x16user995139035672819231,
fileContext=[usesHBaseChecksum=true, checksumType=CRC32C, bytesPerChecksum=16384, blocksize=65536,
encoding=NONE, includesMvcc=true, includesTags=false, compressAlgo=NONE, compressTags=false,
cryptoContext=[cipher=NONE keyHash=NONE]]]
> 2016-03-03 22:20:30,193 TRACE [B.defaultRpcServer.handler=20,queue=2,port=16020] hfile.HFileReaderImpl:
Cache hit return [blockType=DATA, fileOffset=2055421, headerSize=33, onDiskSizeWithoutHeader=2024,
uncompressedSizeWithoutHeader=2020, prevBlockOffset=2053364, isUseHBaseChecksum=true, checksumType=CRC32C,
bytesPerChecksum=16384, onDiskDataSizeWithHeader=2053, getOnDiskSizeWithHeader=2057, totalChecksumBytes=4,
isUnpacked=true, buf=[org.apache.hadoop.hbase.nio.SingleByteBuff@e19fbd54], dataBeginsWith=\x00\x00\x00)\x00\x00\x01`\x00\x16user995139035672819231,
fileContext=[usesHBaseChecksum=true, checksumType=CRC32C, bytesPerChecksum=16384, blocksize=65536,
encoding=NONE, includesMvcc=true, includesTags=false, compressAlgo=NONE, compressTags=false,
cryptoContext=[cipher=NONE keyHash=NONE]]]
> java.lang.Throwable
>         at org.apache.hadoop.hbase.io.hfile.HFileReaderImpl.readBlock(HFileReaderImpl.java:1515)
>         at org.apache.hadoop.hbase.io.hfile.HFileBlockIndex$CellBasedKeyBlockIndexReader.loadDataBlockWithScanInfo(HFileBlockIndex.java:324)
>         at org.apache.hadoop.hbase.io.hfile.HFileReaderImpl$HFileScannerImpl.seekTo(HFileReaderImpl.java:831)
>         at org.apache.hadoop.hbase.io.hfile.HFileReaderImpl$HFileScannerImpl.reseekTo(HFileReaderImpl.java:812)
>         at org.apache.hadoop.hbase.regionserver.StoreFileScanner.reseekAtOrAfter(StoreFileScanner.java:288)
>         at org.apache.hadoop.hbase.regionserver.StoreFileScanner.reseek(StoreFileScanner.java:198)
>         at org.apache.hadoop.hbase.regionserver.NonLazyKeyValueScanner.doRealSeek(NonLazyKeyValueScanner.java:54)
>         at org.apache.hadoop.hbase.regionserver.KeyValueHeap.generalizedSeek(KeyValueHeap.java:321)
>         at org.apache.hadoop.hbase.regionserver.KeyValueHeap.requestSeek(KeyValueHeap.java:279)
>         at org.apache.hadoop.hbase.regionserver.StoreScanner.reseek(StoreScanner.java:806)
>         at org.apache.hadoop.hbase.regionserver.StoreScanner.seekAsDirection(StoreScanner.java:795)
>         at org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:624)
>         at org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:153)
>         at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5703)
>         at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:5849)
>         at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5622)
>         at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.next(HRegion.java:5598)
>         at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.next(HRegion.java:5584)
>         at org.apache.hadoop.hbase.regionserver.RSRpcServices.get(RSRpcServices.java:2187)
>         at org.apache.hadoop.hbase.regionserver.RSRpcServices.get(RSRpcServices.java:2123)
>         at org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:33512)
>         at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2229)
>         at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:109)
>         at org.apache.hadoop.hbase.ipc.RpcExecutor.consumerLoop(RpcExecutor.java:136)
>         at org.apache.hadoop.hbase.ipc.RpcExecutor$1.run(RpcExecutor.java:111)
>         at java.lang.Thread.run(Thread.java:745)
> 2016-03-03 22:20:30,193 INFO  [B.defaultRpcServer.handler=20,queue=2,port=16020] hfile.HFileBlockIndex:
READ [blockType=DATA, fileOffset=2055421, headerSize=33, onDiskSizeWithoutHeader=2024, uncompressedSizeWithoutHeader=2020,
prevBlockOffset=2053364, isUseHBaseChecksum=true, checksumType=CRC32C, bytesPerChecksum=16384,
onDiskDataSizeWithHeader=2053, getOnDiskSizeWithHeader=2057, totalChecksumBytes=4, isUnpacked=true,
buf=[org.apache.hadoop.hbase.nio.SingleByteBuff@e19fbd54], dataBeginsWith=\x00\x00\x00)\x00\x00\x01`\x00\x16user995139035672819231,
fileContext=[usesHBaseChecksum=true, checksumType=CRC32C, bytesPerChecksum=16384, blocksize=65536,
encoding=NONE, includesMvcc=true, includesTags=false, compressAlgo=NONE, compressTags=false,
cryptoContext=[cipher=NONE keyHash=NONE]]]
> 2016-03-03 22:20:30,193 INFO  [B.defaultRpcServer.handler=20,queue=2,port=16020] hfile.HFileBlockIndex:
DATA BLOCK IS TRUE BREAK
> 2016-03-03 22:20:30,193 INFO  [B.defaultRpcServer.handler=20,queue=2,port=16020] hfile.HFileBlockIndex:
RETURNING
> 2016-03-03 22:20:30,194 INFO  [B.defaultRpcServer.handler=20,queue=2,port=16020] regionserver.StoreScanner:
QCODE INCLUDE
> 2016-03-03 22:20:30,194 INFO  [B.defaultRpcServer.handler=20,queue=2,port=16020] regionserver.StoreScanner:
QCODE SKIP
> 2016-03-03 22:20:30,194 INFO  [B.defaultRpcServer.handler=20,queue=2,port=16020] hfile.CombinedBlockCache:
OUT OF L2
> 2016-03-03 22:20:30,194 TRACE [B.defaultRpcServer.handler=20,queue=2,port=16020] bucket.BucketCache:
Read offset=30932992, len=2103
> 2016-03-03 22:20:30,194 TRACE [B.defaultRpcServer.handler=20,queue=2,port=16020] bucket.FileIOEngine:
offset=30932992, length=2103
> 2016-03-03 22:20:30,194 TRACE [B.defaultRpcServer.handler=20,queue=2,port=16020] hfile.HFileReaderImpl:
From Cache [blockType=DATA, fileOffset=2057478, headerSize=33, onDiskSizeWithoutHeader=2024,
uncompressedSizeWithoutHeader=2020, prevBlockOffset=2055421, isUseHBaseChecksum=true, checksumType=CRC32C,
bytesPerChecksum=16384, onDiskDataSizeWithHeader=2053, getOnDiskSizeWithHeader=2057, totalChecksumBytes=4,
isUnpacked=true, buf=[org.apache.hadoop.hbase.nio.SingleByteBuff@6063ac95], dataBeginsWith=\x00\x00\x00)\x00\x00\x01`\x00\x16user995698996184959679,
fileContext=[usesHBaseChecksum=true, checksumType=CRC32C, bytesPerChecksum=16384, blocksize=65536,
encoding=NONE, includesMvcc=true, includesTags=false, compressAlgo=NONE, compressTags=false,
cryptoContext=[cipher=NONE keyHash=NONE]]]
> 2016-03-03 22:20:30,194 TRACE [B.defaultRpcServer.handler=20,queue=2,port=16020] hfile.HFileReaderImpl:
Cache hit return [blockType=DATA, fileOffset=2057478, headerSize=33, onDiskSizeWithoutHeader=2024,
uncompressedSizeWithoutHeader=2020, prevBlockOffset=2055421, isUseHBaseChecksum=true, checksumType=CRC32C,
bytesPerChecksum=16384, onDiskDataSizeWithHeader=2053, getOnDiskSizeWithHeader=2057, totalChecksumBytes=4,
isUnpacked=true, buf=[org.apache.hadoop.hbase.nio.SingleByteBuff@6063ac95], dataBeginsWith=\x00\x00\x00)\x00\x00\x01`\x00\x16user995698996184959679,
fileContext=[usesHBaseChecksum=true, checksumType=CRC32C, bytesPerChecksum=16384, blocksize=65536,
encoding=NONE, includesMvcc=true, includesTags=false, compressAlgo=NONE, compressTags=false,
cryptoContext=[cipher=NONE keyHash=NONE]]]
> java.lang.Throwable
>         at org.apache.hadoop.hbase.io.hfile.HFileReaderImpl.readBlock(HFileReaderImpl.java:1515)
>         at org.apache.hadoop.hbase.io.hfile.HFileReaderImpl$HFileScannerImpl.readNextDataBlock(HFileReaderImpl.java:906)
>         at org.apache.hadoop.hbase.io.hfile.HFileReaderImpl$HFileScannerImpl.isNextBlock(HFileReaderImpl.java:1106)
>         at org.apache.hadoop.hbase.io.hfile.HFileReaderImpl$HFileScannerImpl.positionForNextBlock(HFileReaderImpl.java:1100)
>         at org.apache.hadoop.hbase.io.hfile.HFileReaderImpl$HFileScannerImpl._next(HFileReaderImpl.java:1118)
>         at org.apache.hadoop.hbase.io.hfile.HFileReaderImpl$HFileScannerImpl.next(HFileReaderImpl.java:1139)
>         at org.apache.hadoop.hbase.regionserver.StoreFileScanner.next(StoreFileScanner.java:152)
>         at org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:114)
>         at org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:628)
>         at org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:153)
>         at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5703)
>         at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:5849)
>         at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5622)
>         at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.next(HRegion.java:5598)
>         at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.next(HRegion.java:5584)
>         at org.apache.hadoop.hbase.regionserver.RSRpcServices.get(RSRpcServices.java:2187)
>         at org.apache.hadoop.hbase.regionserver.RSRpcServices.get(RSRpcServices.java:2123)
>         at org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:33512)
>         at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2229)
>         at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:109)
>         at org.apache.hadoop.hbase.ipc.RpcExecutor.consumerLoop(RpcExecutor.java:136)
>         at org.apache.hadoop.hbase.ipc.RpcExecutor$1.run(RpcExecutor.java:111)
>         at java.lang.Thread.run(Thread.java:745)
> 2016-03-03 22:20:30,195 INFO  [B.defaultRpcServer.handler=20,queue=2,port=16020] regionserver.StoreScanner:
QCODE DONE
> {code}
> We are in StoreScanner#next.
> Matcher does SEEK_NEXT_COL.
> We pull in the block that has our Cell in it.
> Matcher does INCLUDE
> Then Matcher does SKIP
> SKIP has us go read the next block.
> Then Matcher does DONE.
> Why ain't I getting to DONE after INCLUDE?
> Any clues?
> This code has been like this a while.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message