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.io.OutputStream;
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.HBaseConfiguration;
40 import org.apache.hadoop.hbase.KeyValue;
41 import org.apache.hadoop.hbase.io.encoding.EncodedDataBlock;
42 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder;
43 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
44 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
45 import org.apache.hadoop.hbase.io.hfile.Compression;
46 import org.apache.hadoop.hbase.io.hfile.NoOpDataBlockEncoder;
47 import org.apache.hadoop.hbase.io.hfile.Compression.Algorithm;
48 import org.apache.hadoop.hbase.util.Bytes;
49 import org.apache.hadoop.io.compress.Compressor;
50 import org.apache.hadoop.io.compress.Decompressor;
51
52
53
54
55
56 public class DataBlockEncodingTool {
57 private static final Log LOG = LogFactory.getLog(
58 DataBlockEncodingTool.class);
59
60 private static final boolean includesMemstoreTS = true;
61
62
63
64
65
66
67 public static int BENCHMARK_N_TIMES = 12;
68
69
70
71
72
73
74 public static int BENCHMARK_N_OMIT = 2;
75
76
77 private static final Algorithm DEFAULT_COMPRESSION =
78 Compression.Algorithm.GZ;
79
80 private List<EncodedDataBlock> codecs = new ArrayList<EncodedDataBlock>();
81 private int totalPrefixLength = 0;
82 private int totalKeyLength = 0;
83 private int totalValueLength = 0;
84 private int totalKeyRedundancyLength = 0;
85
86 final private String compressionAlgorithmName;
87 final private Algorithm compressionAlgorithm;
88 final private Compressor compressor;
89 final private Decompressor decompressor;
90
91
92
93
94
95 public DataBlockEncodingTool(String compressionAlgorithmName) {
96 this.compressionAlgorithmName = compressionAlgorithmName;
97 this.compressionAlgorithm = Compression.getCompressionAlgorithmByName(
98 compressionAlgorithmName);
99 this.compressor = this.compressionAlgorithm.getCompressor();
100 this.decompressor = this.compressionAlgorithm.getDecompressor();
101 }
102
103
104
105
106
107
108
109 public void checkStatistics(final KeyValueScanner scanner, final int kvLimit)
110 throws IOException {
111 scanner.seek(KeyValue.LOWESTKEY);
112
113 KeyValue currentKv;
114
115 byte[] previousKey = null;
116 byte[] currentKey;
117
118 List<DataBlockEncoder> dataBlockEncoders =
119 DataBlockEncoding.getAllEncoders();
120
121 for (DataBlockEncoder d : dataBlockEncoders) {
122 codecs.add(new EncodedDataBlock(d, includesMemstoreTS));
123 }
124
125 int j = 0;
126 while ((currentKv = scanner.next()) != null && j < kvLimit) {
127
128 j++;
129 currentKey = currentKv.getKey();
130 if (previousKey != null) {
131 for (int i = 0; i < previousKey.length && i < currentKey.length &&
132 previousKey[i] == currentKey[i]; ++i) {
133 totalKeyRedundancyLength++;
134 }
135 }
136
137 for (EncodedDataBlock codec : codecs) {
138 codec.addKv(currentKv);
139 }
140
141 previousKey = currentKey;
142
143 totalPrefixLength += currentKv.getLength() - currentKv.getKeyLength() -
144 currentKv.getValueLength();
145 totalKeyLength += currentKv.getKeyLength();
146 totalValueLength += currentKv.getValueLength();
147 }
148 }
149
150
151
152
153
154
155
156
157
158 public boolean verifyCodecs(final KeyValueScanner scanner, final int kvLimit)
159 throws IOException {
160 KeyValue currentKv;
161
162 scanner.seek(KeyValue.LOWESTKEY);
163 List<Iterator<KeyValue>> codecIterators =
164 new ArrayList<Iterator<KeyValue>>();
165 for(EncodedDataBlock codec : codecs) {
166 codecIterators.add(codec.getIterator());
167 }
168
169 int j = 0;
170 while ((currentKv = scanner.next()) != null && j < kvLimit) {
171
172 ++j;
173 for (Iterator<KeyValue> it : codecIterators) {
174 KeyValue codecKv = it.next();
175 if (codecKv == null || 0 != Bytes.compareTo(
176 codecKv.getBuffer(), codecKv.getOffset(), codecKv.getLength(),
177 currentKv.getBuffer(), currentKv.getOffset(),
178 currentKv.getLength())) {
179 if (codecKv == null) {
180 LOG.error("There is a bug in codec " + it +
181 " it returned null KeyValue,");
182 } else {
183 int prefix = 0;
184 int limitLength = 2 * Bytes.SIZEOF_INT +
185 Math.min(codecKv.getLength(), currentKv.getLength());
186 while (prefix < limitLength &&
187 codecKv.getBuffer()[prefix + codecKv.getOffset()] ==
188 currentKv.getBuffer()[prefix + currentKv.getOffset()]) {
189 prefix++;
190 }
191
192 LOG.error("There is bug in codec " + it.toString() +
193 "\n on element " + j +
194 "\n codecKv.getKeyLength() " + codecKv.getKeyLength() +
195 "\n codecKv.getValueLength() " + codecKv.getValueLength() +
196 "\n codecKv.getLength() " + codecKv.getLength() +
197 "\n currentKv.getKeyLength() " + currentKv.getKeyLength() +
198 "\n currentKv.getValueLength() " + currentKv.getValueLength() +
199 "\n codecKv.getLength() " + currentKv.getLength() +
200 "\n currentKV rowLength " + currentKv.getRowLength() +
201 " familyName " + currentKv.getFamilyLength() +
202 " qualifier " + currentKv.getQualifierLength() +
203 "\n prefix " + prefix +
204 "\n codecKv '" + Bytes.toStringBinary(codecKv.getBuffer(),
205 codecKv.getOffset(), prefix) + "' diff '" +
206 Bytes.toStringBinary(codecKv.getBuffer(),
207 codecKv.getOffset() + prefix, codecKv.getLength() -
208 prefix) + "'" +
209 "\n currentKv '" + Bytes.toStringBinary(
210 currentKv.getBuffer(),
211 currentKv.getOffset(), prefix) + "' diff '" +
212 Bytes.toStringBinary(currentKv.getBuffer(),
213 currentKv.getOffset() + prefix, currentKv.getLength() -
214 prefix) + "'"
215 );
216 }
217 return false;
218 }
219 }
220 }
221
222 LOG.info("Verification was successful!");
223
224 return true;
225 }
226
227
228
229
230 public void benchmarkCodecs() {
231 int prevTotalSize = -1;
232 for (EncodedDataBlock codec : codecs) {
233 prevTotalSize = benchmarkEncoder(prevTotalSize, codec);
234 }
235
236 byte[] buffer = codecs.get(0).getRawKeyValues();
237
238 benchmarkDefaultCompression(prevTotalSize, buffer);
239 }
240
241
242
243
244
245
246
247
248 private int benchmarkEncoder(int previousTotalSize, EncodedDataBlock codec) {
249 int prevTotalSize = previousTotalSize;
250 int totalSize = 0;
251
252
253 List<Long> durations = new ArrayList<Long>();
254 for (int itTime = 0; itTime < BENCHMARK_N_TIMES; ++itTime) {
255 totalSize = 0;
256
257 Iterator<KeyValue> it;
258
259 it = codec.getIterator();
260
261
262
263 final long startTime = System.nanoTime();
264 while (it.hasNext()) {
265 totalSize += it.next().getLength();
266 }
267 final long finishTime = System.nanoTime();
268 if (itTime >= BENCHMARK_N_OMIT) {
269 durations.add(finishTime - startTime);
270 }
271
272 if (prevTotalSize != -1 && prevTotalSize != totalSize) {
273 throw new IllegalStateException(String.format(
274 "Algorithm '%s' decoded data to different size", codec.toString()));
275 }
276 prevTotalSize = totalSize;
277 }
278
279
280 List<Long> compressDurations = new ArrayList<Long>();
281 for (int itTime = 0; itTime < BENCHMARK_N_TIMES; ++itTime) {
282 final long startTime = System.nanoTime();
283 codec.doCompressData();
284 final long finishTime = System.nanoTime();
285 if (itTime >= BENCHMARK_N_OMIT) {
286 compressDurations.add(finishTime - startTime);
287 }
288 }
289
290 System.out.println(codec.toString() + ":");
291 printBenchmarkResult(totalSize, compressDurations, false);
292 printBenchmarkResult(totalSize, durations, true);
293
294 return prevTotalSize;
295 }
296
297 private void benchmarkDefaultCompression(int totalSize, byte[] rawBuffer) {
298 benchmarkAlgorithm(compressionAlgorithm, compressor, decompressor,
299 compressionAlgorithmName.toUpperCase(), rawBuffer, 0, totalSize);
300 }
301
302
303
304
305
306
307
308
309
310
311
312 public static void benchmarkAlgorithm(
313 Compression.Algorithm algorithm,
314 Compressor compressorCodec,
315 Decompressor decompressorCodec,
316 String name,
317 byte[] buffer, int offset, int length) {
318 System.out.println(name + ":");
319
320
321 List<Long> compressDurations = new ArrayList<Long>();
322 ByteArrayOutputStream compressedStream = new ByteArrayOutputStream();
323 OutputStream compressingStream;
324 try {
325 for (int itTime = 0; itTime < BENCHMARK_N_TIMES; ++itTime) {
326 final long startTime = System.nanoTime();
327 compressingStream = algorithm.createCompressionStream(
328 compressedStream, compressorCodec, 0);
329 compressingStream.write(buffer, offset, length);
330 compressingStream.flush();
331 compressedStream.toByteArray();
332
333 final long finishTime = System.nanoTime();
334
335
336 if (itTime >= BENCHMARK_N_OMIT) {
337 compressDurations.add(finishTime - startTime);
338 }
339
340 if (itTime + 1 < BENCHMARK_N_TIMES) {
341 compressedStream.reset();
342 }
343 }
344 } catch (IOException e) {
345 throw new RuntimeException(String.format(
346 "Benchmark, or encoding algorithm '%s' cause some stream problems",
347 name), e);
348 }
349 printBenchmarkResult(length, compressDurations, false);
350
351
352 byte[] compBuffer = compressedStream.toByteArray();
353
354
355 List<Long> durations = new ArrayList<Long>();
356 for (int itTime = 0; itTime < BENCHMARK_N_TIMES; ++itTime) {
357 final long startTime = System.nanoTime();
358 byte[] newBuf = new byte[length + 1];
359
360 try {
361
362 ByteArrayInputStream downStream = new ByteArrayInputStream(compBuffer,
363 0, compBuffer.length);
364 InputStream decompressedStream = algorithm.createDecompressionStream(
365 downStream, decompressorCodec, 0);
366
367 int destOffset = 0;
368 int nextChunk;
369 while ((nextChunk = decompressedStream.available()) > 0) {
370 destOffset += decompressedStream.read(newBuf, destOffset, nextChunk);
371 }
372 decompressedStream.close();
373
374
375 KeyValue kv;
376 for (int pos = 0; pos < length; pos += kv.getLength()) {
377 kv = new KeyValue(newBuf, pos);
378 }
379
380 } catch (IOException e) {
381 throw new RuntimeException(String.format(
382 "Decoding path in '%s' algorithm cause exception ", name), e);
383 }
384
385 final long finishTime = System.nanoTime();
386
387
388 if (0 != Bytes.compareTo(buffer, 0, length, newBuf, 0, length)) {
389 int prefix = 0;
390 for(; prefix < buffer.length && prefix < newBuf.length; ++prefix) {
391 if (buffer[prefix] != newBuf[prefix]) {
392 break;
393 }
394 }
395 throw new RuntimeException(String.format(
396 "Algorithm '%s' is corrupting the data", name));
397 }
398
399
400 if (itTime >= BENCHMARK_N_OMIT) {
401 durations.add(finishTime - startTime);
402 }
403 }
404 printBenchmarkResult(length, durations, true);
405 }
406
407 private static void printBenchmarkResult(int totalSize,
408 List<Long> durationsInNanoSed, boolean isDecompression) {
409 long meanTime = 0;
410 for (long time : durationsInNanoSed) {
411 meanTime += time;
412 }
413 meanTime /= durationsInNanoSed.size();
414
415 long standardDev = 0;
416 for (long time : durationsInNanoSed) {
417 standardDev += (time - meanTime) * (time - meanTime);
418 }
419 standardDev = (long) Math.sqrt(standardDev / durationsInNanoSed.size());
420
421 final double million = 1000.0 * 1000.0 * 1000.0;
422 double mbPerSec = (totalSize * million) / (1024.0 * 1024.0 * meanTime);
423 double mbPerSecDev = (totalSize * million) /
424 (1024.0 * 1024.0 * (meanTime - standardDev));
425
426 System.out.println(String.format(
427 " %s performance:%s %6.2f MB/s (+/- %.2f MB/s)",
428 isDecompression ? "Decompression" : "Compression",
429 isDecompression ? "" : " ",
430 mbPerSec, mbPerSecDev - mbPerSec));
431 }
432
433
434
435
436 public void displayStatistics() {
437 int totalLength = totalPrefixLength + totalKeyLength + totalValueLength;
438 if (compressor != null) {
439 compressor.reset();
440 }
441
442 for(EncodedDataBlock codec : codecs) {
443 System.out.println(codec.toString());
444 int saved = totalKeyLength + totalPrefixLength + totalValueLength
445 - codec.getSize();
446 System.out.println(
447 String.format(" Saved bytes: %8d", saved));
448 double keyRatio = (saved * 100.0) / (totalPrefixLength + totalKeyLength);
449 double allRatio = (saved * 100.0) / totalLength;
450 System.out.println(
451 String.format(" Key compression ratio: %.2f %%", keyRatio));
452 System.out.println(
453 String.format(" All compression ratio: %.2f %%", allRatio));
454
455 String compressedSizeCaption =
456 String.format(" %s compressed size: ",
457 compressionAlgorithmName.toUpperCase());
458 String compressOnlyRatioCaption =
459 String.format(" %s compression ratio: ",
460 compressionAlgorithmName.toUpperCase());
461
462 if (compressor != null) {
463 int compressedSize = codec.checkCompressedSize(compressor);
464 System.out.println(compressedSizeCaption +
465 String.format("%8d", compressedSize));
466 double compressOnlyRatio =
467 100.0 * (1.0 - compressedSize / (0.0 + totalLength));
468 System.out.println(compressOnlyRatioCaption
469 + String.format("%.2f %%", compressOnlyRatio));
470 } else {
471 System.out.println(compressedSizeCaption + "N/A");
472 System.out.println(compressOnlyRatioCaption + "N/A");
473 }
474 }
475
476 System.out.println(
477 String.format("Total KV prefix length: %8d", totalPrefixLength));
478 System.out.println(
479 String.format("Total key length: %8d", totalKeyLength));
480 System.out.println(
481 String.format("Total key redundancy: %8d",
482 totalKeyRedundancyLength));
483 System.out.println(
484 String.format("Total value length: %8d", totalValueLength));
485 }
486
487
488
489
490
491
492
493
494
495
496 public static void testCodecs(Configuration conf, int kvLimit,
497 String hfilePath, String compressionName, boolean doBenchmark,
498 boolean doVerify) throws IOException {
499
500 Path path = new Path(hfilePath);
501 CacheConfig cacheConf = new CacheConfig(conf);
502 FileSystem fs = FileSystem.get(conf);
503 StoreFile hsf = new StoreFile(fs, path, conf, cacheConf,
504 StoreFile.BloomType.NONE, NoOpDataBlockEncoder.INSTANCE);
505
506 StoreFile.Reader reader = hsf.createReader();
507 reader.loadFileInfo();
508 KeyValueScanner scanner = reader.getStoreFileScanner(true, true);
509
510
511 DataBlockEncodingTool comp = new DataBlockEncodingTool(compressionName);
512 comp.checkStatistics(scanner, kvLimit);
513 if (doVerify) {
514 comp.verifyCodecs(scanner, kvLimit);
515 }
516 if (doBenchmark) {
517 comp.benchmarkCodecs();
518 }
519 comp.displayStatistics();
520
521
522 scanner.close();
523 reader.close(cacheConf.shouldEvictOnClose());
524 }
525
526 private static void printUsage(Options options) {
527 System.err.println("Usage:");
528 System.err.println(String.format("./hbase %s <options>",
529 DataBlockEncodingTool.class.getName()));
530 System.err.println("Options:");
531 for (Object it : options.getOptions()) {
532 Option opt = (Option) it;
533 if (opt.hasArg()) {
534 System.err.println(String.format("-%s %s: %s", opt.getOpt(),
535 opt.getArgName(), opt.getDescription()));
536 } else {
537 System.err.println(String.format("-%s: %s", opt.getOpt(),
538 opt.getDescription()));
539 }
540 }
541 }
542
543
544
545
546
547
548 public static void main(final String[] args) throws IOException {
549
550 Options options = new Options();
551 options.addOption("f", true, "HFile to analyse (REQUIRED)");
552 options.getOption("f").setArgName("FILENAME");
553 options.addOption("n", true,
554 "Limit number of KeyValue which will be analysed");
555 options.getOption("n").setArgName("NUMBER");
556 options.addOption("b", false, "Measure read throughput");
557 options.addOption("c", false, "Omit corectness tests.");
558 options.addOption("a", true,
559 "What kind of compression algorithm use for comparison.");
560
561
562 CommandLineParser parser = new PosixParser();
563 CommandLine cmd = null;
564 try {
565 cmd = parser.parse(options, args);
566 } catch (ParseException e) {
567 System.err.println("Could not parse arguments!");
568 System.exit(-1);
569 return;
570 }
571
572 int kvLimit = Integer.MAX_VALUE;
573 if (cmd.hasOption("n")) {
574 kvLimit = Integer.parseInt(cmd.getOptionValue("n"));
575 }
576
577
578 if (!cmd.hasOption("f")) {
579 System.err.println("ERROR: Filename is required!");
580 printUsage(options);
581 System.exit(-1);
582 }
583
584 String pathName = cmd.getOptionValue("f");
585 String compressionName = DEFAULT_COMPRESSION.getName();
586 if (cmd.hasOption("a")) {
587 compressionName = cmd.getOptionValue("a").toLowerCase();
588 }
589 boolean doBenchmark = cmd.hasOption("b");
590 boolean doVerify = !cmd.hasOption("c");
591
592 final Configuration conf = HBaseConfiguration.create();
593 try {
594 testCodecs(conf, kvLimit, pathName, compressionName, doBenchmark,
595 doVerify);
596 } finally {
597 (new CacheConfig(conf)).getBlockCache().shutdown();
598 }
599 }
600
601 }