You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2013/08/18 20:25:38 UTC

svn commit: r1515195 - /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java

Author: apurtell
Date: Sun Aug 18 18:25:38 2013
New Revision: 1515195

URL: http://svn.apache.org/r1515195
Log:
HBASE-9243. Add more useful statistics in the HFile tool (Alexandre Normand)

Modified:
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java?rev=1515195&r1=1515194&r2=1515195&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java Sun Aug 18 18:25:38 2013
@@ -19,11 +19,18 @@
  */
 package org.apache.hadoop.hbase.io.hfile;
 
+import java.io.ByteArrayOutputStream;
 import java.io.DataInput;
 import java.io.IOException;
+import java.io.PrintStream;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
+import java.util.SortedMap;
+
+import com.yammer.metrics.core.*;
+import com.yammer.metrics.reporting.ConsoleReporter;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
@@ -368,32 +375,14 @@ public class HFilePrettyPrinter {
     }
   }
 
-  private static class LongStats {
-    private long min = Long.MAX_VALUE;
-    private long max = Long.MIN_VALUE;
-    private long sum = 0;
-    private long count = 0;
-
-    void collect(long d) {
-      if (d < min) min = d;
-      if (d > max) max = d;
-      sum += d;
-      count++;
-    }
-
-    public String toString() {
-      return "count: " + count +
-        "\tmin: " + min +
-        "\tmax: " + max +
-        "\tmean: " + ((double)sum/count);
-    }
-  }
-
   private static class KeyValueStatsCollector {
-    LongStats keyLen = new LongStats();
-    LongStats valLen = new LongStats();
-    LongStats rowSizeBytes = new LongStats();
-    LongStats rowSizeCols = new LongStats();
+    private final MetricsRegistry metricsRegistry = new MetricsRegistry();
+    private final ByteArrayOutputStream metricsOutput = new ByteArrayOutputStream();
+    private final SimpleReporter simpleReporter = new SimpleReporter(metricsRegistry, new PrintStream(metricsOutput));
+    Histogram keyLen = metricsRegistry.newHistogram(HFilePrettyPrinter.class, "Key length");
+    Histogram valLen = metricsRegistry.newHistogram(HFilePrettyPrinter.class, "Val length");
+    Histogram rowSizeBytes = metricsRegistry.newHistogram(HFilePrettyPrinter.class, "Row size (bytes)");
+    Histogram rowSizeCols = metricsRegistry.newHistogram(HFilePrettyPrinter.class, "Row size (columns)");
 
     long curRowBytes = 0;
     long curRowCols = 0;
@@ -402,23 +391,25 @@ public class HFilePrettyPrinter {
 
     private KeyValue prevKV = null;
     private long maxRowBytes = 0;
+    private long curRowKeyLength;
 
     public void collect(KeyValue kv) {
-      keyLen.collect(kv.getKeyLength());
-      valLen.collect(kv.getValueLength());
+      valLen.update(kv.getValueLength());
       if (prevKV != null &&
           KeyValue.COMPARATOR.compareRows(prevKV, kv) != 0) {
         // new row
         collectRow();
       }
       curRowBytes += kv.getLength();
+      curRowKeyLength = kv.getKeyLength();
       curRowCols++;
       prevKV = kv;
     }
 
     private void collectRow() {
-      rowSizeBytes.collect(curRowBytes);
-      rowSizeCols.collect(curRowCols);
+      rowSizeBytes.update(curRowBytes);
+      rowSizeCols.update(curRowCols);
+      keyLen.update(curRowKeyLength);
 
       if (curRowBytes > maxRowBytes && prevKV != null) {
         biggestRow = prevKV.getRow();
@@ -440,12 +431,46 @@ public class HFilePrettyPrinter {
       if (prevKV == null)
         return "no data available for statistics";
 
+      // Dump the metrics to the output stream
+      simpleReporter.shutdown();
+      simpleReporter.run();
+      metricsRegistry.shutdown();
+
       return
-        "Key length: " + keyLen + "\n" +
-        "Val length: " + valLen + "\n" +
-        "Row size (bytes): " + rowSizeBytes + "\n" +
-        "Row size (columns): " + rowSizeCols + "\n" +
-        "Key of biggest row: " + Bytes.toStringBinary(biggestRow);
+              metricsOutput.toString() +
+                      "Key of biggest row: " + Bytes.toStringBinary(biggestRow);
+    }
+  }
+
+  private static class SimpleReporter extends ConsoleReporter {
+    private final PrintStream out;
+
+    public SimpleReporter(MetricsRegistry metricsRegistry, PrintStream out) {
+      super(metricsRegistry, out, MetricPredicate.ALL);
+      this.out = out;
+    }
+
+    @Override
+    public void run() {
+      for (Map.Entry<String, SortedMap<MetricName, Metric>> entry : getMetricsRegistry().groupedMetrics(
+              MetricPredicate.ALL).entrySet()) {
+        try {
+          for (Map.Entry<MetricName, Metric> subEntry : entry.getValue().entrySet()) {
+            out.print("   " + subEntry.getKey().getName());
+            out.println(':');
+
+            subEntry.getValue().processWith(this, subEntry.getKey(), out);
+          }
+        } catch (Exception e) {
+          e.printStackTrace(out);
+        }
+      }
+    }
+
+    @Override
+    public void processHistogram(MetricName name, Histogram histogram, PrintStream stream) {
+      super.processHistogram(name, histogram, stream);
+      stream.printf(Locale.getDefault(), "             count = %d\n", histogram.count());
     }
   }
 }