Reading compressed HFile blocks causes way too many DFS RPC calls severly 
impacting performance
-----------------------------------------------------------------------------------------------

                 Key: HBASE-3006
                 URL: https://issues.apache.org/jira/browse/HBASE-3006
             Project: HBase
          Issue Type: Bug
    Affects Versions: 0.89.20100621
            Reporter: Kannan Muthukkaruppan
            Priority: Critical


On some read perf tests, we noticed several perf outliers (10 second plus 
range). The rows were large (spanning multiple blocks, but still the numbers 
didn't add up). We had compression turned on.

We enabled DN clienttrace logging,
log4j.logger.org.apache.hadoop.hdfs.server.datanode.DataNode.clienttrace=DEBUG

and noticed lots of 516 byte reads at the DN level, several of them at the same 
offset in the block.

{code}
2010-09-16 09:28:32,335 INFO 
org.apache.hadoop.hdfs.server.datanode.DataNode.clienttrace: src: 
/10.30.251.189:50010, dest: /10.30.251.189:38713, bytes: 516, op: HDFS_READ, 
cliID: DFSClient_-436329957, offset: 39884800, srvID: DS-1757894045-10.3\
0.251.189-50010-1283993662994, blockid: blk_-4686540439725119008_1985, 
duration: 203000
2010-09-16 09:28:32,336 INFO 
org.apache.hadoop.hdfs.server.datanode.DataNode.clienttrace: src: 
/10.30.251.189:50010, dest: /10.30.251.189:40547, bytes: 516, op: HDFS_READ, 
cliID: DFSClient_-436329957, offset: 39884800, srvID: DS-1757894045-10.3\
0.251.189-50010-1283993662994, blockid: blk_-4686540439725119008_1985, 
duration: 119000
2010-09-16 09:28:32,337 INFO 
org.apache.hadoop.hdfs.server.datanode.DataNode.clienttrace: src: 
/10.30.251.189:50010, dest: /10.30.251.189:40650, bytes: 516, op: HDFS_READ, 
cliID: DFSClient_-436329957, offset: 39884800, srvID: DS-1757894045-10.3\
0.251.189-50010-1283993662994, blockid: blk_-4686540439725119008_1985, 
duration: 149000
2010-09-16 09:28:32,337 INFO 
org.apache.hadoop.hdfs.server.datanode.DataNode.clienttrace: src: 
/10.30.251.189:50010, dest: /10.30.251.189:40861, bytes: 516, op: HDFS_READ, 
cliID: DFSClient_-436329957, offset: 39884800, srvID: DS-1757894045-10.3\
0.251.189-50010-1283993662994, blockid: blk_-4686540439725119008_1985, 
duration: 135000
2010-09-16 09:28:32,338 INFO 
org.apache.hadoop.hdfs.server.datanode.DataNode.clienttrace: src: 
/10.30.251.189:50010, dest: /10.30.251.189:41129, bytes: 516, op: HDFS_READ, 
cliID: DFSClient_-436329957, offset: 39884800, srvID: DS-1757894045-10.3\
0.251.189-50010-1283993662994, blockid: blk_-4686540439725119008_1985, 
duration: 117000
2010-09-16 09:28:32,339 INFO 
org.apache.hadoop.hdfs.server.datanode.DataNode.clienttrace: src: 
/10.30.251.189:50010, dest: /10.30.251.189:41691, bytes: 516, op: HDFS_READ, 
cliID: DFSClient_-436329957, offset: 39884800, srvID: DS-1757894045-10.3\
0.251.189-50010-1283993662994, blockid: blk_-4686540439725119008_1985, 
duration: 148000
2010-09-16 09:28:32,339 INFO 
org.apache.hadoop.hdfs.server.datanode.DataNode.clienttrace: src: 
/10.30.251.189:50010, dest: /10.30.251.189:42881, bytes: 516, op: HDFS_READ, 
cliID: DFSClient_-436329957, offset: 39884800, srvID: DS-1757894045-10.3\
0.251.189-50010-1283993662994, blockid: blk_-4686540439725119008_1985, 
duration: 114000
2010-09-16 09:28:32,341 INFO 
org.apache.hadoop.hdfs.server.datanode.DataNode.clienttrace: src: 
/10.30.251.189:50010, dest: /10.30.251.189:49511, bytes: 516, op: HDFS_READ, 
cliID: DFSClient_-436329957, offset: 39884800, srvID: DS-1757894045-10.3\
0.251.189-50010-1283993662994, blockid: blk_-4686540439725119008_1985, 
duration: 153000
2010-09-16 09:28:32,342 INFO 
org.apache.hadoop.hdfs.server.datanode.DataNode.clienttrace: src: 
/10.30.251.189:50010, dest: /10.30.251.189:51158, bytes: 3096, op: HDFS_READ, 
cliID: DFSClient_-436329957, offset: 39884800, srvID: DS-1757894045-10.\
30.251.189-50010-1283993662994, blockid: blk_-4686540439725119008_1985, 
duration: 139000
{code}

This was strange coz our block size was 64k, and on disk block size after 
compression should generally have been around 6k.

Some print debugging at the HFile and BoundedRangeFileInputStream (which is 
wrapped by createDecompressionStream) revealed the following:

We are trying to read 20k from DFS @ HFile layer. The 
BounderRangeFileInputStream instead reads several header bytes 1 byte at a 
time, and then reads a 11k chunk and later a 9k chunk.

{code}
2010-09-16 09:21:27,912 INFO org.apache.hadoop.hbase.io.hfile.HFile: ### fs 
read @ offset = 34386760 compressedSize = 20711 decompressedSize = 92324
2010-09-16 09:21:27,912 INFO 
org.apache.hadoop.hbase.io.hfile.BoundedRangeFileInputStream: ### seeking to 
reading @ 34386760; bytes: 1
2010-09-16 09:21:27,912 INFO 
org.apache.hadoop.hbase.io.hfile.BoundedRangeFileInputStream: ### seeking to 
reading @ 34386761; bytes: 1
2010-09-16 09:21:27,912 INFO 
org.apache.hadoop.hbase.io.hfile.BoundedRangeFileInputStream: ### seeking to 
reading @ 34386762; bytes: 1
2010-09-16 09:21:27,912 INFO 
org.apache.hadoop.hbase.io.hfile.BoundedRangeFileInputStream: ### seeking to 
reading @ 34386763; bytes: 1
2010-09-16 09:21:27,912 INFO 
org.apache.hadoop.hbase.io.hfile.BoundedRangeFileInputStream: ### seeking to 
reading @ 34386764; bytes: 1
2010-09-16 09:21:27,912 INFO 
org.apache.hadoop.hbase.io.hfile.BoundedRangeFileInputStream: ### seeking to 
reading @ 34386765; bytes: 1
2010-09-16 09:21:27,912 INFO 
org.apache.hadoop.hbase.io.hfile.BoundedRangeFileInputStream: ### seeking to 
reading @ 34386766; bytes: 1
2010-09-16 09:21:27,912 INFO 
org.apache.hadoop.hbase.io.hfile.BoundedRangeFileInputStream: ### seeking to 
reading @ 34386767; bytes: 1
2010-09-16 09:21:27,912 INFO 
org.apache.hadoop.hbase.io.hfile.BoundedRangeFileInputStream: ### seeking to 
reading @ 34386768; bytes: 11005
2010-09-16 09:21:27,912 INFO 
org.apache.hadoop.hbase.io.hfile.BoundedRangeFileInputStream: ### seeking to 
reading @ 34397773; bytes: 1
2010-09-16 09:21:27,912 INFO 
org.apache.hadoop.hbase.io.hfile.BoundedRangeFileInputStream: ### seeking to 
reading @ 34397774; bytes: 1
2010-09-16 09:21:27,912 INFO 
org.apache.hadoop.hbase.io.hfile.BoundedRangeFileInputStream: ### seeking to 
reading @ 34397775; bytes: 1
2010-09-16 09:21:27,912 INFO 
org.apache.hadoop.hbase.io.hfile.BoundedRangeFileInputStream: ### seeking to 
reading @ 34397776; bytes: 1
2010-09-16 09:21:27,912 INFO 
org.apache.hadoop.hbase.io.hfile.BoundedRangeFileInputStream: ### seeking to 
reading @ 34397777; bytes: 1
2010-09-16 09:21:27,912 INFO 
org.apache.hadoop.hbase.io.hfile.BoundedRangeFileInputStream: ### seeking to 
reading @ 34397778; bytes: 1
2010-09-16 09:21:27,912 INFO 
org.apache.hadoop.hbase.io.hfile.BoundedRangeFileInputStream: ### seeking to 
reading @ 34397779; bytes: 1
2010-09-16 09:21:27,913 INFO 
org.apache.hadoop.hbase.io.hfile.BoundedRangeFileInputStream: ### seeking to 
reading @ 34397780; bytes: 1
2010-09-16 09:21:27,913 INFO 
org.apache.hadoop.hbase.io.hfile.BoundedRangeFileInputStream: ### seeking to 
reading @ 34397781; bytes: 9690
{code}

Seems like it should be an easy fix to prefetch the compressed size... rather 
than incremental fetches.


-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.

Reply via email to