001/*
002 *
003 * Licensed to the Apache Software Foundation (ASF) under one
004 * or more contributor license agreements.  See the NOTICE file
005 * distributed with this work for additional information
006 * regarding copyright ownership.  The ASF licenses this file
007 * to you under the Apache License, Version 2.0 (the
008 * "License"); you may not use this file except in compliance
009 * with the License.  You may obtain a copy of the License at
010 *
011 *     http://www.apache.org/licenses/LICENSE-2.0
012 *
013 * Unless required by applicable law or agreed to in writing, software
014 * distributed under the License is distributed on an "AS IS" BASIS,
015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
016 * See the License for the specific language governing permissions and
017 * limitations under the License.
018 */
019package org.apache.hadoop.hbase.io.hfile;
020
021import static com.codahale.metrics.MetricRegistry.name;
022
023import java.io.ByteArrayOutputStream;
024import java.io.DataInput;
025import java.io.IOException;
026import java.io.PrintStream;
027import java.text.DateFormat;
028import java.util.ArrayList;
029import java.util.HashMap;
030import java.util.Iterator;
031import java.util.LinkedHashSet;
032import java.util.List;
033import java.util.Locale;
034import java.util.Map;
035import java.util.Set;
036import java.util.SortedMap;
037import java.util.TimeZone;
038import java.util.concurrent.TimeUnit;
039
040import org.apache.commons.lang3.StringUtils;
041import org.apache.hadoop.conf.Configuration;
042import org.apache.hadoop.conf.Configured;
043import org.apache.hadoop.fs.FileSystem;
044import org.apache.hadoop.fs.Path;
045import org.apache.hadoop.hbase.Cell;
046import org.apache.hadoop.hbase.CellComparator;
047import org.apache.hadoop.hbase.CellUtil;
048import org.apache.hadoop.hbase.HBaseConfiguration;
049import org.apache.hadoop.hbase.HBaseInterfaceAudience;
050import org.apache.hadoop.hbase.HConstants;
051import org.apache.hadoop.hbase.HRegionInfo;
052import org.apache.hadoop.hbase.KeyValue;
053import org.apache.hadoop.hbase.KeyValueUtil;
054import org.apache.hadoop.hbase.PrivateCellUtil;
055import org.apache.hadoop.hbase.TableName;
056import org.apache.hadoop.hbase.Tag;
057import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
058import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
059import org.apache.hadoop.hbase.mob.MobUtils;
060import org.apache.hadoop.hbase.regionserver.HStoreFile;
061import org.apache.hadoop.hbase.regionserver.TimeRangeTracker;
062import org.apache.hadoop.hbase.util.BloomFilter;
063import org.apache.hadoop.hbase.util.BloomFilterFactory;
064import org.apache.hadoop.hbase.util.BloomFilterUtil;
065import org.apache.hadoop.hbase.util.Bytes;
066import org.apache.hadoop.hbase.util.FSUtils;
067import org.apache.hadoop.hbase.util.HFileArchiveUtil;
068import org.apache.hadoop.util.Tool;
069import org.apache.hadoop.util.ToolRunner;
070import org.apache.yetus.audience.InterfaceAudience;
071import org.apache.yetus.audience.InterfaceStability;
072import org.slf4j.Logger;
073import org.slf4j.LoggerFactory;
074
075import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
076import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLineParser;
077import org.apache.hbase.thirdparty.org.apache.commons.cli.HelpFormatter;
078import org.apache.hbase.thirdparty.org.apache.commons.cli.Option;
079import org.apache.hbase.thirdparty.org.apache.commons.cli.OptionGroup;
080import org.apache.hbase.thirdparty.org.apache.commons.cli.Options;
081import org.apache.hbase.thirdparty.org.apache.commons.cli.ParseException;
082import org.apache.hbase.thirdparty.org.apache.commons.cli.PosixParser;
083
084import com.codahale.metrics.ConsoleReporter;
085import com.codahale.metrics.Counter;
086import com.codahale.metrics.Gauge;
087import com.codahale.metrics.Histogram;
088import com.codahale.metrics.Meter;
089import com.codahale.metrics.MetricFilter;
090import com.codahale.metrics.MetricRegistry;
091import com.codahale.metrics.ScheduledReporter;
092import com.codahale.metrics.Snapshot;
093import com.codahale.metrics.Timer;
094
095/**
096 * Implements pretty-printing functionality for {@link HFile}s.
097 */
098@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
099@InterfaceStability.Evolving
100public class HFilePrettyPrinter extends Configured implements Tool {
101
102  private static final Logger LOG = LoggerFactory.getLogger(HFilePrettyPrinter.class);
103
104  private Options options = new Options();
105
106  private boolean verbose;
107  private boolean printValue;
108  private boolean printKey;
109  private boolean shouldPrintMeta;
110  private boolean printBlockIndex;
111  private boolean printBlockHeaders;
112  private boolean printStats;
113  private boolean checkRow;
114  private boolean checkFamily;
115  private boolean isSeekToRow = false;
116  private boolean checkMobIntegrity = false;
117  private Map<String, List<Path>> mobFileLocations;
118  private static final int FOUND_MOB_FILES_CACHE_CAPACITY = 50;
119  private static final int MISSING_MOB_FILES_CACHE_CAPACITY = 20;
120  private PrintStream out = System.out;
121  private PrintStream err = System.err;
122
123  /**
124   * The row which the user wants to specify and print all the KeyValues for.
125   */
126  private byte[] row = null;
127
128  private List<Path> files = new ArrayList<>();
129  private int count;
130
131  private static final String FOUR_SPACES = "    ";
132
133  public HFilePrettyPrinter() {
134    super();
135    init();
136  }
137
138  public HFilePrettyPrinter(Configuration conf) {
139    super(conf);
140    init();
141  }
142
143  private void init() {
144    options.addOption("v", "verbose", false,
145        "Verbose output; emits file and meta data delimiters");
146    options.addOption("p", "printkv", false, "Print key/value pairs");
147    options.addOption("e", "printkey", false, "Print keys");
148    options.addOption("m", "printmeta", false, "Print meta data of file");
149    options.addOption("b", "printblocks", false, "Print block index meta data");
150    options.addOption("h", "printblockheaders", false, "Print block headers for each block.");
151    options.addOption("k", "checkrow", false,
152        "Enable row order check; looks for out-of-order keys");
153    options.addOption("a", "checkfamily", false, "Enable family check");
154    options.addOption("w", "seekToRow", true,
155      "Seek to this row and print all the kvs for this row only");
156    options.addOption("s", "stats", false, "Print statistics");
157    options.addOption("i", "checkMobIntegrity", false,
158      "Print all cells whose mob files are missing");
159
160    OptionGroup files = new OptionGroup();
161    files.addOption(new Option("f", "file", true,
162      "File to scan. Pass full-path; e.g. hdfs://a:9000/hbase/hbase:meta/12/34"));
163    files.addOption(new Option("r", "region", true,
164      "Region to scan. Pass region name; e.g. 'hbase:meta,,1'"));
165    options.addOptionGroup(files);
166  }
167
168  public void setPrintStreams(PrintStream out, PrintStream err) {
169    this.out = out;
170    this.err = err;
171  }
172
173  public boolean parseOptions(String args[]) throws ParseException,
174      IOException {
175    if (args.length == 0) {
176      HelpFormatter formatter = new HelpFormatter();
177      formatter.printHelp("hfile", options, true);
178      return false;
179    }
180    CommandLineParser parser = new PosixParser();
181    CommandLine cmd = parser.parse(options, args);
182
183    verbose = cmd.hasOption("v");
184    printValue = cmd.hasOption("p");
185    printKey = cmd.hasOption("e") || printValue;
186    shouldPrintMeta = cmd.hasOption("m");
187    printBlockIndex = cmd.hasOption("b");
188    printBlockHeaders = cmd.hasOption("h");
189    printStats = cmd.hasOption("s");
190    checkRow = cmd.hasOption("k");
191    checkFamily = cmd.hasOption("a");
192    checkMobIntegrity = cmd.hasOption("i");
193
194    if (cmd.hasOption("f")) {
195      files.add(new Path(cmd.getOptionValue("f")));
196    }
197
198    if (cmd.hasOption("w")) {
199      String key = cmd.getOptionValue("w");
200      if (key != null && key.length() != 0) {
201        row = Bytes.toBytesBinary(key);
202        isSeekToRow = true;
203      } else {
204        err.println("Invalid row is specified.");
205        System.exit(-1);
206      }
207    }
208
209    if (cmd.hasOption("r")) {
210      String regionName = cmd.getOptionValue("r");
211      byte[] rn = Bytes.toBytes(regionName);
212      byte[][] hri = HRegionInfo.parseRegionName(rn);
213      Path rootDir = FSUtils.getRootDir(getConf());
214      Path tableDir = FSUtils.getTableDir(rootDir, TableName.valueOf(hri[0]));
215      String enc = HRegionInfo.encodeRegionName(rn);
216      Path regionDir = new Path(tableDir, enc);
217      if (verbose)
218        out.println("region dir -> " + regionDir);
219      List<Path> regionFiles = HFile.getStoreFiles(FileSystem.get(getConf()),
220          regionDir);
221      if (verbose)
222        out.println("Number of region files found -> "
223            + regionFiles.size());
224      if (verbose) {
225        int i = 1;
226        for (Path p : regionFiles) {
227          if (verbose)
228            out.println("Found file[" + i++ + "] -> " + p);
229        }
230      }
231      files.addAll(regionFiles);
232    }
233
234    if(checkMobIntegrity) {
235      if (verbose) {
236        System.out.println("checkMobIntegrity is enabled");
237      }
238      mobFileLocations = new HashMap<>();
239    }
240
241    cmd.getArgList().forEach((file) -> files.add(new Path(file)));
242
243    return true;
244  }
245
246  /**
247   * Runs the command-line pretty-printer, and returns the desired command
248   * exit code (zero for success, non-zero for failure).
249   */
250  @Override
251  public int run(String[] args) {
252    if (getConf() == null) {
253      throw new RuntimeException("A Configuration instance must be provided.");
254    }
255    try {
256      FSUtils.setFsDefault(getConf(), FSUtils.getRootDir(getConf()));
257      if (!parseOptions(args))
258        return 1;
259    } catch (IOException ex) {
260      LOG.error("Error parsing command-line options", ex);
261      return 1;
262    } catch (ParseException ex) {
263      LOG.error("Error parsing command-line options", ex);
264      return 1;
265    }
266
267    // iterate over all files found
268    for (Path fileName : files) {
269      try {
270        int exitCode = processFile(fileName, false);
271        if (exitCode != 0) {
272          return exitCode;
273        }
274      } catch (IOException ex) {
275        LOG.error("Error reading " + fileName, ex);
276        return -2;
277      }
278    }
279
280    if (verbose || printKey) {
281      out.println("Scanned kv count -> " + count);
282    }
283
284    return 0;
285  }
286
287  // HBASE-22561 introduces boolean checkRootDir for WebUI specificly
288  public int processFile(Path file, boolean checkRootDir) throws IOException {
289    if (verbose) {
290      out.println("Scanning -> " + file);
291    }
292
293    if (checkRootDir) {
294      Path rootPath = FSUtils.getRootDir(getConf());
295      String rootString = rootPath + rootPath.SEPARATOR;
296      if (!file.toString().startsWith(rootString)) {
297        // First we see if fully-qualified URI matches the root dir. It might
298        // also be an absolute path in the same filesystem, so we prepend the FS
299        // of the root dir and see if that fully-qualified URI matches.
300        FileSystem rootFS = rootPath.getFileSystem(getConf());
301        String qualifiedFile = rootFS.getUri().toString() + file.toString();
302        if (!qualifiedFile.startsWith(rootString)) {
303          err.println(
304            "ERROR, file (" + file + ") is not in HBase's root directory (" + rootString + ")");
305          return -2;
306        }
307      }
308    }
309
310    FileSystem fs = file.getFileSystem(getConf());
311    if (!fs.exists(file)) {
312      err.println("ERROR, file doesnt exist: " + file);
313      return -2;
314    }
315
316    HFile.Reader reader = HFile.createReader(fs, file, CacheConfig.DISABLED, true, getConf());
317
318    Map<byte[], byte[]> fileInfo = reader.loadFileInfo();
319
320    KeyValueStatsCollector fileStats = null;
321
322    if (verbose || printKey || checkRow || checkFamily || printStats || checkMobIntegrity) {
323      // scan over file and read key/value's and check if requested
324      HFileScanner scanner = reader.getScanner(false, false, false);
325      fileStats = new KeyValueStatsCollector();
326      boolean shouldScanKeysValues = false;
327      if (this.isSeekToRow) {
328        // seek to the first kv on this row
329        shouldScanKeysValues =
330          (scanner.seekTo(PrivateCellUtil.createFirstOnRow(this.row)) != -1);
331      } else {
332        shouldScanKeysValues = scanner.seekTo();
333      }
334      if (shouldScanKeysValues)
335        scanKeysValues(file, fileStats, scanner, row);
336    }
337
338    // print meta data
339    if (shouldPrintMeta) {
340      printMeta(reader, fileInfo);
341    }
342
343    if (printBlockIndex) {
344      out.println("Block Index:");
345      out.println(reader.getDataBlockIndexReader());
346    }
347
348    if (printBlockHeaders) {
349      out.println("Block Headers:");
350      /*
351       * TODO: this same/similar block iteration logic is used in HFileBlock#blockRange and
352       * TestLazyDataBlockDecompression. Refactor?
353       */
354      FSDataInputStreamWrapper fsdis = new FSDataInputStreamWrapper(fs, file);
355      long fileSize = fs.getFileStatus(file).getLen();
356      FixedFileTrailer trailer =
357        FixedFileTrailer.readFromStream(fsdis.getStream(false), fileSize);
358      long offset = trailer.getFirstDataBlockOffset(),
359        max = trailer.getLastDataBlockOffset();
360      HFileBlock block;
361      while (offset <= max) {
362        block = reader.readBlock(offset, -1, /* cacheBlock */ false, /* pread */ false,
363          /* isCompaction */ false, /* updateCacheMetrics */ false, null, null);
364        offset += block.getOnDiskSizeWithHeader();
365        out.println(block);
366      }
367    }
368
369    if (printStats) {
370      fileStats.finish();
371      out.println("Stats:\n" + fileStats);
372    }
373
374    reader.close();
375    return 0;
376  }
377
378  private void scanKeysValues(Path file, KeyValueStatsCollector fileStats,
379      HFileScanner scanner,  byte[] row) throws IOException {
380    Cell pCell = null;
381    FileSystem fs = FileSystem.get(getConf());
382    Set<String> foundMobFiles = new LinkedHashSet<>(FOUND_MOB_FILES_CACHE_CAPACITY);
383    Set<String> missingMobFiles = new LinkedHashSet<>(MISSING_MOB_FILES_CACHE_CAPACITY);
384    do {
385      Cell cell = scanner.getCell();
386      if (row != null && row.length != 0) {
387        int result = CellComparator.getInstance().compareRows(cell, row, 0, row.length);
388        if (result > 0) {
389          break;
390        } else if (result < 0) {
391          continue;
392        }
393      }
394      // collect stats
395      if (printStats) {
396        fileStats.collect(cell);
397      }
398      // dump key value
399      if (printKey) {
400        out.print("K: " + cell);
401        if (printValue) {
402          out.print(" V: "
403              + Bytes.toStringBinary(cell.getValueArray(), cell.getValueOffset(),
404                  cell.getValueLength()));
405          int i = 0;
406          List<Tag> tags = PrivateCellUtil.getTags(cell);
407          for (Tag tag : tags) {
408            out.print(String.format(" T[%d]: %s", i++, tag.toString()));
409          }
410        }
411        out.println();
412      }
413      // check if rows are in order
414      if (checkRow && pCell != null) {
415        if (CellComparator.getInstance().compareRows(pCell, cell) > 0) {
416          err.println("WARNING, previous row is greater then"
417              + " current row\n\tfilename -> " + file + "\n\tprevious -> "
418              + CellUtil.getCellKeyAsString(pCell) + "\n\tcurrent  -> "
419              + CellUtil.getCellKeyAsString(cell));
420        }
421      }
422      // check if families are consistent
423      if (checkFamily) {
424        String fam = Bytes.toString(cell.getFamilyArray(), cell.getFamilyOffset(),
425            cell.getFamilyLength());
426        if (!file.toString().contains(fam)) {
427          err.println("WARNING, filename does not match kv family,"
428              + "\n\tfilename -> " + file + "\n\tkeyvalue -> "
429              + CellUtil.getCellKeyAsString(cell));
430        }
431        if (pCell != null && CellComparator.getInstance().compareFamilies(pCell, cell) != 0) {
432          err.println("WARNING, previous kv has different family"
433              + " compared to current key\n\tfilename -> " + file
434              + "\n\tprevious -> " + CellUtil.getCellKeyAsString(pCell)
435              + "\n\tcurrent  -> " + CellUtil.getCellKeyAsString(cell));
436        }
437      }
438      // check if mob files are missing.
439      if (checkMobIntegrity && MobUtils.isMobReferenceCell(cell)) {
440        Tag tnTag = MobUtils.getTableNameTag(cell);
441        if (tnTag == null) {
442          System.err.println("ERROR, wrong tag format in mob reference cell "
443            + CellUtil.getCellKeyAsString(cell));
444        } else if (!MobUtils.hasValidMobRefCellValue(cell)) {
445          System.err.println("ERROR, wrong value format in mob reference cell "
446            + CellUtil.getCellKeyAsString(cell));
447        } else {
448          TableName tn = TableName.valueOf(Tag.cloneValue(tnTag));
449          String mobFileName = MobUtils.getMobFileName(cell);
450          boolean exist = mobFileExists(fs, tn, mobFileName,
451            Bytes.toString(CellUtil.cloneFamily(cell)), foundMobFiles, missingMobFiles);
452          if (!exist) {
453            // report error
454            System.err.println("ERROR, the mob file [" + mobFileName
455              + "] is missing referenced by cell " + CellUtil.getCellKeyAsString(cell));
456          }
457        }
458      }
459      pCell = cell;
460      ++count;
461    } while (scanner.next());
462  }
463
464  /**
465   * Checks whether the referenced mob file exists.
466   */
467  private boolean mobFileExists(FileSystem fs, TableName tn, String mobFileName, String family,
468    Set<String> foundMobFiles, Set<String> missingMobFiles) throws IOException {
469    if (foundMobFiles.contains(mobFileName)) {
470      return true;
471    }
472    if (missingMobFiles.contains(mobFileName)) {
473      return false;
474    }
475    String tableName = tn.getNameAsString();
476    List<Path> locations = mobFileLocations.get(tableName);
477    if (locations == null) {
478      locations = new ArrayList<>(2);
479      locations.add(MobUtils.getMobFamilyPath(getConf(), tn, family));
480      locations.add(HFileArchiveUtil.getStoreArchivePath(getConf(), tn,
481        MobUtils.getMobRegionInfo(tn).getEncodedName(), family));
482      mobFileLocations.put(tn.getNameAsString(), locations);
483    }
484    boolean exist = false;
485    for (Path location : locations) {
486      Path mobFilePath = new Path(location, mobFileName);
487      if (fs.exists(mobFilePath)) {
488        exist = true;
489        break;
490      }
491    }
492    if (exist) {
493      evictMobFilesIfNecessary(foundMobFiles, FOUND_MOB_FILES_CACHE_CAPACITY);
494      foundMobFiles.add(mobFileName);
495    } else {
496      evictMobFilesIfNecessary(missingMobFiles, MISSING_MOB_FILES_CACHE_CAPACITY);
497      missingMobFiles.add(mobFileName);
498    }
499    return exist;
500  }
501
502  /**
503   * Evicts the cached mob files if the set is larger than the limit.
504   */
505  private void evictMobFilesIfNecessary(Set<String> mobFileNames, int limit) {
506    if (mobFileNames.size() < limit) {
507      return;
508    }
509    int index = 0;
510    int evict = limit / 2;
511    Iterator<String> fileNamesItr = mobFileNames.iterator();
512    while (index < evict && fileNamesItr.hasNext()) {
513      fileNamesItr.next();
514      fileNamesItr.remove();
515      index++;
516    }
517  }
518
519  /**
520   * Format a string of the form "k1=v1, k2=v2, ..." into separate lines
521   * with a four-space indentation.
522   */
523  private static String asSeparateLines(String keyValueStr) {
524    return keyValueStr.replaceAll(", ([a-zA-Z]+=)",
525                                  ",\n" + FOUR_SPACES + "$1");
526  }
527
528  private void printMeta(HFile.Reader reader, Map<byte[], byte[]> fileInfo)
529      throws IOException {
530    out.println("Block index size as per heapsize: "
531        + reader.indexSize());
532    out.println(asSeparateLines(reader.toString()));
533    out.println("Trailer:\n    "
534        + asSeparateLines(reader.getTrailer().toString()));
535    out.println("Fileinfo:");
536    for (Map.Entry<byte[], byte[]> e : fileInfo.entrySet()) {
537      out.print(FOUR_SPACES + Bytes.toString(e.getKey()) + " = ");
538      if (Bytes.equals(e.getKey(), HStoreFile.MAX_SEQ_ID_KEY)
539          || Bytes.equals(e.getKey(), HStoreFile.DELETE_FAMILY_COUNT)
540          || Bytes.equals(e.getKey(), HStoreFile.EARLIEST_PUT_TS)
541          || Bytes.equals(e.getKey(), HFileWriterImpl.MAX_MEMSTORE_TS_KEY)
542          || Bytes.equals(e.getKey(), FileInfo.CREATE_TIME_TS)
543          || Bytes.equals(e.getKey(), HStoreFile.BULKLOAD_TIME_KEY)) {
544        out.println(Bytes.toLong(e.getValue()));
545      } else if (Bytes.equals(e.getKey(), HStoreFile.TIMERANGE_KEY)) {
546        TimeRangeTracker timeRangeTracker = TimeRangeTracker.parseFrom(e.getValue());
547        out.println(timeRangeTracker.getMin() + "...." + timeRangeTracker.getMax());
548      } else if (Bytes.equals(e.getKey(), FileInfo.AVG_KEY_LEN)
549          || Bytes.equals(e.getKey(), FileInfo.AVG_VALUE_LEN)
550          || Bytes.equals(e.getKey(), HFileWriterImpl.KEY_VALUE_VERSION)
551          || Bytes.equals(e.getKey(), FileInfo.MAX_TAGS_LEN)) {
552        out.println(Bytes.toInt(e.getValue()));
553      } else if (Bytes.equals(e.getKey(), HStoreFile.MAJOR_COMPACTION_KEY)
554          || Bytes.equals(e.getKey(), FileInfo.TAGS_COMPRESSED)
555          || Bytes.equals(e.getKey(), HStoreFile.EXCLUDE_FROM_MINOR_COMPACTION_KEY)) {
556        out.println(Bytes.toBoolean(e.getValue()));
557      } else if (Bytes.equals(e.getKey(), FileInfo.LASTKEY)) {
558        out.println(new KeyValue.KeyOnlyKeyValue(e.getValue()).toString());
559      } else {
560        out.println(Bytes.toStringBinary(e.getValue()));
561      }
562    }
563
564    try {
565      out.println("Mid-key: " + reader.midKey().map(CellUtil::getCellKeyAsString));
566    } catch (Exception e) {
567      out.println ("Unable to retrieve the midkey");
568    }
569
570    // Printing general bloom information
571    DataInput bloomMeta = reader.getGeneralBloomFilterMetadata();
572    BloomFilter bloomFilter = null;
573    if (bloomMeta != null)
574      bloomFilter = BloomFilterFactory.createFromMeta(bloomMeta, reader);
575
576    out.println("Bloom filter:");
577    if (bloomFilter != null) {
578      out.println(FOUR_SPACES + bloomFilter.toString().replaceAll(
579          BloomFilterUtil.STATS_RECORD_SEP, "\n" + FOUR_SPACES));
580    } else {
581      out.println(FOUR_SPACES + "Not present");
582    }
583
584    // Printing delete bloom information
585    bloomMeta = reader.getDeleteBloomFilterMetadata();
586    bloomFilter = null;
587    if (bloomMeta != null)
588      bloomFilter = BloomFilterFactory.createFromMeta(bloomMeta, reader);
589
590    out.println("Delete Family Bloom filter:");
591    if (bloomFilter != null) {
592      out.println(FOUR_SPACES
593          + bloomFilter.toString().replaceAll(BloomFilterUtil.STATS_RECORD_SEP,
594              "\n" + FOUR_SPACES));
595    } else {
596      out.println(FOUR_SPACES + "Not present");
597    }
598  }
599
600  private static class KeyValueStatsCollector {
601    private final MetricRegistry metricsRegistry = new MetricRegistry();
602    private final ByteArrayOutputStream metricsOutput = new ByteArrayOutputStream();
603    private final SimpleReporter simpleReporter = SimpleReporter.forRegistry(metricsRegistry).
604        outputTo(new PrintStream(metricsOutput)).filter(MetricFilter.ALL).build();
605
606    Histogram keyLen = metricsRegistry.histogram(name(HFilePrettyPrinter.class, "Key length"));
607    Histogram valLen = metricsRegistry.histogram(name(HFilePrettyPrinter.class, "Val length"));
608    Histogram rowSizeBytes = metricsRegistry.histogram(
609      name(HFilePrettyPrinter.class, "Row size (bytes)"));
610    Histogram rowSizeCols = metricsRegistry.histogram(
611      name(HFilePrettyPrinter.class, "Row size (columns)"));
612
613    long curRowBytes = 0;
614    long curRowCols = 0;
615
616    byte[] biggestRow = null;
617
618    private Cell prevCell = null;
619    private long maxRowBytes = 0;
620    private long curRowKeyLength;
621
622    public void collect(Cell cell) {
623      valLen.update(cell.getValueLength());
624      if (prevCell != null &&
625          CellComparator.getInstance().compareRows(prevCell, cell) != 0) {
626        // new row
627        collectRow();
628      }
629      curRowBytes += cell.getSerializedSize();
630      curRowKeyLength = KeyValueUtil.keyLength(cell);
631      curRowCols++;
632      prevCell = cell;
633    }
634
635    private void collectRow() {
636      rowSizeBytes.update(curRowBytes);
637      rowSizeCols.update(curRowCols);
638      keyLen.update(curRowKeyLength);
639
640      if (curRowBytes > maxRowBytes && prevCell != null) {
641        biggestRow = CellUtil.cloneRow(prevCell);
642        maxRowBytes = curRowBytes;
643      }
644
645      curRowBytes = 0;
646      curRowCols = 0;
647    }
648
649    public void finish() {
650      if (curRowCols > 0) {
651        collectRow();
652      }
653    }
654
655    @Override
656    public String toString() {
657      if (prevCell == null)
658        return "no data available for statistics";
659
660      // Dump the metrics to the output stream
661      simpleReporter.stop();
662      simpleReporter.report();
663
664      return
665              metricsOutput.toString() +
666                      "Key of biggest row: " + Bytes.toStringBinary(biggestRow);
667    }
668  }
669
670  /**
671   * Almost identical to ConsoleReporter, but extending ScheduledReporter,
672   * as extending ConsoleReporter in this version of dropwizard is now too much trouble.
673   */
674  private static class SimpleReporter extends ScheduledReporter {
675    /**
676     * Returns a new {@link Builder} for {@link ConsoleReporter}.
677     *
678     * @param registry the registry to report
679     * @return a {@link Builder} instance for a {@link ConsoleReporter}
680     */
681    public static Builder forRegistry(MetricRegistry registry) {
682      return new Builder(registry);
683    }
684
685    /**
686     * A builder for {@link SimpleReporter} instances. Defaults to using the default locale and
687     * time zone, writing to {@code System.out}, converting rates to events/second, converting
688     * durations to milliseconds, and not filtering metrics.
689     */
690    public static class Builder {
691      private final MetricRegistry registry;
692      private PrintStream output;
693      private Locale locale;
694      private TimeZone timeZone;
695      private TimeUnit rateUnit;
696      private TimeUnit durationUnit;
697      private MetricFilter filter;
698
699      private Builder(MetricRegistry registry) {
700        this.registry = registry;
701        this.output = System.out;
702        this.locale = Locale.getDefault();
703        this.timeZone = TimeZone.getDefault();
704        this.rateUnit = TimeUnit.SECONDS;
705        this.durationUnit = TimeUnit.MILLISECONDS;
706        this.filter = MetricFilter.ALL;
707      }
708
709      /**
710       * Write to the given {@link PrintStream}.
711       *
712       * @param output a {@link PrintStream} instance.
713       * @return {@code this}
714       */
715      public Builder outputTo(PrintStream output) {
716        this.output = output;
717        return this;
718      }
719
720      /**
721       * Only report metrics which match the given filter.
722       *
723       * @param filter a {@link MetricFilter}
724       * @return {@code this}
725       */
726      public Builder filter(MetricFilter filter) {
727        this.filter = filter;
728        return this;
729      }
730
731      /**
732       * Builds a {@link ConsoleReporter} with the given properties.
733       *
734       * @return a {@link ConsoleReporter}
735       */
736      public SimpleReporter build() {
737        return new SimpleReporter(registry,
738            output,
739            locale,
740            timeZone,
741            rateUnit,
742            durationUnit,
743            filter);
744      }
745    }
746
747    private final PrintStream output;
748    private final Locale locale;
749    private final DateFormat dateFormat;
750
751    private SimpleReporter(MetricRegistry registry,
752                            PrintStream output,
753                            Locale locale,
754                            TimeZone timeZone,
755                            TimeUnit rateUnit,
756                            TimeUnit durationUnit,
757                            MetricFilter filter) {
758      super(registry, "simple-reporter", filter, rateUnit, durationUnit);
759      this.output = output;
760      this.locale = locale;
761
762      this.dateFormat = DateFormat.getDateTimeInstance(DateFormat.SHORT,
763          DateFormat.MEDIUM,
764          locale);
765      dateFormat.setTimeZone(timeZone);
766    }
767
768    @Override
769    public void report(SortedMap<String, Gauge> gauges,
770                       SortedMap<String, Counter> counters,
771                       SortedMap<String, Histogram> histograms,
772                       SortedMap<String, Meter> meters,
773                       SortedMap<String, Timer> timers) {
774      // we know we only have histograms
775      if (!histograms.isEmpty()) {
776        for (Map.Entry<String, Histogram> entry : histograms.entrySet()) {
777          output.print("   " + StringUtils.substringAfterLast(entry.getKey(), "."));
778          output.println(':');
779          printHistogram(entry.getValue());
780        }
781        output.println();
782      }
783
784      output.println();
785      output.flush();
786    }
787
788    private void printHistogram(Histogram histogram) {
789      Snapshot snapshot = histogram.getSnapshot();
790      output.printf(locale, "               min = %d%n", snapshot.getMin());
791      output.printf(locale, "               max = %d%n", snapshot.getMax());
792      output.printf(locale, "              mean = %2.2f%n", snapshot.getMean());
793      output.printf(locale, "            stddev = %2.2f%n", snapshot.getStdDev());
794      output.printf(locale, "            median = %2.2f%n", snapshot.getMedian());
795      output.printf(locale, "              75%% <= %2.2f%n", snapshot.get75thPercentile());
796      output.printf(locale, "              95%% <= %2.2f%n", snapshot.get95thPercentile());
797      output.printf(locale, "              98%% <= %2.2f%n", snapshot.get98thPercentile());
798      output.printf(locale, "              99%% <= %2.2f%n", snapshot.get99thPercentile());
799      output.printf(locale, "            99.9%% <= %2.2f%n", snapshot.get999thPercentile());
800      output.printf(locale, "             count = %d%n", histogram.getCount());
801    }
802  }
803
804  public static void main(String[] args) throws Exception {
805    Configuration conf = HBaseConfiguration.create();
806    // no need for a block cache
807    conf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0);
808    int ret = ToolRunner.run(conf, new HFilePrettyPrinter(), args);
809    System.exit(ret);
810  }
811}