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