001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.regionserver; 019 020import java.io.ByteArrayInputStream; 021import java.io.ByteArrayOutputStream; 022import java.io.DataOutputStream; 023import java.io.IOException; 024import java.io.InputStream; 025import java.text.DecimalFormat; 026import java.util.ArrayList; 027import java.util.Iterator; 028import java.util.List; 029import java.util.Locale; 030import org.apache.hadoop.conf.Configuration; 031import org.apache.hadoop.fs.FileSystem; 032import org.apache.hadoop.fs.Path; 033import org.apache.hadoop.hbase.Cell; 034import org.apache.hadoop.hbase.HBaseConfiguration; 035import org.apache.hadoop.hbase.KeyValue; 036import org.apache.hadoop.hbase.KeyValueUtil; 037import org.apache.hadoop.hbase.io.compress.Compression; 038import org.apache.hadoop.hbase.io.compress.Compression.Algorithm; 039import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder; 040import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; 041import org.apache.hadoop.hbase.io.encoding.EncodedDataBlock; 042import org.apache.hadoop.hbase.io.hfile.CacheConfig; 043import org.apache.hadoop.hbase.io.hfile.HFileBlock; 044import org.apache.hadoop.hbase.io.hfile.HFileContext; 045import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; 046import org.apache.hadoop.hbase.io.hfile.HFileReaderImpl; 047import org.apache.hadoop.hbase.util.Bytes; 048import org.apache.hadoop.io.WritableUtils; 049import org.apache.hadoop.io.compress.CompressionOutputStream; 050import org.apache.hadoop.io.compress.Compressor; 051import org.apache.hadoop.io.compress.Decompressor; 052import org.slf4j.Logger; 053import org.slf4j.LoggerFactory; 054 055import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; 056import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLineParser; 057import org.apache.hbase.thirdparty.org.apache.commons.cli.Option; 058import org.apache.hbase.thirdparty.org.apache.commons.cli.Options; 059import org.apache.hbase.thirdparty.org.apache.commons.cli.ParseException; 060import org.apache.hbase.thirdparty.org.apache.commons.cli.PosixParser; 061 062/** 063 * Tests various algorithms for key compression on an existing HFile. Useful for testing, debugging 064 * and benchmarking. 065 */ 066public class DataBlockEncodingTool { 067 private static final Logger LOG = LoggerFactory.getLogger(DataBlockEncodingTool.class); 068 069 private static final boolean includesMemstoreTS = true; 070 071 /** 072 * How many times to run the benchmark. More times means better data in terms of statistics but 073 * slower execution. Has to be strictly larger than {@link #DEFAULT_BENCHMARK_N_OMIT}. 074 */ 075 private static final int DEFAULT_BENCHMARK_N_TIMES = 12; 076 077 /** 078 * How many first runs should not be included in the benchmark. Done in order to exclude setup 079 * cost. 080 */ 081 private static final int DEFAULT_BENCHMARK_N_OMIT = 2; 082 083 /** HFile name to be used in benchmark */ 084 private static final String OPT_HFILE_NAME = "f"; 085 086 /** Maximum number of key/value pairs to process in a single benchmark run */ 087 private static final String OPT_KV_LIMIT = "n"; 088 089 /** Whether to run a benchmark to measure read throughput */ 090 private static final String OPT_MEASURE_THROUGHPUT = "b"; 091 092 /** If this is specified, no correctness testing will be done */ 093 private static final String OPT_OMIT_CORRECTNESS_TEST = "c"; 094 095 /** What compression algorithm to test */ 096 private static final String OPT_COMPRESSION_ALGORITHM = "a"; 097 098 /** Number of times to run each benchmark */ 099 private static final String OPT_BENCHMARK_N_TIMES = "t"; 100 101 /** Number of first runs of every benchmark to omit from statistics */ 102 private static final String OPT_BENCHMARK_N_OMIT = "omit"; 103 104 /** Compression algorithm to use if not specified on the command line */ 105 private static final Algorithm DEFAULT_COMPRESSION = Compression.Algorithm.GZ; 106 107 private static final DecimalFormat DELIMITED_DECIMAL_FORMAT = new DecimalFormat(); 108 109 static { 110 DELIMITED_DECIMAL_FORMAT.setGroupingSize(3); 111 } 112 113 private static final String PCT_FORMAT = "%.2f %%"; 114 private static final String INT_FORMAT = "%d"; 115 116 private static int benchmarkNTimes = DEFAULT_BENCHMARK_N_TIMES; 117 private static int benchmarkNOmit = DEFAULT_BENCHMARK_N_OMIT; 118 119 private final Configuration conf; 120 private List<EncodedDataBlock> codecs = new ArrayList<>(); 121 private long totalPrefixLength = 0; 122 private long totalKeyLength = 0; 123 private long totalValueLength = 0; 124 private long totalKeyRedundancyLength = 0; 125 private long totalCFLength = 0; 126 127 private byte[] rawKVs; 128 private boolean useHBaseChecksum = false; 129 130 private final String compressionAlgorithmName; 131 private final Algorithm compressionAlgorithm; 132 private final Compressor compressor; 133 private final Decompressor decompressor; 134 135 // Check if HFile use Tag. 136 private static boolean USE_TAG = false; 137 138 private enum Manipulation { 139 ENCODING, 140 DECODING, 141 COMPRESSION, 142 DECOMPRESSION; 143 144 @Override 145 public String toString() { 146 String s = super.toString(); 147 StringBuilder sb = new StringBuilder(); 148 sb.append(s.charAt(0)); 149 sb.append(s.substring(1).toLowerCase(Locale.ROOT)); 150 return sb.toString(); 151 } 152 } 153 154 /** 155 * @param compressionAlgorithmName What kind of algorithm should be used as baseline for 156 * comparison (e.g. lzo, gz). 157 */ 158 public DataBlockEncodingTool(Configuration conf, String compressionAlgorithmName) { 159 this.conf = conf; 160 this.compressionAlgorithmName = compressionAlgorithmName; 161 this.compressionAlgorithm = Compression.getCompressionAlgorithmByName(compressionAlgorithmName); 162 this.compressor = this.compressionAlgorithm.getCompressor(); 163 this.decompressor = this.compressionAlgorithm.getDecompressor(); 164 } 165 166 /** 167 * Check statistics for given HFile for different data block encoders. 168 * @param scanner Of file which will be compressed. 169 * @param kvLimit Maximal count of KeyValue which will be processed. 170 * @throws IOException thrown if scanner is invalid 171 */ 172 public void checkStatistics(final KeyValueScanner scanner, final int kvLimit) throws IOException { 173 scanner.seek(KeyValue.LOWESTKEY); 174 175 KeyValue currentKV; 176 177 byte[] previousKey = null; 178 byte[] currentKey; 179 180 DataBlockEncoding[] encodings = DataBlockEncoding.values(); 181 182 ByteArrayOutputStream uncompressedOutputStream = new ByteArrayOutputStream(); 183 184 int j = 0; 185 while ((currentKV = KeyValueUtil.ensureKeyValue(scanner.next())) != null && j < kvLimit) { 186 // Iterates through key/value pairs 187 j++; 188 currentKey = currentKV.getKey(); 189 if (previousKey != null) { 190 for (int i = 0; i < previousKey.length && i < currentKey.length 191 && previousKey[i] == currentKey[i]; ++i) { 192 totalKeyRedundancyLength++; 193 } 194 } 195 196 // Add tagsLen zero to cells don't include tags. Since the process of 197 // scanner converts byte array to KV would abandon tagsLen part if tagsLen 198 // is zero. But we still needs the tagsLen part to check if current cell 199 // include tags. If USE_TAG is true, HFile contains cells with tags, 200 // if the cell tagsLen equals 0, it means other cells may have tags. 201 if (USE_TAG && currentKV.getTagsLength() == 0) { 202 uncompressedOutputStream.write(currentKV.getBuffer(), currentKV.getOffset(), 203 currentKV.getLength()); 204 // write tagsLen = 0. 205 uncompressedOutputStream.write(Bytes.toBytes((short) 0)); 206 } else { 207 uncompressedOutputStream.write(currentKV.getBuffer(), currentKV.getOffset(), 208 currentKV.getLength()); 209 } 210 211 if (includesMemstoreTS) { 212 WritableUtils.writeVLong(new DataOutputStream(uncompressedOutputStream), 213 currentKV.getSequenceId()); 214 } 215 216 previousKey = currentKey; 217 218 int kLen = currentKV.getKeyLength(); 219 int vLen = currentKV.getValueLength(); 220 int cfOffset = currentKV.getFamilyOffset(); 221 int cfLen = currentKV.getFamilyLength(); 222 int restLen = currentKV.getLength() - kLen - vLen; 223 224 totalKeyLength += kLen; 225 totalValueLength += vLen; 226 totalPrefixLength += restLen; 227 totalCFLength += cfLen; 228 } 229 230 rawKVs = uncompressedOutputStream.toByteArray(); 231 for (DataBlockEncoding encoding : encodings) { 232 if (encoding == DataBlockEncoding.NONE) { 233 continue; 234 } 235 DataBlockEncoder d = encoding.getEncoder(); 236 HFileContext meta = new HFileContextBuilder().withDataBlockEncoding(encoding) 237 .withCompression(Compression.Algorithm.NONE).withIncludesMvcc(includesMemstoreTS) 238 .withIncludesTags(USE_TAG).build(); 239 codecs.add(new EncodedDataBlock(conf, d, encoding, rawKVs, meta)); 240 } 241 } 242 243 /** 244 * Verify if all data block encoders are working properly. 245 * @param scanner Of file which was compressed. 246 * @param kvLimit Maximal count of KeyValue which will be processed. 247 * @return true if all data block encoders compressed/decompressed correctly. 248 * @throws IOException thrown if scanner is invalid 249 */ 250 public boolean verifyCodecs(final KeyValueScanner scanner, final int kvLimit) throws IOException { 251 KeyValue currentKv; 252 253 scanner.seek(KeyValue.LOWESTKEY); 254 List<Iterator<Cell>> codecIterators = new ArrayList<>(); 255 for (EncodedDataBlock codec : codecs) { 256 codecIterators.add(codec.getIterator(HFileBlock.headerSize(useHBaseChecksum))); 257 } 258 259 int j = 0; 260 while ((currentKv = KeyValueUtil.ensureKeyValue(scanner.next())) != null && j < kvLimit) { 261 // Iterates through key/value pairs 262 ++j; 263 for (Iterator<Cell> it : codecIterators) { 264 Cell c = it.next(); 265 KeyValue codecKv = KeyValueUtil.ensureKeyValue(c); 266 if ( 267 codecKv == null 268 || 0 != Bytes.compareTo(codecKv.getBuffer(), codecKv.getOffset(), codecKv.getLength(), 269 currentKv.getBuffer(), currentKv.getOffset(), currentKv.getLength()) 270 ) { 271 if (codecKv == null) { 272 LOG.error("There is a bug in codec " + it + " it returned null KeyValue,"); 273 } else { 274 int prefix = 0; 275 int limitLength = 276 2 * Bytes.SIZEOF_INT + Math.min(codecKv.getLength(), currentKv.getLength()); 277 while ( 278 prefix < limitLength && codecKv.getBuffer()[prefix + codecKv.getOffset()] 279 == currentKv.getBuffer()[prefix + currentKv.getOffset()] 280 ) { 281 prefix++; 282 } 283 284 LOG.error("There is bug in codec " + it.toString() + "\n on element " + j 285 + "\n codecKv.getKeyLength() " + codecKv.getKeyLength() 286 + "\n codecKv.getValueLength() " + codecKv.getValueLength() 287 + "\n codecKv.getLength() " + codecKv.getLength() + "\n currentKv.getKeyLength() " 288 + currentKv.getKeyLength() + "\n currentKv.getValueLength() " 289 + currentKv.getValueLength() + "\n codecKv.getLength() " + currentKv.getLength() 290 + "\n currentKV rowLength " + currentKv.getRowLength() + " familyName " 291 + currentKv.getFamilyLength() + " qualifier " + currentKv.getQualifierLength() 292 + "\n prefix " + prefix + "\n codecKv '" 293 + Bytes.toStringBinary(codecKv.getBuffer(), codecKv.getOffset(), prefix) + "' diff '" 294 + Bytes.toStringBinary(codecKv.getBuffer(), codecKv.getOffset() + prefix, 295 codecKv.getLength() - prefix) 296 + "'" + "\n currentKv '" 297 + Bytes.toStringBinary(currentKv.getBuffer(), currentKv.getOffset(), prefix) 298 + "' diff '" + Bytes.toStringBinary(currentKv.getBuffer(), 299 currentKv.getOffset() + prefix, currentKv.getLength() - prefix) 300 + "'"); 301 } 302 return false; 303 } 304 } 305 } 306 307 LOG.info("Verification was successful!"); 308 309 return true; 310 } 311 312 /** 313 * Benchmark codec's speed. 314 */ 315 public void benchmarkCodecs() throws IOException { 316 LOG.info("Starting a throughput benchmark for data block encoding codecs"); 317 int prevTotalSize = -1; 318 for (EncodedDataBlock codec : codecs) { 319 prevTotalSize = benchmarkEncoder(prevTotalSize, codec); 320 } 321 322 benchmarkDefaultCompression(prevTotalSize, rawKVs); 323 } 324 325 /** 326 * Benchmark compression/decompression throughput. 327 * @param previousTotalSize Total size used for verification. Use -1 if unknown. 328 * @param codec Tested encoder. 329 * @return Size of uncompressed data. 330 */ 331 private int benchmarkEncoder(int previousTotalSize, EncodedDataBlock codec) { 332 int prevTotalSize = previousTotalSize; 333 int totalSize = 0; 334 335 // decompression time 336 List<Long> durations = new ArrayList<>(); 337 for (int itTime = 0; itTime < benchmarkNTimes; ++itTime) { 338 totalSize = 0; 339 340 Iterator<Cell> it; 341 342 it = codec.getIterator(HFileBlock.headerSize(useHBaseChecksum)); 343 344 // count only the algorithm time, without memory allocations 345 // (expect first time) 346 final long startTime = System.nanoTime(); 347 while (it.hasNext()) { 348 totalSize += KeyValueUtil.ensureKeyValue(it.next()).getLength(); 349 } 350 final long finishTime = System.nanoTime(); 351 if (itTime >= benchmarkNOmit) { 352 durations.add(finishTime - startTime); 353 } 354 355 if (prevTotalSize != -1 && prevTotalSize != totalSize) { 356 throw new IllegalStateException( 357 String.format("Algorithm '%s' decoded data to different size", codec.toString())); 358 } 359 prevTotalSize = totalSize; 360 } 361 362 List<Long> encodingDurations = new ArrayList<>(); 363 for (int itTime = 0; itTime < benchmarkNTimes; ++itTime) { 364 final long startTime = System.nanoTime(); 365 codec.encodeData(); 366 final long finishTime = System.nanoTime(); 367 if (itTime >= benchmarkNOmit) { 368 encodingDurations.add(finishTime - startTime); 369 } 370 } 371 372 System.out.println(codec.toString() + ":"); 373 printBenchmarkResult(totalSize, encodingDurations, Manipulation.ENCODING); 374 printBenchmarkResult(totalSize, durations, Manipulation.DECODING); 375 System.out.println(); 376 377 return prevTotalSize; 378 } 379 380 private void benchmarkDefaultCompression(int totalSize, byte[] rawBuffer) throws IOException { 381 benchmarkAlgorithm(compressionAlgorithm, compressionAlgorithmName.toUpperCase(Locale.ROOT), 382 rawBuffer, 0, totalSize); 383 } 384 385 /** 386 * Check decompress performance of a given algorithm and print it. 387 * @param algorithm Compression algorithm. 388 * @param name Name of algorithm. 389 * @param buffer Buffer to be compressed. 390 * @param offset Position of the beginning of the data. 391 * @param length Length of data in buffer. n 392 */ 393 public void benchmarkAlgorithm(Compression.Algorithm algorithm, String name, byte[] buffer, 394 int offset, int length) throws IOException { 395 System.out.println(name + ":"); 396 397 // compress it 398 List<Long> compressDurations = new ArrayList<>(); 399 ByteArrayOutputStream compressedStream = new ByteArrayOutputStream(); 400 CompressionOutputStream compressingStream = 401 algorithm.createPlainCompressionStream(compressedStream, compressor); 402 try { 403 for (int itTime = 0; itTime < benchmarkNTimes; ++itTime) { 404 final long startTime = System.nanoTime(); 405 // The compressedStream should reset before compressingStream resetState since in GZ 406 // resetStatue will write header in the outputstream. 407 compressedStream.reset(); 408 compressingStream.resetState(); 409 compressingStream.write(buffer, offset, length); 410 compressingStream.flush(); 411 compressedStream.toByteArray(); 412 413 final long finishTime = System.nanoTime(); 414 415 // add time record 416 if (itTime >= benchmarkNOmit) { 417 compressDurations.add(finishTime - startTime); 418 } 419 } 420 } catch (IOException e) { 421 throw new RuntimeException( 422 String.format("Benchmark, or encoding algorithm '%s' cause some stream problems", name), e); 423 } 424 compressingStream.close(); 425 printBenchmarkResult(length, compressDurations, Manipulation.COMPRESSION); 426 427 byte[] compBuffer = compressedStream.toByteArray(); 428 429 // uncompress it several times and measure performance 430 List<Long> durations = new ArrayList<>(); 431 for (int itTime = 0; itTime < benchmarkNTimes; ++itTime) { 432 final long startTime = System.nanoTime(); 433 byte[] newBuf = new byte[length + 1]; 434 435 try { 436 ByteArrayInputStream downStream = 437 new ByteArrayInputStream(compBuffer, 0, compBuffer.length); 438 InputStream decompressedStream = 439 algorithm.createDecompressionStream(downStream, decompressor, 0); 440 441 int destOffset = 0; 442 int nextChunk; 443 while ((nextChunk = decompressedStream.available()) > 0) { 444 destOffset += decompressedStream.read(newBuf, destOffset, nextChunk); 445 } 446 decompressedStream.close(); 447 448 } catch (IOException e) { 449 throw new RuntimeException( 450 String.format("Decoding path in '%s' algorithm cause exception ", name), e); 451 } 452 453 final long finishTime = System.nanoTime(); 454 455 // check correctness 456 if (0 != Bytes.compareTo(buffer, 0, length, newBuf, 0, length)) { 457 int prefix = 0; 458 for (; prefix < buffer.length && prefix < newBuf.length; ++prefix) { 459 if (buffer[prefix] != newBuf[prefix]) { 460 break; 461 } 462 } 463 throw new RuntimeException(String.format("Algorithm '%s' is corrupting the data", name)); 464 } 465 466 // add time record 467 if (itTime >= benchmarkNOmit) { 468 durations.add(finishTime - startTime); 469 } 470 } 471 printBenchmarkResult(length, durations, Manipulation.DECOMPRESSION); 472 System.out.println(); 473 } 474 475 private static final double BYTES_IN_MB = 1024 * 1024.0; 476 private static final double NS_IN_SEC = 1000.0 * 1000.0 * 1000.0; 477 private static final double MB_SEC_COEF = NS_IN_SEC / BYTES_IN_MB; 478 479 private static void printBenchmarkResult(int totalSize, List<Long> durationsInNanoSec, 480 Manipulation manipulation) { 481 final int n = durationsInNanoSec.size(); 482 long meanTime = 0; 483 for (long time : durationsInNanoSec) { 484 meanTime += time; 485 } 486 meanTime /= n; 487 488 double meanMBPerSec = totalSize * MB_SEC_COEF / meanTime; 489 double mbPerSecSTD = 0; 490 if (n > 0) { 491 for (long time : durationsInNanoSec) { 492 double mbPerSec = totalSize * MB_SEC_COEF / time; 493 double dev = mbPerSec - meanMBPerSec; 494 mbPerSecSTD += dev * dev; 495 } 496 mbPerSecSTD = Math.sqrt(mbPerSecSTD / n); 497 } 498 499 outputTuple(manipulation + " performance", "%6.2f MB/s (+/- %.2f MB/s)", meanMBPerSec, 500 mbPerSecSTD); 501 } 502 503 private static void outputTuple(String caption, String format, Object... values) { 504 if (format.startsWith(INT_FORMAT)) { 505 format = "%s" + format.substring(INT_FORMAT.length()); 506 values[0] = DELIMITED_DECIMAL_FORMAT.format(values[0]); 507 } 508 509 StringBuilder sb = new StringBuilder(); 510 sb.append(" "); 511 sb.append(caption); 512 sb.append(":"); 513 514 String v = String.format(format, values); 515 int padding = 60 - sb.length() - v.length(); 516 for (int i = 0; i < padding; ++i) { 517 sb.append(' '); 518 } 519 sb.append(v); 520 System.out.println(sb); 521 } 522 523 /** 524 * Display statistics of different compression algorithms. n 525 */ 526 public void displayStatistics() throws IOException { 527 final String comprAlgo = compressionAlgorithmName.toUpperCase(Locale.ROOT); 528 long rawBytes = totalKeyLength + totalPrefixLength + totalValueLength; 529 530 System.out.println("Raw data size:"); 531 outputTuple("Raw bytes", INT_FORMAT, rawBytes); 532 outputTuplePct("Key bytes", totalKeyLength); 533 outputTuplePct("Value bytes", totalValueLength); 534 outputTuplePct("KV infrastructure", totalPrefixLength); 535 outputTuplePct("CF overhead", totalCFLength); 536 outputTuplePct("Total key redundancy", totalKeyRedundancyLength); 537 538 int compressedSize = EncodedDataBlock.getCompressedSize(compressionAlgorithm, compressor, 539 rawKVs, 0, rawKVs.length); 540 outputTuple(comprAlgo + " only size", INT_FORMAT, compressedSize); 541 outputSavings(comprAlgo + " only", compressedSize, rawBytes); 542 System.out.println(); 543 544 for (EncodedDataBlock codec : codecs) { 545 System.out.println(codec.toString()); 546 long encodedBytes = codec.getSize(); 547 outputTuple("Encoded bytes", INT_FORMAT, encodedBytes); 548 outputSavings("Key encoding", encodedBytes - totalValueLength, rawBytes - totalValueLength); 549 outputSavings("Total encoding", encodedBytes, rawBytes); 550 551 int encodedCompressedSize = codec.getEncodedCompressedSize(compressionAlgorithm, compressor); 552 outputTuple("Encoding + " + comprAlgo + " size", INT_FORMAT, encodedCompressedSize); 553 outputSavings("Encoding + " + comprAlgo, encodedCompressedSize, rawBytes); 554 outputSavings("Encoding with " + comprAlgo, encodedCompressedSize, compressedSize); 555 556 System.out.println(); 557 } 558 } 559 560 private void outputTuplePct(String caption, long size) { 561 outputTuple(caption, INT_FORMAT + " (" + PCT_FORMAT + ")", size, size * 100.0 / rawKVs.length); 562 } 563 564 private void outputSavings(String caption, long part, long whole) { 565 double pct = 100.0 * (1 - 1.0 * part / whole); 566 double times = whole * 1.0 / part; 567 outputTuple(caption + " savings", PCT_FORMAT + " (%.2f x)", pct, times); 568 } 569 570 /** 571 * Test a data block encoder on the given HFile. Output results to console. 572 * @param kvLimit The limit of KeyValue which will be analyzed. 573 * @param hfilePath an HFile path on the file system. 574 * @param compressionName Compression algorithm used for comparison. 575 * @param doBenchmark Run performance benchmarks. 576 * @param doVerify Verify correctness. 577 * @throws IOException When pathName is incorrect. 578 */ 579 public static void testCodecs(Configuration conf, int kvLimit, String hfilePath, 580 String compressionName, boolean doBenchmark, boolean doVerify) throws IOException { 581 // create environment 582 Path path = new Path(hfilePath); 583 CacheConfig cacheConf = new CacheConfig(conf); 584 FileSystem fs = FileSystem.get(conf); 585 HStoreFile hsf = new HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true); 586 hsf.initReader(); 587 StoreFileReader reader = hsf.getReader(); 588 reader.loadFileInfo(); 589 KeyValueScanner scanner = 590 reader.getStoreFileScanner(true, true, false, hsf.getMaxMemStoreTS(), 0, false); 591 USE_TAG = reader.getHFileReader().getFileContext().isIncludesTags(); 592 // run the utilities 593 DataBlockEncodingTool comp = new DataBlockEncodingTool(conf, compressionName); 594 int majorVersion = reader.getHFileVersion(); 595 comp.useHBaseChecksum = majorVersion > 2 || (majorVersion == 2 596 && reader.getHFileMinorVersion() >= HFileReaderImpl.MINOR_VERSION_WITH_CHECKSUM); 597 comp.checkStatistics(scanner, kvLimit); 598 if (doVerify) { 599 comp.verifyCodecs(scanner, kvLimit); 600 } 601 if (doBenchmark) { 602 comp.benchmarkCodecs(); 603 } 604 comp.displayStatistics(); 605 606 // cleanup 607 scanner.close(); 608 reader.close(cacheConf.shouldEvictOnClose()); 609 } 610 611 private static void printUsage(Options options) { 612 System.err.println("Usage:"); 613 System.err 614 .println(String.format("./hbase %s <options>", DataBlockEncodingTool.class.getName())); 615 System.err.println("Options:"); 616 for (Object it : options.getOptions()) { 617 Option opt = (Option) it; 618 if (opt.hasArg()) { 619 System.err.println( 620 String.format("-%s %s: %s", opt.getOpt(), opt.getArgName(), opt.getDescription())); 621 } else { 622 System.err.println(String.format("-%s: %s", opt.getOpt(), opt.getDescription())); 623 } 624 } 625 } 626 627 /** 628 * A command line interface to benchmarks. Parses command-line arguments and runs the appropriate 629 * benchmarks. 630 * @param args Should have length at least 1 and holds the file path to HFile. 631 * @throws IOException If you specified the wrong file. 632 */ 633 public static void main(final String[] args) throws IOException { 634 // set up user arguments 635 Options options = new Options(); 636 options.addOption(OPT_HFILE_NAME, true, "HFile to analyse (REQUIRED)"); 637 options.getOption(OPT_HFILE_NAME).setArgName("FILENAME"); 638 options.addOption(OPT_KV_LIMIT, true, 639 "Maximum number of KeyValues to process. A benchmark stops running " 640 + "after iterating over this many KV pairs."); 641 options.getOption(OPT_KV_LIMIT).setArgName("NUMBER"); 642 options.addOption(OPT_MEASURE_THROUGHPUT, false, "Measure read throughput"); 643 options.addOption(OPT_OMIT_CORRECTNESS_TEST, false, "Omit corectness tests."); 644 options.addOption(OPT_COMPRESSION_ALGORITHM, true, 645 "What kind of compression algorithm use for comparison."); 646 options.addOption(OPT_BENCHMARK_N_TIMES, true, 647 "Number of times to run each benchmark. Default value: " + DEFAULT_BENCHMARK_N_TIMES); 648 options.addOption(OPT_BENCHMARK_N_OMIT, true, 649 "Number of first runs of every benchmark to exclude from " + "statistics (" 650 + DEFAULT_BENCHMARK_N_OMIT + " by default, so that " + "only the last " 651 + (DEFAULT_BENCHMARK_N_TIMES - DEFAULT_BENCHMARK_N_OMIT) 652 + " times are included in statistics.)"); 653 654 // parse arguments 655 CommandLineParser parser = new PosixParser(); 656 CommandLine cmd = null; 657 try { 658 cmd = parser.parse(options, args); 659 } catch (ParseException e) { 660 System.err.println("Could not parse arguments!"); 661 System.exit(-1); 662 return; // avoid warning 663 } 664 665 int kvLimit = Integer.MAX_VALUE; 666 if (cmd.hasOption(OPT_KV_LIMIT)) { 667 kvLimit = Integer.parseInt(cmd.getOptionValue(OPT_KV_LIMIT)); 668 if (kvLimit <= 0) { 669 LOG.error("KV_LIMIT should not less than 1."); 670 } 671 } 672 673 // basic argument sanity checks 674 if (!cmd.hasOption(OPT_HFILE_NAME)) { 675 LOG.error("Please specify HFile name using the " + OPT_HFILE_NAME + " option"); 676 printUsage(options); 677 System.exit(-1); 678 } 679 680 String pathName = cmd.getOptionValue(OPT_HFILE_NAME); 681 String compressionName = DEFAULT_COMPRESSION.getName(); 682 if (cmd.hasOption(OPT_COMPRESSION_ALGORITHM)) { 683 compressionName = cmd.getOptionValue(OPT_COMPRESSION_ALGORITHM).toLowerCase(Locale.ROOT); 684 } 685 boolean doBenchmark = cmd.hasOption(OPT_MEASURE_THROUGHPUT); 686 boolean doVerify = !cmd.hasOption(OPT_OMIT_CORRECTNESS_TEST); 687 688 if (cmd.hasOption(OPT_BENCHMARK_N_TIMES)) { 689 benchmarkNTimes = Integer.valueOf(cmd.getOptionValue(OPT_BENCHMARK_N_TIMES)); 690 } 691 if (cmd.hasOption(OPT_BENCHMARK_N_OMIT)) { 692 benchmarkNOmit = Integer.valueOf(cmd.getOptionValue(OPT_BENCHMARK_N_OMIT)); 693 } 694 if (benchmarkNTimes < benchmarkNOmit) { 695 LOG.error("The number of times to run each benchmark (" + benchmarkNTimes 696 + ") must be greater than the number of benchmark runs to exclude " + "from statistics (" 697 + benchmarkNOmit + ")"); 698 System.exit(1); 699 } 700 LOG.info("Running benchmark " + benchmarkNTimes + " times. " + "Excluding the first " 701 + benchmarkNOmit + " times from statistics."); 702 703 final Configuration conf = HBaseConfiguration.create(); 704 testCodecs(conf, kvLimit, pathName, compressionName, doBenchmark, doVerify); 705 } 706 707}