Return-Path: X-Original-To: apmail-hbase-issues-archive@www.apache.org Delivered-To: apmail-hbase-issues-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 7146319904 for ; Thu, 10 Mar 2016 14:32:44 +0000 (UTC) Received: (qmail 2165 invoked by uid 500); 10 Mar 2016 14:32:41 -0000 Delivered-To: apmail-hbase-issues-archive@hbase.apache.org Received: (qmail 2128 invoked by uid 500); 10 Mar 2016 14:32:41 -0000 Mailing-List: contact issues-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list issues@hbase.apache.org Received: (qmail 1998 invoked by uid 99); 10 Mar 2016 14:32:41 -0000 Received: from arcas.apache.org (HELO arcas) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 10 Mar 2016 14:32:41 +0000 Received: from arcas.apache.org (localhost [127.0.0.1]) by arcas (Postfix) with ESMTP id 01DCC2C1F6A for ; Thu, 10 Mar 2016 14:32:41 +0000 (UTC) Date: Thu, 10 Mar 2016 14:32:41 +0000 (UTC) From: "Daniel Pol (JIRA)" To: issues@hbase.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (HBASE-15392) Single Cell Get reads two HFileBlocks MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/HBASE-15392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15189361#comment-15189361 ] Daniel Pol commented on HBASE-15392: ------------------------------------ Anoop - thanks for clarifying that its skip vs seek inside a block. On the performance side. I've tested with patch with L2 cache on SSD and it doubles my performance for single cell gets in my case. In my case 1 row fits evenly in a block. The performance for single row gets was not improved. It looks to me that if we have the row/cell info in indexes we should be able to tell from indexes where are all the blocks for a single row. And since indexes are in L1 (at least for all my customers) that should be faster. Seems there's optimization work being done that makes sense for rotational HDD (HDFS use) but its not helping SSDs (BC use). Have you heard of any customer putting BC on rotational HDD ? What would be the point of that since the HDFS data is already on rotational HDD and probably on more of them ? > 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)