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