hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Vikas Vishwakarma (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HBASE-12866) TestHFilePerformance is broken
Date Tue, 20 Jan 2015 11:35:34 GMT

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

Vikas Vishwakarma commented on HBASE-12866:
-------------------------------------------

Actually I started with this request HBASE-9910 where there is a proposal to merge TestHFilePerformance
and HFilePerformanceEvaluation. TestHFilePerformance has few additional tests including codec
and compression tests, while HFilePerformanceEvaluation handles concurrency and different
read tests. However while attempting to merge these two, I ran into above problems. The compression
and codec tests in TestHFilePerformance looks good to have in general for benchmarks.

The problem seems to be coming from the failure of this check {if (onDiskSizeWithoutHeader
!= uncompressedSizeWithoutHeader +
        totalChecksumBytes()) } in HFileBlock where the BlockSize is not matching the expected
value after deducting the Header size (which I think is being picked up from HConstants).
I could not check further why this difference is coming. 

Exception:
Caused by: java.io.IOException: Using no compression but onDiskSizeWithoutHeader=134, uncompressedSizeWithoutHeader=113,
numChecksumbytes=4
	at org.apache.hadoop.hbase.io.hfile.HFileBlock.assumeUncompressed(HFileBlock.java:565)

HFileBlock Check:
  public void assumeUncompressed() throws IOException {
    if (onDiskSizeWithoutHeader != uncompressedSizeWithoutHeader +
        totalChecksumBytes()) {
      throw new IOException("Using no compression but "
          + "onDiskSizeWithoutHeader=" + onDiskSizeWithoutHeader + ", "
          + "uncompressedSizeWithoutHeader=" + uncompressedSizeWithoutHeader
          + ", numChecksumbytes=" + totalChecksumBytes());
    }
  }

> TestHFilePerformance is broken
> ------------------------------
>
>                 Key: HBASE-12866
>                 URL: https://issues.apache.org/jira/browse/HBASE-12866
>             Project: HBase
>          Issue Type: Bug
>          Components: test
>    Affects Versions: 0.98.8
>         Environment: Command
> bin/hbase org.apache.hadoop.hbase.io.hfile.TestHFilePerformance
> Failure observed for test with the following options: Read HFile with codecName: none
cipherName: aes
>            Reporter: Vikas Vishwakarma
>            Assignee: Andrew Purtell
>            Priority: Minor
>             Fix For: 1.0.0, 2.0.0, 1.1.0, 0.98.11
>
>
> Command
> bin/hbase org.apache.hadoop.hbase.io.hfile.TestHFilePerformance
> File Type: HFile
> Writing HFile with codecName: none cipherName: aes
> 2015-01-15 16:54:51 Started timing.
> /home/vvishwakarma/vikas/projects/hbase-src-0.98.8/hbase-0.98.8/target/test-data/03d50949-0185-4fac-b072-957d2da6ae0e/TestHFilePerformanceHFile.Performance
> HFile write method: 
> 2015-01-15 16:54:52 Stopped timing.
> 2015-01-15 16:54:52  Data written: 
> 2015-01-15 16:54:52    rate  = 66MB/s
> 2015-01-15 16:54:52    total = 52200000B
> 2015-01-15 16:54:52  File written: 
> 2015-01-15 16:54:52    rate  = 66MB/s
> 2015-01-15 16:54:52    total = 52303530B
> +++++++
> Reading file of type: HFile
> Input file size: 52303530
> 2015-01-15 16:54:52 Started timing.
> 2015-01-15 16:54:52,680 ERROR [main] util.AbstractHBaseTool: Error running command-line
tool
> org.apache.hadoop.hbase.io.hfile.CorruptHFileException: Problem reading HFile Trailer
from file /home/vvishwakarma/vikas/projects/hbase-src-0.98.8/hbase-0.98.8/target/test-data/03d50949-0185-4fac-b072-957d2da6ae0e/TestHFilePerformance/HFile.Performance
> 	at org.apache.hadoop.hbase.io.hfile.HFile.pickReaderVersion(HFile.java:463)
> 	at org.apache.hadoop.hbase.io.hfile.HFile.createReaderFromStream(HFile.java:517)
> 	at org.apache.hadoop.hbase.io.hfile.TestHFilePerformance.timeReading(TestHFilePerformance.java:272)
> 	at org.apache.hadoop.hbase.io.hfile.TestHFilePerformance.testRunComparisons(TestHFilePerformance.java:390)
> 	at org.apache.hadoop.hbase.io.hfile.TestHFilePerformance.doWork(TestHFilePerformance.java:447)
> 	at org.apache.hadoop.hbase.util.AbstractHBaseTool.run(AbstractHBaseTool.java:112)
> 	at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:70)
> 	at org.apache.hadoop.hbase.io.hfile.TestHFilePerformance.main(TestHFilePerformance.java:452)
> Caused by: java.io.IOException: Using no compression but onDiskSizeWithoutHeader=134,
uncompressedSizeWithoutHeader=113, numChecksumbytes=4
> 	at org.apache.hadoop.hbase.io.hfile.HFileBlock.assumeUncompressed(HFileBlock.java:561)
> 	at org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderV2.readBlockDataInternal(HFileBlock.java:1589)
> 	at org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderV2.readBlockData(HFileBlock.java:1408)
> 	at org.apache.hadoop.hbase.io.hfile.HFileBlock$AbstractFSReader$1.nextBlock(HFileBlock.java:1248)
> 	at org.apache.hadoop.hbase.io.hfile.HFileBlock$AbstractFSReader$1.nextBlockWithBlockType(HFileBlock.java:1256)
> 	at org.apache.hadoop.hbase.io.hfile.HFileReaderV2.<init>(HFileReaderV2.java:146)
> 	at org.apache.hadoop.hbase.io.hfile.HFile.pickReaderVersion(HFile.java:451)
> 	... 7 more



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

Mime
View raw message