You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by wc...@apache.org on 2019/12/04 10:16:12 UTC

[hbase] branch branch-2 updated: HBASE-23362: [WalPrettyPrinter] print/filter by table name. (#898)

This is an automated email from the ASF dual-hosted git repository.

wchevreuil pushed a commit to branch branch-2
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2 by this push:
     new e87db35  HBASE-23362: [WalPrettyPrinter] print/filter by table name. (#898)
e87db35 is described below

commit e87db35d1a01439cd8d829d9e005e5a5077cd751
Author: Bharath Vissapragada <bh...@apache.org>
AuthorDate: Wed Dec 4 02:03:33 2019 -0800

    HBASE-23362: [WalPrettyPrinter] print/filter by table name. (#898)
    
    Signed-off-by: Wellington Chevreuil <wc...@apache.org>
    (cherry picked from commit 570cc3c4a4d30bdd3dd11e6b4db4329fe207925c)
---
 .../apache/hadoop/hbase/wal/WALPrettyPrinter.java  | 87 ++++++++++++++++------
 1 file changed, 66 insertions(+), 21 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALPrettyPrinter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALPrettyPrinter.java
index 1167ef0..b49a451 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALPrettyPrinter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALPrettyPrinter.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.PrivateCellUtil;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -41,7 +42,8 @@ import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.GsonUtil;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
-
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hbase.thirdparty.com.google.gson.Gson;
 import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
 import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLineParser;
@@ -66,10 +68,17 @@ import org.apache.hbase.thirdparty.org.apache.commons.cli.PosixParser;
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
 @InterfaceStability.Evolving
 public class WALPrettyPrinter {
+  private static final Logger LOG = LoggerFactory.getLogger(WALPrettyPrinter.class);
+
+  // Output template for pretty printing.
+  private static final String outputTmpl =
+      "Sequence=%s, table=%s, region=%s, at write timestamp=%s";
+
   private boolean outputValues;
   private boolean outputJSON;
   // The following enable filtering by sequence, region, and row, respectively
   private long sequence;
+  private String table;
   private String region;
   private String row;
   // enable in order to output a single list of transactions from several files
@@ -89,6 +98,7 @@ public class WALPrettyPrinter {
     outputValues = false;
     outputJSON = false;
     sequence = -1;
+    table = null;
     region = null;
     row = null;
     persistentOutput = false;
@@ -108,6 +118,9 @@ public class WALPrettyPrinter {
    * @param sequence
    *          when nonnegative, serves as a filter; only log entries with this
    *          sequence id will be printed
+   * @param table
+   *          when non null, serves as a filter. only entries corresponding to this
+   *          table will be printed.
    * @param region
    *          when not null, serves as a filter; only log entries from this
    *          region will be printed
@@ -122,11 +135,12 @@ public class WALPrettyPrinter {
    *          PrettyPrinter's output.
    */
   public WALPrettyPrinter(boolean outputValues, boolean outputJSON,
-      long sequence, String region, String row, boolean persistentOutput,
+      long sequence, String table, String region, String row, boolean persistentOutput,
       PrintStream out) {
     this.outputValues = outputValues;
     this.outputJSON = outputJSON;
     this.sequence = sequence;
+    this.table = table;
     this.region = region;
     this.row = row;
     this.persistentOutput = persistentOutput;
@@ -177,6 +191,13 @@ public class WALPrettyPrinter {
   }
 
   /**
+   * Sets the table filter. Only log entries for this table are printed.
+   * @param table table name to set.
+   */
+  public void setTableFilter(String table) {
+    this.table = table;
+  }
+  /**
    * sets the region by which output will be filtered
    *
    * @param region
@@ -212,12 +233,14 @@ public class WALPrettyPrinter {
    * the case of JSON output.
    */
   public void beginPersistentOutput() {
-    if (persistentOutput)
+    if (persistentOutput) {
       return;
+    }
     persistentOutput = true;
     firstTxn = true;
-    if (outputJSON)
+    if (outputJSON) {
       out.print("[");
+    }
   }
 
   /**
@@ -225,11 +248,13 @@ public class WALPrettyPrinter {
    * case of JSON output.
    */
   public void endPersistentOutput() {
-    if (!persistentOutput)
+    if (!persistentOutput) {
       return;
+    }
     persistentOutput = false;
-    if (outputJSON)
+    if (outputJSON) {
       out.print("]");
+    }
   }
 
   /**
@@ -293,37 +318,46 @@ public class WALPrettyPrinter {
         Map<String, Object> txn = key.toStringMap();
         long writeTime = key.getWriteTime();
         // check output filters
-        if (sequence >= 0 && ((Long) txn.get("sequence")) != sequence)
+        if (table != null && !((TableName) txn.get("table")).toString().equals(table)) {
           continue;
-        if (region != null && !((String) txn.get("region")).equals(region))
+        }
+        if (sequence >= 0 && ((Long) txn.get("sequence")) != sequence) {
           continue;
+        }
+        if (region != null && !txn.get("region").equals(region)) {
+          continue;
+        }
         // initialize list into which we will store atomic actions
         List<Map> actions = new ArrayList<>();
         for (Cell cell : edit.getCells()) {
           // add atomic operation to txn
           Map<String, Object> op = new HashMap<>(toStringMap(cell));
-          if (outputValues) op.put("value", Bytes.toStringBinary(CellUtil.cloneValue(cell)));
+          if (outputValues) {
+            op.put("value", Bytes.toStringBinary(CellUtil.cloneValue(cell)));
+          }
           // check row output filter
           if (row == null || ((String) op.get("row")).equals(row)) {
             actions.add(op);
           }
           op.put("total_size_sum", cell.heapSize());
         }
-        if (actions.isEmpty())
+        if (actions.isEmpty()) {
           continue;
+        }
         txn.put("actions", actions);
         if (outputJSON) {
           // JSON output is a straightforward "toString" on the txn object
-          if (firstTxn)
+          if (firstTxn) {
             firstTxn = false;
-          else
+          } else {
             out.print(",");
+          }
           // encode and print JSON
           out.print(GSON.toJson(txn));
         } else {
           // Pretty output, complete with indentation by atomic action
-          out.println("Sequence=" + txn.get("sequence") + " "
-              + ", region=" + txn.get("region") + " at write timestamp=" + new Date(writeTime));
+          out.println(String.format(outputTmpl,
+              txn.get("sequence"), txn.get("table"), txn.get("region"), new Date(writeTime)));
           for (int i = 0; i < actions.size(); i++) {
             Map op = actions.get(i);
             out.println("row=" + op.get("row") +
@@ -331,7 +365,9 @@ public class WALPrettyPrinter {
             if (op.get("tag") != null) {
               out.println("    tag: " + op.get("tag"));
             }
-            if (outputValues) out.println("    value: " + op.get("value"));
+            if (outputValues) {
+              out.println("    value: " + op.get("value"));
+            }
             out.println("cell total size sum: " + op.get("total_size_sum"));
           }
         }
@@ -389,6 +425,7 @@ public class WALPrettyPrinter {
     options.addOption("h", "help", false, "Output help message");
     options.addOption("j", "json", false, "Output JSON");
     options.addOption("p", "printvals", false, "Print values");
+    options.addOption("t", "table", true, "Table name to filter by.");
     options.addOption("r", "region", true,
         "Region to filter by. Pass encoded region name; e.g. '9192caead6a5a20acb4454ffbc79fa14'");
     options.addOption("s", "sequence", true,
@@ -408,21 +445,29 @@ public class WALPrettyPrinter {
         System.exit(-1);
       }
       // configure the pretty printer using command line options
-      if (cmd.hasOption("p"))
+      if (cmd.hasOption("p")) {
         printer.enableValues();
-      if (cmd.hasOption("j"))
+      }
+      if (cmd.hasOption("j")) {
         printer.enableJSON();
-      if (cmd.hasOption("r"))
+      }
+      if (cmd.hasOption("t")) {
+        printer.setTableFilter(cmd.getOptionValue("t"));
+      }
+      if (cmd.hasOption("r")) {
         printer.setRegionFilter(cmd.getOptionValue("r"));
-      if (cmd.hasOption("s"))
+      }
+      if (cmd.hasOption("s")) {
         printer.setSequenceFilter(Long.parseLong(cmd.getOptionValue("s")));
-      if (cmd.hasOption("w"))
+      }
+      if (cmd.hasOption("w")) {
         printer.setRowFilter(cmd.getOptionValue("w"));
+      }
       if (cmd.hasOption("g")) {
         printer.setPosition(Long.parseLong(cmd.getOptionValue("g")));
       }
     } catch (ParseException e) {
-      e.printStackTrace();
+      LOG.error("Failed to parse commandLine arguments", e);
       HelpFormatter formatter = new HelpFormatter();
       formatter.printHelp("HFile filename(s) ", options, true);
       System.exit(-1);