1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17 package org.apache.hadoop.hbase.regionserver;
18
19 import java.io.ByteArrayInputStream;
20 import java.io.ByteArrayOutputStream;
21 import java.io.IOException;
22 import java.io.InputStream;
23 import java.text.DecimalFormat;
24 import java.util.ArrayList;
25 import java.util.Iterator;
26 import java.util.List;
27 import java.util.Locale;
28
29 import org.apache.commons.cli.CommandLine;
30 import org.apache.commons.cli.CommandLineParser;
31 import org.apache.commons.cli.Option;
32 import org.apache.commons.cli.Options;
33 import org.apache.commons.cli.ParseException;
34 import org.apache.commons.cli.PosixParser;
35 import org.apache.commons.logging.Log;
36 import org.apache.commons.logging.LogFactory;
37 import org.apache.hadoop.conf.Configuration;
38 import org.apache.hadoop.fs.FileSystem;
39 import org.apache.hadoop.fs.Path;
40 import org.apache.hadoop.hbase.Cell;
41 import org.apache.hadoop.hbase.HBaseConfiguration;
42 import org.apache.hadoop.hbase.KeyValue;
43 import org.apache.hadoop.hbase.KeyValueUtil;
44 import org.apache.hadoop.hbase.io.compress.Compression;
45 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
46 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder;
47 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
48 import org.apache.hadoop.hbase.io.encoding.EncodedDataBlock;
49 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
50 import org.apache.hadoop.hbase.io.hfile.HFileBlock;
51 import org.apache.hadoop.hbase.io.hfile.HFileContext;
52 import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
53 import org.apache.hadoop.hbase.io.hfile.HFileReaderV2;
54 import org.apache.hadoop.hbase.util.Bytes;
55 import org.apache.hadoop.io.compress.CompressionOutputStream;
56 import org.apache.hadoop.io.compress.Compressor;
57 import org.apache.hadoop.io.compress.Decompressor;
58
59
60
61
62
63 public class DataBlockEncodingTool {
64 private static final Log LOG = LogFactory.getLog(
65 DataBlockEncodingTool.class);
66
67 private static final boolean includesMemstoreTS = true;
68
69
70
71
72
73
74 private static final int DEFAULT_BENCHMARK_N_TIMES = 12;
75
76
77
78
79
80 private static final int DEFAULT_BENCHMARK_N_OMIT = 2;
81
82
83 private static final String OPT_HFILE_NAME = "f";
84
85
86 private static final String OPT_KV_LIMIT = "n";
87
88
89 private static final String OPT_MEASURE_THROUGHPUT = "b";
90
91
92 private static final String OPT_OMIT_CORRECTNESS_TEST = "c";
93
94
95 private static final String OPT_ENCODING_ALGORITHM = "a";
96
97
98 private static final String OPT_BENCHMARK_N_TIMES = "t";
99
100
101 private static final String OPT_BENCHMARK_N_OMIT = "omit";
102
103
104 private static final Algorithm DEFAULT_COMPRESSION =
105 Compression.Algorithm.GZ;
106
107 private static final DecimalFormat DELIMITED_DECIMAL_FORMAT =
108 new DecimalFormat();
109
110 static {
111 DELIMITED_DECIMAL_FORMAT.setGroupingSize(3);
112 }
113
114 private static final String PCT_FORMAT = "%.2f %%";
115 private static final String INT_FORMAT = "%d";
116
117 private static int benchmarkNTimes = DEFAULT_BENCHMARK_N_TIMES;
118 private static int benchmarkNOmit = DEFAULT_BENCHMARK_N_OMIT;
119
120 private List<EncodedDataBlock> codecs = new ArrayList<EncodedDataBlock>();
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 private long totalTagsLength = 0;
127
128 private byte[] rawKVs;
129 private boolean useHBaseChecksum = false;
130
131 private final String compressionAlgorithmName;
132 private final Algorithm compressionAlgorithm;
133 private final Compressor compressor;
134 private final Decompressor decompressor;
135
136 private static enum Manipulation {
137 ENCODING,
138 DECODING,
139 COMPRESSION,
140 DECOMPRESSION;
141
142 @Override
143 public String toString() {
144 String s = super.toString();
145 StringBuilder sb = new StringBuilder();
146 sb.append(s.charAt(0));
147 sb.append(s.substring(1).toLowerCase(Locale.ROOT));
148 return sb.toString();
149 }
150 }
151
152
153
154
155
156 public DataBlockEncodingTool(String compressionAlgorithmName) {
157 this.compressionAlgorithmName = compressionAlgorithmName;
158 this.compressionAlgorithm = Compression.getCompressionAlgorithmByName(
159 compressionAlgorithmName);
160 this.compressor = this.compressionAlgorithm.getCompressor();
161 this.decompressor = this.compressionAlgorithm.getDecompressor();
162 }
163
164
165
166
167
168
169
170 public void checkStatistics(final KeyValueScanner scanner, final int kvLimit)
171 throws IOException {
172 scanner.seek(KeyValue.LOWESTKEY);
173
174 KeyValue currentKV;
175
176 byte[] previousKey = null;
177 byte[] currentKey;
178
179 DataBlockEncoding[] encodings = DataBlockEncoding.values();
180
181 ByteArrayOutputStream uncompressedOutputStream =
182 new ByteArrayOutputStream();
183
184 int j = 0;
185 while ((currentKV = KeyValueUtil.ensureKeyValue(scanner.next())) != null && j < kvLimit) {
186
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 uncompressedOutputStream.write(currentKV.getBuffer(),
197 currentKV.getOffset(), currentKV.getLength());
198
199 previousKey = currentKey;
200
201 int kLen = currentKV.getKeyLength();
202 int vLen = currentKV.getValueLength();
203 int cfLen = currentKV.getFamilyLength(currentKV.getFamilyOffset());
204 int restLen = currentKV.getLength() - kLen - vLen;
205 int tagsLen = currentKV.getTagsLength();
206
207 totalKeyLength += kLen;
208 totalValueLength += vLen;
209 totalPrefixLength += restLen;
210 totalCFLength += cfLen;
211 totalTagsLength += tagsLen;
212 }
213
214 rawKVs = uncompressedOutputStream.toByteArray();
215 boolean useTag = (totalTagsLength > 0);
216 for (DataBlockEncoding encoding : encodings) {
217 if (encoding == DataBlockEncoding.NONE) {
218 continue;
219 }
220 DataBlockEncoder d = encoding.getEncoder();
221 HFileContext meta = new HFileContextBuilder()
222 .withCompression(Compression.Algorithm.NONE)
223 .withIncludesMvcc(includesMemstoreTS)
224 .withIncludesTags(useTag).build();
225 codecs.add(new EncodedDataBlock(d, encoding, rawKVs, meta ));
226 }
227 }
228
229
230
231
232
233
234
235
236
237 public boolean verifyCodecs(final KeyValueScanner scanner, final int kvLimit)
238 throws IOException {
239 KeyValue currentKv;
240
241 scanner.seek(KeyValue.LOWESTKEY);
242 List<Iterator<Cell>> codecIterators =
243 new ArrayList<Iterator<Cell>>();
244 for(EncodedDataBlock codec : codecs) {
245 codecIterators.add(codec.getIterator(HFileBlock.headerSize(useHBaseChecksum)));
246 }
247
248 int j = 0;
249 while ((currentKv = KeyValueUtil.ensureKeyValue(scanner.next())) != null && j < kvLimit) {
250
251 ++j;
252 for (Iterator<Cell> it : codecIterators) {
253 Cell c = it.next();
254 KeyValue codecKv = KeyValueUtil.ensureKeyValue(c);
255 if (codecKv == null || 0 != Bytes.compareTo(
256 codecKv.getBuffer(), codecKv.getOffset(), codecKv.getLength(),
257 currentKv.getBuffer(), currentKv.getOffset(),
258 currentKv.getLength())) {
259 if (codecKv == null) {
260 LOG.error("There is a bug in codec " + it +
261 " it returned null KeyValue,");
262 } else {
263 int prefix = 0;
264 int limitLength = 2 * Bytes.SIZEOF_INT +
265 Math.min(codecKv.getLength(), currentKv.getLength());
266 while (prefix < limitLength &&
267 codecKv.getBuffer()[prefix + codecKv.getOffset()] ==
268 currentKv.getBuffer()[prefix + currentKv.getOffset()]) {
269 prefix++;
270 }
271
272 LOG.error("There is bug in codec " + it.toString() +
273 "\n on element " + j +
274 "\n codecKv.getKeyLength() " + codecKv.getKeyLength() +
275 "\n codecKv.getValueLength() " + codecKv.getValueLength() +
276 "\n codecKv.getLength() " + codecKv.getLength() +
277 "\n currentKv.getKeyLength() " + currentKv.getKeyLength() +
278 "\n currentKv.getValueLength() " + currentKv.getValueLength() +
279 "\n codecKv.getLength() " + currentKv.getLength() +
280 "\n currentKV rowLength " + currentKv.getRowLength() +
281 " familyName " + currentKv.getFamilyLength() +
282 " qualifier " + currentKv.getQualifierLength() +
283 "\n prefix " + prefix +
284 "\n codecKv '" + Bytes.toStringBinary(codecKv.getBuffer(),
285 codecKv.getOffset(), prefix) + "' diff '" +
286 Bytes.toStringBinary(codecKv.getBuffer(),
287 codecKv.getOffset() + prefix, codecKv.getLength() -
288 prefix) + "'" +
289 "\n currentKv '" + Bytes.toStringBinary(
290 currentKv.getBuffer(),
291 currentKv.getOffset(), prefix) + "' diff '" +
292 Bytes.toStringBinary(currentKv.getBuffer(),
293 currentKv.getOffset() + prefix, currentKv.getLength() -
294 prefix) + "'"
295 );
296 }
297 return false;
298 }
299 }
300 }
301
302 LOG.info("Verification was successful!");
303
304 return true;
305 }
306
307
308
309
310 public void benchmarkCodecs() throws IOException {
311 LOG.info("Starting a throughput benchmark for data block encoding codecs");
312 int prevTotalSize = -1;
313 for (EncodedDataBlock codec : codecs) {
314 prevTotalSize = benchmarkEncoder(prevTotalSize, codec);
315 }
316
317 benchmarkDefaultCompression(prevTotalSize, rawKVs);
318 }
319
320
321
322
323
324
325
326
327 private int benchmarkEncoder(int previousTotalSize, EncodedDataBlock codec) {
328 int prevTotalSize = previousTotalSize;
329 int totalSize = 0;
330
331
332 List<Long> durations = new ArrayList<Long>();
333 for (int itTime = 0; itTime < benchmarkNTimes; ++itTime) {
334 totalSize = 0;
335
336 Iterator<Cell> it;
337
338 it = codec.getIterator(HFileBlock.headerSize(useHBaseChecksum));
339
340
341
342 final long startTime = System.nanoTime();
343 while (it.hasNext()) {
344 totalSize += KeyValueUtil.ensureKeyValue(it.next()).getLength();
345 }
346 final long finishTime = System.nanoTime();
347 if (itTime >= benchmarkNOmit) {
348 durations.add(finishTime - startTime);
349 }
350
351 if (prevTotalSize != -1 && prevTotalSize != totalSize) {
352 throw new IllegalStateException(String.format(
353 "Algorithm '%s' decoded data to different size", codec.toString()));
354 }
355 prevTotalSize = totalSize;
356 }
357
358 List<Long> encodingDurations = new ArrayList<Long>();
359 for (int itTime = 0; itTime < benchmarkNTimes; ++itTime) {
360 final long startTime = System.nanoTime();
361 codec.encodeData();
362 final long finishTime = System.nanoTime();
363 if (itTime >= benchmarkNOmit) {
364 encodingDurations.add(finishTime - startTime);
365 }
366 }
367
368 System.out.println(codec.toString() + ":");
369 printBenchmarkResult(totalSize, encodingDurations, Manipulation.ENCODING);
370 printBenchmarkResult(totalSize, durations, Manipulation.DECODING);
371 System.out.println();
372
373 return prevTotalSize;
374 }
375
376 private void benchmarkDefaultCompression(int totalSize, byte[] rawBuffer)
377 throws IOException {
378 benchmarkAlgorithm(compressionAlgorithm,
379 compressionAlgorithmName.toUpperCase(Locale.ROOT), rawBuffer, 0, totalSize);
380 }
381
382
383
384
385
386
387
388
389
390
391 public void benchmarkAlgorithm(Compression.Algorithm algorithm, String name,
392 byte[] buffer, int offset, int length) throws IOException {
393 System.out.println(name + ":");
394
395
396 List<Long> compressDurations = new ArrayList<Long>();
397 ByteArrayOutputStream compressedStream = new ByteArrayOutputStream();
398 CompressionOutputStream compressingStream =
399 algorithm.createPlainCompressionStream(compressedStream, compressor);
400 try {
401 for (int itTime = 0; itTime < benchmarkNTimes; ++itTime) {
402 final long startTime = System.nanoTime();
403 compressingStream.resetState();
404 compressedStream.reset();
405 compressingStream.write(buffer, offset, length);
406 compressingStream.flush();
407 compressedStream.toByteArray();
408
409 final long finishTime = System.nanoTime();
410
411
412 if (itTime >= benchmarkNOmit) {
413 compressDurations.add(finishTime - startTime);
414 }
415 }
416 } catch (IOException e) {
417 throw new RuntimeException(String.format(
418 "Benchmark, or encoding algorithm '%s' cause some stream problems",
419 name), e);
420 }
421 compressingStream.close();
422 printBenchmarkResult(length, compressDurations, Manipulation.COMPRESSION);
423
424 byte[] compBuffer = compressedStream.toByteArray();
425
426
427 List<Long> durations = new ArrayList<Long>();
428 for (int itTime = 0; itTime < benchmarkNTimes; ++itTime) {
429 final long startTime = System.nanoTime();
430 byte[] newBuf = new byte[length + 1];
431
432 try {
433 ByteArrayInputStream downStream = new ByteArrayInputStream(compBuffer,
434 0, compBuffer.length);
435 InputStream decompressedStream = algorithm.createDecompressionStream(
436 downStream, decompressor, 0);
437
438 int destOffset = 0;
439 int nextChunk;
440 while ((nextChunk = decompressedStream.available()) > 0) {
441 destOffset += decompressedStream.read(newBuf, destOffset, nextChunk);
442 }
443 decompressedStream.close();
444
445
446 KeyValue kv;
447 for (int pos = 0; pos < length; pos += kv.getLength()) {
448 kv = new KeyValue(newBuf, pos);
449 }
450
451 } catch (IOException e) {
452 throw new RuntimeException(String.format(
453 "Decoding path in '%s' algorithm cause exception ", name), e);
454 }
455
456 final long finishTime = System.nanoTime();
457
458
459 if (0 != Bytes.compareTo(buffer, 0, length, newBuf, 0, length)) {
460 int prefix = 0;
461 for(; prefix < buffer.length && prefix < newBuf.length; ++prefix) {
462 if (buffer[prefix] != newBuf[prefix]) {
463 break;
464 }
465 }
466 throw new RuntimeException(String.format(
467 "Algorithm '%s' is corrupting the data", name));
468 }
469
470
471 if (itTime >= benchmarkNOmit) {
472 durations.add(finishTime - startTime);
473 }
474 }
475 printBenchmarkResult(length, durations, Manipulation.DECOMPRESSION);
476 System.out.println();
477 }
478
479 private static final double BYTES_IN_MB = 1024 * 1024.0;
480 private static final double NS_IN_SEC = 1000.0 * 1000.0 * 1000.0;
481 private static final double MB_SEC_COEF = NS_IN_SEC / BYTES_IN_MB;
482
483 private static void printBenchmarkResult(int totalSize,
484 List<Long> durationsInNanoSec, Manipulation manipulation) {
485 final int n = durationsInNanoSec.size();
486 long meanTime = 0;
487 for (long time : durationsInNanoSec) {
488 meanTime += time;
489 }
490 meanTime /= n;
491
492 double meanMBPerSec = totalSize * MB_SEC_COEF / meanTime;
493 double mbPerSecSTD = 0;
494 if (n > 0) {
495 for (long time : durationsInNanoSec) {
496 double mbPerSec = totalSize * MB_SEC_COEF / time;
497 double dev = mbPerSec - meanMBPerSec;
498 mbPerSecSTD += dev * dev;
499 }
500 mbPerSecSTD = Math.sqrt(mbPerSecSTD / n);
501 }
502
503 outputTuple(manipulation + " performance", "%6.2f MB/s (+/- %.2f MB/s)",
504 meanMBPerSec, mbPerSecSTD);
505 }
506
507 private static void outputTuple(String caption, String format,
508 Object... values) {
509 if (format.startsWith(INT_FORMAT)) {
510 format = "%s" + format.substring(INT_FORMAT.length());
511 values[0] = DELIMITED_DECIMAL_FORMAT.format(values[0]);
512 }
513
514 StringBuilder sb = new StringBuilder();
515 sb.append(" ");
516 sb.append(caption);
517 sb.append(":");
518
519 String v = String.format(format, values);
520 int padding = 60 - sb.length() - v.length();
521 for (int i = 0; i < padding; ++i) {
522 sb.append(' ');
523 }
524 sb.append(v);
525 System.out.println(sb);
526 }
527
528
529
530
531
532 public void displayStatistics() throws IOException {
533 final String comprAlgo = compressionAlgorithmName.toUpperCase(Locale.ROOT);
534 long rawBytes = totalKeyLength + totalPrefixLength + totalValueLength;
535
536 System.out.println("Raw data size:");
537 outputTuple("Raw bytes", INT_FORMAT, rawBytes);
538 outputTuplePct("Key bytes", totalKeyLength);
539 outputTuplePct("Value bytes", totalValueLength);
540 outputTuplePct("KV infrastructure", totalPrefixLength);
541 outputTuplePct("CF overhead", totalCFLength);
542 outputTuplePct("Total key redundancy", totalKeyRedundancyLength);
543
544 int compressedSize = EncodedDataBlock.getCompressedSize(
545 compressionAlgorithm, compressor, rawKVs, 0, rawKVs.length);
546 outputTuple(comprAlgo + " only size", INT_FORMAT,
547 compressedSize);
548 outputSavings(comprAlgo + " only", compressedSize, rawBytes);
549 System.out.println();
550
551 for (EncodedDataBlock codec : codecs) {
552 System.out.println(codec.toString());
553 long encodedBytes = codec.getSize();
554 outputTuple("Encoded bytes", INT_FORMAT, encodedBytes);
555 outputSavings("Key encoding", encodedBytes - totalValueLength,
556 rawBytes - totalValueLength);
557 outputSavings("Total encoding", encodedBytes, rawBytes);
558
559 int encodedCompressedSize = codec.getEncodedCompressedSize(
560 compressionAlgorithm, compressor);
561 outputTuple("Encoding + " + comprAlgo + " size", INT_FORMAT,
562 encodedCompressedSize);
563 outputSavings("Encoding + " + comprAlgo, encodedCompressedSize, rawBytes);
564 outputSavings("Encoding with " + comprAlgo, encodedCompressedSize,
565 compressedSize);
566
567 System.out.println();
568 }
569 }
570
571 private void outputTuplePct(String caption, long size) {
572 outputTuple(caption, INT_FORMAT + " (" + PCT_FORMAT + ")",
573 size, size * 100.0 / rawKVs.length);
574 }
575
576 private void outputSavings(String caption, long part, long whole) {
577 double pct = 100.0 * (1 - 1.0 * part / whole);
578 double times = whole * 1.0 / part;
579 outputTuple(caption + " savings", PCT_FORMAT + " (%.2f x)",
580 pct, times);
581 }
582
583
584
585
586
587
588
589
590
591
592 public static void testCodecs(Configuration conf, int kvLimit,
593 String hfilePath, String compressionName, boolean doBenchmark,
594 boolean doVerify) throws IOException {
595
596 Path path = new Path(hfilePath);
597 CacheConfig cacheConf = new CacheConfig(conf);
598 FileSystem fs = FileSystem.get(conf);
599 StoreFile hsf = new StoreFile(fs, path, conf, cacheConf,
600 BloomType.NONE);
601
602 StoreFile.Reader reader = hsf.createReader();
603 reader.loadFileInfo();
604 KeyValueScanner scanner = reader.getStoreFileScanner(true, true, false, 0, 0, false);
605
606
607 DataBlockEncodingTool comp = new DataBlockEncodingTool(compressionName);
608 int majorVersion = reader.getHFileVersion();
609 comp.useHBaseChecksum = majorVersion > 2
610 || (majorVersion == 2 && reader.getHFileMinorVersion() >= HFileReaderV2.MINOR_VERSION_WITH_CHECKSUM);
611 comp.checkStatistics(scanner, kvLimit);
612 if (doVerify) {
613 comp.verifyCodecs(scanner, kvLimit);
614 }
615 if (doBenchmark) {
616 comp.benchmarkCodecs();
617 }
618 comp.displayStatistics();
619
620
621 scanner.close();
622 reader.close(cacheConf.shouldEvictOnClose());
623 }
624
625 private static void printUsage(Options options) {
626 System.err.println("Usage:");
627 System.err.println(String.format("./hbase %s <options>",
628 DataBlockEncodingTool.class.getName()));
629 System.err.println("Options:");
630 for (Object it : options.getOptions()) {
631 Option opt = (Option) it;
632 if (opt.hasArg()) {
633 System.err.println(String.format("-%s %s: %s", opt.getOpt(),
634 opt.getArgName(), opt.getDescription()));
635 } else {
636 System.err.println(String.format("-%s: %s", opt.getOpt(),
637 opt.getDescription()));
638 }
639 }
640 }
641
642
643
644
645
646
647
648 public static void main(final String[] args) throws IOException {
649
650 Options options = new Options();
651 options.addOption(OPT_HFILE_NAME, true, "HFile to analyse (REQUIRED)");
652 options.getOption(OPT_HFILE_NAME).setArgName("FILENAME");
653 options.addOption(OPT_KV_LIMIT, true,
654 "Maximum number of KeyValues to process. A benchmark stops running " +
655 "after iterating over this many KV pairs.");
656 options.getOption(OPT_KV_LIMIT).setArgName("NUMBER");
657 options.addOption(OPT_MEASURE_THROUGHPUT, false,
658 "Measure read throughput");
659 options.addOption(OPT_OMIT_CORRECTNESS_TEST, false,
660 "Omit corectness tests.");
661 options.addOption(OPT_ENCODING_ALGORITHM, true,
662 "What kind of compression algorithm use for comparison.");
663 options.addOption(OPT_BENCHMARK_N_TIMES,
664 true, "Number of times to run each benchmark. Default value: " +
665 DEFAULT_BENCHMARK_N_TIMES);
666 options.addOption(OPT_BENCHMARK_N_OMIT, true,
667 "Number of first runs of every benchmark to exclude from "
668 + "statistics (" + DEFAULT_BENCHMARK_N_OMIT
669 + " by default, so that " + "only the last "
670 + (DEFAULT_BENCHMARK_N_TIMES - DEFAULT_BENCHMARK_N_OMIT)
671 + " times are included in statistics.)");
672
673
674 CommandLineParser parser = new PosixParser();
675 CommandLine cmd = null;
676 try {
677 cmd = parser.parse(options, args);
678 } catch (ParseException e) {
679 System.err.println("Could not parse arguments!");
680 System.exit(-1);
681 return;
682 }
683
684 int kvLimit = Integer.MAX_VALUE;
685 if (cmd.hasOption(OPT_KV_LIMIT)) {
686 kvLimit = Integer.parseInt(cmd.getOptionValue(OPT_KV_LIMIT));
687 }
688
689
690 if (!cmd.hasOption(OPT_HFILE_NAME)) {
691 LOG.error("Please specify HFile name using the " + OPT_HFILE_NAME
692 + " option");
693 printUsage(options);
694 System.exit(-1);
695 }
696
697 String pathName = cmd.getOptionValue(OPT_HFILE_NAME);
698 String compressionName = DEFAULT_COMPRESSION.getName();
699 if (cmd.hasOption(OPT_ENCODING_ALGORITHM)) {
700 compressionName =
701 cmd.getOptionValue(OPT_ENCODING_ALGORITHM).toLowerCase(Locale.ROOT);
702 }
703 boolean doBenchmark = cmd.hasOption(OPT_MEASURE_THROUGHPUT);
704 boolean doVerify = !cmd.hasOption(OPT_OMIT_CORRECTNESS_TEST);
705
706 if (cmd.hasOption(OPT_BENCHMARK_N_TIMES)) {
707 benchmarkNTimes = Integer.valueOf(cmd.getOptionValue(
708 OPT_BENCHMARK_N_TIMES));
709 }
710 if (cmd.hasOption(OPT_BENCHMARK_N_OMIT)) {
711 benchmarkNOmit =
712 Integer.valueOf(cmd.getOptionValue(OPT_BENCHMARK_N_OMIT));
713 }
714 if (benchmarkNTimes < benchmarkNOmit) {
715 LOG.error("The number of times to run each benchmark ("
716 + benchmarkNTimes
717 + ") must be greater than the number of benchmark runs to exclude "
718 + "from statistics (" + benchmarkNOmit + ")");
719 System.exit(1);
720 }
721 LOG.info("Running benchmark " + benchmarkNTimes + " times. " +
722 "Excluding the first " + benchmarkNOmit + " times from statistics.");
723
724 final Configuration conf = HBaseConfiguration.create();
725 try {
726 testCodecs(conf, kvLimit, pathName, compressionName, doBenchmark,
727 doVerify);
728 } finally {
729 (new CacheConfig(conf)).getBlockCache().shutdown();
730 }
731 }
732
733 }