2 * Licensed to the Apache Software Foundation (ASF) under one or more
3 * contributor license agreements. See the NOTICE file distributed with this
4 * work for additional information regarding copyright ownership. The ASF
5 * licenses this file to you under the Apache License, Version 2.0 (the
6 * "License"); you may not use this file except in compliance with the License.
7 * You may obtain a copy of the License at
9 * http://www.apache.org/licenses/LICENSE-2.0
11 * Unless required by applicable law or agreed to in writing, software
12 * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
13 * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
14 * License for the specific language governing permissions and limitations
17 package org
.apache
.hadoop
.hbase
.regionserver
;
19 import java
.io
.ByteArrayInputStream
;
20 import java
.io
.ByteArrayOutputStream
;
21 import java
.io
.DataOutputStream
;
22 import java
.io
.IOException
;
23 import java
.io
.InputStream
;
24 import java
.text
.DecimalFormat
;
25 import java
.util
.ArrayList
;
26 import java
.util
.Iterator
;
27 import java
.util
.List
;
28 import java
.util
.Locale
;
30 import org
.apache
.hadoop
.conf
.Configuration
;
31 import org
.apache
.hadoop
.fs
.FileSystem
;
32 import org
.apache
.hadoop
.fs
.Path
;
33 import org
.apache
.hadoop
.hbase
.Cell
;
34 import org
.apache
.hadoop
.hbase
.HBaseConfiguration
;
35 import org
.apache
.hadoop
.hbase
.KeyValue
;
36 import org
.apache
.hadoop
.hbase
.KeyValueUtil
;
37 import org
.apache
.hadoop
.hbase
.io
.compress
.Compression
;
38 import org
.apache
.hadoop
.hbase
.io
.compress
.Compression
.Algorithm
;
39 import org
.apache
.hadoop
.hbase
.io
.encoding
.DataBlockEncoder
;
40 import org
.apache
.hadoop
.hbase
.io
.encoding
.DataBlockEncoding
;
41 import org
.apache
.hadoop
.hbase
.io
.encoding
.EncodedDataBlock
;
42 import org
.apache
.hadoop
.hbase
.io
.hfile
.CacheConfig
;
43 import org
.apache
.hadoop
.hbase
.io
.hfile
.HFileBlock
;
44 import org
.apache
.hadoop
.hbase
.io
.hfile
.HFileContext
;
45 import org
.apache
.hadoop
.hbase
.io
.hfile
.HFileContextBuilder
;
46 import org
.apache
.hadoop
.hbase
.io
.hfile
.HFileReaderImpl
;
47 import org
.apache
.hadoop
.hbase
.util
.Bytes
;
48 import org
.apache
.hadoop
.io
.WritableUtils
;
49 import org
.apache
.hadoop
.io
.compress
.CompressionOutputStream
;
50 import org
.apache
.hadoop
.io
.compress
.Compressor
;
51 import org
.apache
.hadoop
.io
.compress
.Decompressor
;
52 import org
.slf4j
.Logger
;
53 import org
.slf4j
.LoggerFactory
;
54 import org
.apache
.hbase
.thirdparty
.org
.apache
.commons
.cli
.CommandLine
;
55 import org
.apache
.hbase
.thirdparty
.org
.apache
.commons
.cli
.CommandLineParser
;
56 import org
.apache
.hbase
.thirdparty
.org
.apache
.commons
.cli
.Option
;
57 import org
.apache
.hbase
.thirdparty
.org
.apache
.commons
.cli
.Options
;
58 import org
.apache
.hbase
.thirdparty
.org
.apache
.commons
.cli
.ParseException
;
59 import org
.apache
.hbase
.thirdparty
.org
.apache
.commons
.cli
.PosixParser
;
62 * Tests various algorithms for key compression on an existing HFile. Useful
63 * for testing, debugging and benchmarking.
65 public class DataBlockEncodingTool
{
66 private static final Logger LOG
= LoggerFactory
.getLogger(
67 DataBlockEncodingTool
.class);
69 private static final boolean includesMemstoreTS
= true;
72 * How many times to run the benchmark. More times means better data in terms
73 * of statistics but slower execution. Has to be strictly larger than
74 * {@link #DEFAULT_BENCHMARK_N_OMIT}.
76 private static final int DEFAULT_BENCHMARK_N_TIMES
= 12;
79 * How many first runs should not be included in the benchmark. Done in order
80 * to exclude setup cost.
82 private static final int DEFAULT_BENCHMARK_N_OMIT
= 2;
84 /** HFile name to be used in benchmark */
85 private static final String OPT_HFILE_NAME
= "f";
87 /** Maximum number of key/value pairs to process in a single benchmark run */
88 private static final String OPT_KV_LIMIT
= "n";
90 /** Whether to run a benchmark to measure read throughput */
91 private static final String OPT_MEASURE_THROUGHPUT
= "b";
93 /** If this is specified, no correctness testing will be done */
94 private static final String OPT_OMIT_CORRECTNESS_TEST
= "c";
96 /** What compression algorithm to test */
97 private static final String OPT_COMPRESSION_ALGORITHM
= "a";
99 /** Number of times to run each benchmark */
100 private static final String OPT_BENCHMARK_N_TIMES
= "t";
102 /** Number of first runs of every benchmark to omit from statistics */
103 private static final String OPT_BENCHMARK_N_OMIT
= "omit";
105 /** Compression algorithm to use if not specified on the command line */
106 private static final Algorithm DEFAULT_COMPRESSION
=
107 Compression
.Algorithm
.GZ
;
109 private static final DecimalFormat DELIMITED_DECIMAL_FORMAT
=
113 DELIMITED_DECIMAL_FORMAT
.setGroupingSize(3);
116 private static final String PCT_FORMAT
= "%.2f %%";
117 private static final String INT_FORMAT
= "%d";
119 private static int benchmarkNTimes
= DEFAULT_BENCHMARK_N_TIMES
;
120 private static int benchmarkNOmit
= DEFAULT_BENCHMARK_N_OMIT
;
122 private List
<EncodedDataBlock
> codecs
= new ArrayList
<>();
123 private long totalPrefixLength
= 0;
124 private long totalKeyLength
= 0;
125 private long totalValueLength
= 0;
126 private long totalKeyRedundancyLength
= 0;
127 private long totalCFLength
= 0;
129 private byte[] rawKVs
;
130 private boolean useHBaseChecksum
= false;
132 private final String compressionAlgorithmName
;
133 private final Algorithm compressionAlgorithm
;
134 private final Compressor compressor
;
135 private final Decompressor decompressor
;
137 // Check if HFile use Tag.
138 private static boolean USE_TAG
= false;
140 private enum Manipulation
{
147 public String
toString() {
148 String s
= super.toString();
149 StringBuilder sb
= new StringBuilder();
150 sb
.append(s
.charAt(0));
151 sb
.append(s
.substring(1).toLowerCase(Locale
.ROOT
));
152 return sb
.toString();
157 * @param compressionAlgorithmName What kind of algorithm should be used
158 * as baseline for comparison (e.g. lzo, gz).
160 public DataBlockEncodingTool(String compressionAlgorithmName
) {
161 this.compressionAlgorithmName
= compressionAlgorithmName
;
162 this.compressionAlgorithm
= Compression
.getCompressionAlgorithmByName(
163 compressionAlgorithmName
);
164 this.compressor
= this.compressionAlgorithm
.getCompressor();
165 this.decompressor
= this.compressionAlgorithm
.getDecompressor();
169 * Check statistics for given HFile for different data block encoders.
170 * @param scanner Of file which will be compressed.
171 * @param kvLimit Maximal count of KeyValue which will be processed.
172 * @throws IOException thrown if scanner is invalid
174 public void checkStatistics(final KeyValueScanner scanner
, final int kvLimit
)
176 scanner
.seek(KeyValue
.LOWESTKEY
);
180 byte[] previousKey
= null;
183 DataBlockEncoding
[] encodings
= DataBlockEncoding
.values();
185 ByteArrayOutputStream uncompressedOutputStream
=
186 new ByteArrayOutputStream();
189 while ((currentKV
= KeyValueUtil
.ensureKeyValue(scanner
.next())) != null && j
< kvLimit
) {
190 // Iterates through key/value pairs
192 currentKey
= currentKV
.getKey();
193 if (previousKey
!= null) {
194 for (int i
= 0; i
< previousKey
.length
&& i
< currentKey
.length
&&
195 previousKey
[i
] == currentKey
[i
]; ++i
) {
196 totalKeyRedundancyLength
++;
200 // Add tagsLen zero to cells don't include tags. Since the process of
201 // scanner converts byte array to KV would abandon tagsLen part if tagsLen
202 // is zero. But we still needs the tagsLen part to check if current cell
203 // include tags. If USE_TAG is true, HFile contains cells with tags,
204 // if the cell tagsLen equals 0, it means other cells may have tags.
205 if (USE_TAG
&& currentKV
.getTagsLength() == 0) {
206 uncompressedOutputStream
.write(currentKV
.getBuffer(),
207 currentKV
.getOffset(), currentKV
.getLength());
208 // write tagsLen = 0.
209 uncompressedOutputStream
.write(Bytes
.toBytes((short) 0));
211 uncompressedOutputStream
.write(currentKV
.getBuffer(),
212 currentKV
.getOffset(), currentKV
.getLength());
215 if(includesMemstoreTS
) {
216 WritableUtils
.writeVLong(
217 new DataOutputStream(uncompressedOutputStream
), currentKV
.getSequenceId());
220 previousKey
= currentKey
;
222 int kLen
= currentKV
.getKeyLength();
223 int vLen
= currentKV
.getValueLength();
224 int cfLen
= currentKV
.getFamilyLength(currentKV
.getFamilyOffset());
225 int restLen
= currentKV
.getLength() - kLen
- vLen
;
227 totalKeyLength
+= kLen
;
228 totalValueLength
+= vLen
;
229 totalPrefixLength
+= restLen
;
230 totalCFLength
+= cfLen
;
233 rawKVs
= uncompressedOutputStream
.toByteArray();
234 for (DataBlockEncoding encoding
: encodings
) {
235 if (encoding
== DataBlockEncoding
.NONE
) {
238 DataBlockEncoder d
= encoding
.getEncoder();
239 HFileContext meta
= new HFileContextBuilder()
240 .withDataBlockEncoding(encoding
)
241 .withCompression(Compression
.Algorithm
.NONE
)
242 .withIncludesMvcc(includesMemstoreTS
)
243 .withIncludesTags(USE_TAG
).build();
244 codecs
.add(new EncodedDataBlock(d
, encoding
, rawKVs
, meta
));
249 * Verify if all data block encoders are working properly.
251 * @param scanner Of file which was compressed.
252 * @param kvLimit Maximal count of KeyValue which will be processed.
253 * @return true if all data block encoders compressed/decompressed correctly.
254 * @throws IOException thrown if scanner is invalid
256 public boolean verifyCodecs(final KeyValueScanner scanner
, final int kvLimit
)
260 scanner
.seek(KeyValue
.LOWESTKEY
);
261 List
<Iterator
<Cell
>> codecIterators
= new ArrayList
<>();
262 for(EncodedDataBlock codec
: codecs
) {
263 codecIterators
.add(codec
.getIterator(HFileBlock
.headerSize(useHBaseChecksum
)));
267 while ((currentKv
= KeyValueUtil
.ensureKeyValue(scanner
.next())) != null && j
< kvLimit
) {
268 // Iterates through key/value pairs
270 for (Iterator
<Cell
> it
: codecIterators
) {
272 KeyValue codecKv
= KeyValueUtil
.ensureKeyValue(c
);
273 if (codecKv
== null || 0 != Bytes
.compareTo(
274 codecKv
.getBuffer(), codecKv
.getOffset(), codecKv
.getLength(),
275 currentKv
.getBuffer(), currentKv
.getOffset(),
276 currentKv
.getLength())) {
277 if (codecKv
== null) {
278 LOG
.error("There is a bug in codec " + it
+
279 " it returned null KeyValue,");
282 int limitLength
= 2 * Bytes
.SIZEOF_INT
+
283 Math
.min(codecKv
.getLength(), currentKv
.getLength());
284 while (prefix
< limitLength
&&
285 codecKv
.getBuffer()[prefix
+ codecKv
.getOffset()] ==
286 currentKv
.getBuffer()[prefix
+ currentKv
.getOffset()]) {
290 LOG
.error("There is bug in codec " + it
.toString() +
291 "\n on element " + j
+
292 "\n codecKv.getKeyLength() " + codecKv
.getKeyLength() +
293 "\n codecKv.getValueLength() " + codecKv
.getValueLength() +
294 "\n codecKv.getLength() " + codecKv
.getLength() +
295 "\n currentKv.getKeyLength() " + currentKv
.getKeyLength() +
296 "\n currentKv.getValueLength() " + currentKv
.getValueLength() +
297 "\n codecKv.getLength() " + currentKv
.getLength() +
298 "\n currentKV rowLength " + currentKv
.getRowLength() +
299 " familyName " + currentKv
.getFamilyLength() +
300 " qualifier " + currentKv
.getQualifierLength() +
301 "\n prefix " + prefix
+
302 "\n codecKv '" + Bytes
.toStringBinary(codecKv
.getBuffer(),
303 codecKv
.getOffset(), prefix
) + "' diff '" +
304 Bytes
.toStringBinary(codecKv
.getBuffer(),
305 codecKv
.getOffset() + prefix
, codecKv
.getLength() -
307 "\n currentKv '" + Bytes
.toStringBinary(
308 currentKv
.getBuffer(),
309 currentKv
.getOffset(), prefix
) + "' diff '" +
310 Bytes
.toStringBinary(currentKv
.getBuffer(),
311 currentKv
.getOffset() + prefix
, currentKv
.getLength() -
320 LOG
.info("Verification was successful!");
326 * Benchmark codec's speed.
328 public void benchmarkCodecs() throws IOException
{
329 LOG
.info("Starting a throughput benchmark for data block encoding codecs");
330 int prevTotalSize
= -1;
331 for (EncodedDataBlock codec
: codecs
) {
332 prevTotalSize
= benchmarkEncoder(prevTotalSize
, codec
);
335 benchmarkDefaultCompression(prevTotalSize
, rawKVs
);
339 * Benchmark compression/decompression throughput.
340 * @param previousTotalSize Total size used for verification. Use -1 if
342 * @param codec Tested encoder.
343 * @return Size of uncompressed data.
345 private int benchmarkEncoder(int previousTotalSize
, EncodedDataBlock codec
) {
346 int prevTotalSize
= previousTotalSize
;
349 // decompression time
350 List
<Long
> durations
= new ArrayList
<>();
351 for (int itTime
= 0; itTime
< benchmarkNTimes
; ++itTime
) {
356 it
= codec
.getIterator(HFileBlock
.headerSize(useHBaseChecksum
));
358 // count only the algorithm time, without memory allocations
359 // (expect first time)
360 final long startTime
= System
.nanoTime();
361 while (it
.hasNext()) {
362 totalSize
+= KeyValueUtil
.ensureKeyValue(it
.next()).getLength();
364 final long finishTime
= System
.nanoTime();
365 if (itTime
>= benchmarkNOmit
) {
366 durations
.add(finishTime
- startTime
);
369 if (prevTotalSize
!= -1 && prevTotalSize
!= totalSize
) {
370 throw new IllegalStateException(String
.format(
371 "Algorithm '%s' decoded data to different size", codec
.toString()));
373 prevTotalSize
= totalSize
;
376 List
<Long
> encodingDurations
= new ArrayList
<>();
377 for (int itTime
= 0; itTime
< benchmarkNTimes
; ++itTime
) {
378 final long startTime
= System
.nanoTime();
380 final long finishTime
= System
.nanoTime();
381 if (itTime
>= benchmarkNOmit
) {
382 encodingDurations
.add(finishTime
- startTime
);
386 System
.out
.println(codec
.toString() + ":");
387 printBenchmarkResult(totalSize
, encodingDurations
, Manipulation
.ENCODING
);
388 printBenchmarkResult(totalSize
, durations
, Manipulation
.DECODING
);
389 System
.out
.println();
391 return prevTotalSize
;
394 private void benchmarkDefaultCompression(int totalSize
, byte[] rawBuffer
)
396 benchmarkAlgorithm(compressionAlgorithm
,
397 compressionAlgorithmName
.toUpperCase(Locale
.ROOT
), rawBuffer
, 0, totalSize
);
401 * Check decompress performance of a given algorithm and print it.
402 * @param algorithm Compression algorithm.
403 * @param name Name of algorithm.
404 * @param buffer Buffer to be compressed.
405 * @param offset Position of the beginning of the data.
406 * @param length Length of data in buffer.
407 * @throws IOException
409 public void benchmarkAlgorithm(Compression
.Algorithm algorithm
, String name
,
410 byte[] buffer
, int offset
, int length
) throws IOException
{
411 System
.out
.println(name
+ ":");
414 List
<Long
> compressDurations
= new ArrayList
<>();
415 ByteArrayOutputStream compressedStream
= new ByteArrayOutputStream();
416 CompressionOutputStream compressingStream
=
417 algorithm
.createPlainCompressionStream(compressedStream
, compressor
);
419 for (int itTime
= 0; itTime
< benchmarkNTimes
; ++itTime
) {
420 final long startTime
= System
.nanoTime();
421 // The compressedStream should reset before compressingStream resetState since in GZ
422 // resetStatue will write header in the outputstream.
423 compressedStream
.reset();
424 compressingStream
.resetState();
425 compressingStream
.write(buffer
, offset
, length
);
426 compressingStream
.flush();
427 compressedStream
.toByteArray();
429 final long finishTime
= System
.nanoTime();
432 if (itTime
>= benchmarkNOmit
) {
433 compressDurations
.add(finishTime
- startTime
);
436 } catch (IOException e
) {
437 throw new RuntimeException(String
.format(
438 "Benchmark, or encoding algorithm '%s' cause some stream problems",
441 compressingStream
.close();
442 printBenchmarkResult(length
, compressDurations
, Manipulation
.COMPRESSION
);
444 byte[] compBuffer
= compressedStream
.toByteArray();
446 // uncompress it several times and measure performance
447 List
<Long
> durations
= new ArrayList
<>();
448 for (int itTime
= 0; itTime
< benchmarkNTimes
; ++itTime
) {
449 final long startTime
= System
.nanoTime();
450 byte[] newBuf
= new byte[length
+ 1];
453 ByteArrayInputStream downStream
= new ByteArrayInputStream(compBuffer
,
454 0, compBuffer
.length
);
455 InputStream decompressedStream
= algorithm
.createDecompressionStream(
456 downStream
, decompressor
, 0);
460 while ((nextChunk
= decompressedStream
.available()) > 0) {
461 destOffset
+= decompressedStream
.read(newBuf
, destOffset
, nextChunk
);
463 decompressedStream
.close();
465 } catch (IOException e
) {
466 throw new RuntimeException(String
.format(
467 "Decoding path in '%s' algorithm cause exception ", name
), e
);
470 final long finishTime
= System
.nanoTime();
473 if (0 != Bytes
.compareTo(buffer
, 0, length
, newBuf
, 0, length
)) {
475 for(; prefix
< buffer
.length
&& prefix
< newBuf
.length
; ++prefix
) {
476 if (buffer
[prefix
] != newBuf
[prefix
]) {
480 throw new RuntimeException(String
.format(
481 "Algorithm '%s' is corrupting the data", name
));
485 if (itTime
>= benchmarkNOmit
) {
486 durations
.add(finishTime
- startTime
);
489 printBenchmarkResult(length
, durations
, Manipulation
.DECOMPRESSION
);
490 System
.out
.println();
493 private static final double BYTES_IN_MB
= 1024 * 1024.0;
494 private static final double NS_IN_SEC
= 1000.0 * 1000.0 * 1000.0;
495 private static final double MB_SEC_COEF
= NS_IN_SEC
/ BYTES_IN_MB
;
497 private static void printBenchmarkResult(int totalSize
,
498 List
<Long
> durationsInNanoSec
, Manipulation manipulation
) {
499 final int n
= durationsInNanoSec
.size();
501 for (long time
: durationsInNanoSec
) {
506 double meanMBPerSec
= totalSize
* MB_SEC_COEF
/ meanTime
;
507 double mbPerSecSTD
= 0;
509 for (long time
: durationsInNanoSec
) {
510 double mbPerSec
= totalSize
* MB_SEC_COEF
/ time
;
511 double dev
= mbPerSec
- meanMBPerSec
;
512 mbPerSecSTD
+= dev
* dev
;
514 mbPerSecSTD
= Math
.sqrt(mbPerSecSTD
/ n
);
517 outputTuple(manipulation
+ " performance", "%6.2f MB/s (+/- %.2f MB/s)",
518 meanMBPerSec
, mbPerSecSTD
);
521 private static void outputTuple(String caption
, String format
,
523 if (format
.startsWith(INT_FORMAT
)) {
524 format
= "%s" + format
.substring(INT_FORMAT
.length());
525 values
[0] = DELIMITED_DECIMAL_FORMAT
.format(values
[0]);
528 StringBuilder sb
= new StringBuilder();
533 String v
= String
.format(format
, values
);
534 int padding
= 60 - sb
.length() - v
.length();
535 for (int i
= 0; i
< padding
; ++i
) {
539 System
.out
.println(sb
);
543 * Display statistics of different compression algorithms.
544 * @throws IOException
546 public void displayStatistics() throws IOException
{
547 final String comprAlgo
= compressionAlgorithmName
.toUpperCase(Locale
.ROOT
);
548 long rawBytes
= totalKeyLength
+ totalPrefixLength
+ totalValueLength
;
550 System
.out
.println("Raw data size:");
551 outputTuple("Raw bytes", INT_FORMAT
, rawBytes
);
552 outputTuplePct("Key bytes", totalKeyLength
);
553 outputTuplePct("Value bytes", totalValueLength
);
554 outputTuplePct("KV infrastructure", totalPrefixLength
);
555 outputTuplePct("CF overhead", totalCFLength
);
556 outputTuplePct("Total key redundancy", totalKeyRedundancyLength
);
558 int compressedSize
= EncodedDataBlock
.getCompressedSize(
559 compressionAlgorithm
, compressor
, rawKVs
, 0, rawKVs
.length
);
560 outputTuple(comprAlgo
+ " only size", INT_FORMAT
,
562 outputSavings(comprAlgo
+ " only", compressedSize
, rawBytes
);
563 System
.out
.println();
565 for (EncodedDataBlock codec
: codecs
) {
566 System
.out
.println(codec
.toString());
567 long encodedBytes
= codec
.getSize();
568 outputTuple("Encoded bytes", INT_FORMAT
, encodedBytes
);
569 outputSavings("Key encoding", encodedBytes
- totalValueLength
,
570 rawBytes
- totalValueLength
);
571 outputSavings("Total encoding", encodedBytes
, rawBytes
);
573 int encodedCompressedSize
= codec
.getEncodedCompressedSize(
574 compressionAlgorithm
, compressor
);
575 outputTuple("Encoding + " + comprAlgo
+ " size", INT_FORMAT
,
576 encodedCompressedSize
);
577 outputSavings("Encoding + " + comprAlgo
, encodedCompressedSize
, rawBytes
);
578 outputSavings("Encoding with " + comprAlgo
, encodedCompressedSize
,
581 System
.out
.println();
585 private void outputTuplePct(String caption
, long size
) {
586 outputTuple(caption
, INT_FORMAT
+ " (" + PCT_FORMAT
+ ")",
587 size
, size
* 100.0 / rawKVs
.length
);
590 private void outputSavings(String caption
, long part
, long whole
) {
591 double pct
= 100.0 * (1 - 1.0 * part
/ whole
);
592 double times
= whole
* 1.0 / part
;
593 outputTuple(caption
+ " savings", PCT_FORMAT
+ " (%.2f x)",
598 * Test a data block encoder on the given HFile. Output results to console.
599 * @param kvLimit The limit of KeyValue which will be analyzed.
600 * @param hfilePath an HFile path on the file system.
601 * @param compressionName Compression algorithm used for comparison.
602 * @param doBenchmark Run performance benchmarks.
603 * @param doVerify Verify correctness.
604 * @throws IOException When pathName is incorrect.
606 public static void testCodecs(Configuration conf
, int kvLimit
,
607 String hfilePath
, String compressionName
, boolean doBenchmark
,
608 boolean doVerify
) throws IOException
{
609 // create environment
610 Path path
= new Path(hfilePath
);
611 CacheConfig cacheConf
= new CacheConfig(conf
);
612 FileSystem fs
= FileSystem
.get(conf
);
613 HStoreFile hsf
= new HStoreFile(fs
, path
, conf
, cacheConf
, BloomType
.NONE
, true);
615 StoreFileReader reader
= hsf
.getReader();
616 reader
.loadFileInfo();
617 KeyValueScanner scanner
= reader
.getStoreFileScanner(true, true,
618 false, hsf
.getMaxMemStoreTS(), 0, false);
619 USE_TAG
= reader
.getHFileReader().getFileContext().isIncludesTags();
621 DataBlockEncodingTool comp
= new DataBlockEncodingTool(compressionName
);
622 int majorVersion
= reader
.getHFileVersion();
623 comp
.useHBaseChecksum
= majorVersion
> 2 ||
624 (majorVersion
== 2 &&
625 reader
.getHFileMinorVersion() >= HFileReaderImpl
.MINOR_VERSION_WITH_CHECKSUM
);
626 comp
.checkStatistics(scanner
, kvLimit
);
628 comp
.verifyCodecs(scanner
, kvLimit
);
631 comp
.benchmarkCodecs();
633 comp
.displayStatistics();
637 reader
.close(cacheConf
.shouldEvictOnClose());
640 private static void printUsage(Options options
) {
641 System
.err
.println("Usage:");
642 System
.err
.println(String
.format("./hbase %s <options>",
643 DataBlockEncodingTool
.class.getName()));
644 System
.err
.println("Options:");
645 for (Object it
: options
.getOptions()) {
646 Option opt
= (Option
) it
;
648 System
.err
.println(String
.format("-%s %s: %s", opt
.getOpt(),
649 opt
.getArgName(), opt
.getDescription()));
651 System
.err
.println(String
.format("-%s: %s", opt
.getOpt(),
652 opt
.getDescription()));
658 * A command line interface to benchmarks. Parses command-line arguments and
659 * runs the appropriate benchmarks.
660 * @param args Should have length at least 1 and holds the file path to HFile.
661 * @throws IOException If you specified the wrong file.
663 public static void main(final String
[] args
) throws IOException
{
664 // set up user arguments
665 Options options
= new Options();
666 options
.addOption(OPT_HFILE_NAME
, true, "HFile to analyse (REQUIRED)");
667 options
.getOption(OPT_HFILE_NAME
).setArgName("FILENAME");
668 options
.addOption(OPT_KV_LIMIT
, true,
669 "Maximum number of KeyValues to process. A benchmark stops running " +
670 "after iterating over this many KV pairs.");
671 options
.getOption(OPT_KV_LIMIT
).setArgName("NUMBER");
672 options
.addOption(OPT_MEASURE_THROUGHPUT
, false,
673 "Measure read throughput");
674 options
.addOption(OPT_OMIT_CORRECTNESS_TEST
, false,
675 "Omit corectness tests.");
676 options
.addOption(OPT_COMPRESSION_ALGORITHM
, true,
677 "What kind of compression algorithm use for comparison.");
678 options
.addOption(OPT_BENCHMARK_N_TIMES
,
679 true, "Number of times to run each benchmark. Default value: " +
680 DEFAULT_BENCHMARK_N_TIMES
);
681 options
.addOption(OPT_BENCHMARK_N_OMIT
, true,
682 "Number of first runs of every benchmark to exclude from "
683 + "statistics (" + DEFAULT_BENCHMARK_N_OMIT
684 + " by default, so that " + "only the last "
685 + (DEFAULT_BENCHMARK_N_TIMES
- DEFAULT_BENCHMARK_N_OMIT
)
686 + " times are included in statistics.)");
689 CommandLineParser parser
= new PosixParser();
690 CommandLine cmd
= null;
692 cmd
= parser
.parse(options
, args
);
693 } catch (ParseException e
) {
694 System
.err
.println("Could not parse arguments!");
696 return; // avoid warning
699 int kvLimit
= Integer
.MAX_VALUE
;
700 if (cmd
.hasOption(OPT_KV_LIMIT
)) {
701 kvLimit
= Integer
.parseInt(cmd
.getOptionValue(OPT_KV_LIMIT
));
703 LOG
.error("KV_LIMIT should not less than 1.");
707 // basic argument sanity checks
708 if (!cmd
.hasOption(OPT_HFILE_NAME
)) {
709 LOG
.error("Please specify HFile name using the " + OPT_HFILE_NAME
715 String pathName
= cmd
.getOptionValue(OPT_HFILE_NAME
);
716 String compressionName
= DEFAULT_COMPRESSION
.getName();
717 if (cmd
.hasOption(OPT_COMPRESSION_ALGORITHM
)) {
719 cmd
.getOptionValue(OPT_COMPRESSION_ALGORITHM
).toLowerCase(Locale
.ROOT
);
721 boolean doBenchmark
= cmd
.hasOption(OPT_MEASURE_THROUGHPUT
);
722 boolean doVerify
= !cmd
.hasOption(OPT_OMIT_CORRECTNESS_TEST
);
724 if (cmd
.hasOption(OPT_BENCHMARK_N_TIMES
)) {
725 benchmarkNTimes
= Integer
.valueOf(cmd
.getOptionValue(
726 OPT_BENCHMARK_N_TIMES
));
728 if (cmd
.hasOption(OPT_BENCHMARK_N_OMIT
)) {
730 Integer
.valueOf(cmd
.getOptionValue(OPT_BENCHMARK_N_OMIT
));
732 if (benchmarkNTimes
< benchmarkNOmit
) {
733 LOG
.error("The number of times to run each benchmark ("
735 + ") must be greater than the number of benchmark runs to exclude "
736 + "from statistics (" + benchmarkNOmit
+ ")");
739 LOG
.info("Running benchmark " + benchmarkNTimes
+ " times. " +
740 "Excluding the first " + benchmarkNOmit
+ " times from statistics.");
742 final Configuration conf
= HBaseConfiguration
.create();
743 testCodecs(conf
, kvLimit
, pathName
, compressionName
, doBenchmark
, doVerify
);