hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "stack (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HBASE-15392) Single Cell Get reads two HFileBlocks
Date Sat, 05 Mar 2016 06:09:40 GMT

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

stack commented on HBASE-15392:
-------------------------------

Thanks for taking a look [~ramkrishna]

bq. Is this really solving your problem?

It for sure solves my reading two blocks on each get.

bq.  But any way the compare would say that the stoprow has reached and the method would have
returned false anyway.

There is no 'stoprow' when we are doing a Get. There is only the row we were doing the 'Get'
on.

Your comment made me look at all places where moreRowsMayExistAfter is called and it seems
that we should volunteer this knowledge we have if anyone asks, that we KNOW there are no
more rows beyond the current Get/Scan row.

Debugging, down in the SQM#match, the call to checkVersions here...

{code}
...
       * In all the above scenarios, we return the column checker return value except for
       * FilterResponse (INCLUDE_AND_SEEK_NEXT_COL) and ColumnChecker(INCLUDE)
       */
      colChecker = columns.checkVersions(cell, timestamp, typeByte,
          mvccVersion > maxReadPointToTrackVersions);
      //Optimize with stickyNextRow
....
{code}

...comes back with INCLUDE_AND_SEEK_NEXT_ROW if we asked for one column only and if more than
one column specified in the Get, we'd return INCLUDE_AND_NEXT_COL until we'd got the last
asked-for column, and then we'd get a INCLUDE_AND_SEEK_NEXT_ROW back (when INCLUDE_AND_SEEK_NEXT_ROW
is returned out columns.checkVersions, the done() call is coming back true -- all versions
and all columns have been INCLUDED).

So... rationale is that when we come up out of the match AND the return is INCLUDE_AND_SEEK_NEXT_ROW
and its a Get Scan, then we should be done. We are already doing the call to moreRowsMayExistAfter
to see if we've hit the end row or stop row if provided... seemed natural to add the check
here.

Let me know if it makes sense Ram.






> 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.wip.patch
>
>
> 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