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

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

bq, If there's a seek to next row and optimize decides to skip instead of seek, 
that should be the better decision (less overall work to do). 

[~lhofhansl] Is this true? If a row of 1M columns and we are looking for the 
second column only, SKIP'ing through the rest is what we want?

bq. With that I do not see how optimize can make things worse in terms of 
number of blocks read. If the SEEK would get us to another block, optimize 
allows the seek, otherwise it converts it to INCLUDE or SKIP.

Optimize is squashing the SEEK making it an INCLUDE or a SKIP instead when the 
index key is >= current cell.

bq. How would optimize cause us reading further than we would have without? I 
am missing something.

Because it suppresses the SEEK, turning it into a SKIP or INCLUDE, then we are 
skirting the check of there being any more rows in this scan.... so we keep 
going... till we hit the next row... (loading blocks if we have to) and only 
then, do we realize we are actually done.

bq. How can reproduce this easily?

I made a test included in this patch. It is an ugly harness but seems to show 
the condition.

bq.  That's an extra compare, possibly two, we've been trying so hard to remove 
them. This should be done for Gets only. .... -1 on doing that always.

What if we do the compare when a stopRow specified or a Get (yucky).

bq. In the end, I'm still a fan of my -looksee change.

-looksee is good. Doesn't do anything for the case when stopRows are sprecified 
(here again we can over read) and it could make broader use of the fact that we 
know there are no more rows if the Scan is a Get Scan.

bq. What optimize is trying to figure out is whether the SEEK would get us onto 
a new HFileBlock, if so, it'll do the SEEK.

I don't see this. Let me mess w/ my test some more. Must be my test.

bq. So why isn't it working?

Because when it seems to suppress the SEEK swapping it for a SKIP or INCLUDE?

[~ram_krish] you ask:

bq. ....or it can also be lesser?

It'd be a bad bug if it could be less.












> 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, no_optimize.patch, no_optimize.patch, 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)

Reply via email to