001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.wal;
019
020import java.io.FileNotFoundException;
021import java.io.IOException;
022import java.io.PrintStream;
023import java.util.ArrayList;
024import java.util.Date;
025import java.util.HashMap;
026import java.util.Iterator;
027import java.util.List;
028import java.util.Map;
029import org.apache.hadoop.conf.Configuration;
030import org.apache.hadoop.fs.FileSystem;
031import org.apache.hadoop.fs.Path;
032import org.apache.hadoop.hbase.Cell;
033import org.apache.hadoop.hbase.CellUtil;
034import org.apache.hadoop.hbase.HBaseConfiguration;
035import org.apache.hadoop.hbase.HBaseInterfaceAudience;
036import org.apache.hadoop.hbase.PrivateCellUtil;
037import org.apache.hadoop.hbase.TableName;
038import org.apache.hadoop.hbase.Tag;
039import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
040import org.apache.hadoop.hbase.util.Bytes;
041import org.apache.hadoop.hbase.util.CommonFSUtils;
042import org.apache.hadoop.hbase.util.GsonUtil;
043import org.apache.yetus.audience.InterfaceAudience;
044import org.apache.yetus.audience.InterfaceStability;
045import org.slf4j.Logger;
046import org.slf4j.LoggerFactory;
047
048import org.apache.hbase.thirdparty.com.google.gson.Gson;
049import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
050import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLineParser;
051import org.apache.hbase.thirdparty.org.apache.commons.cli.HelpFormatter;
052import org.apache.hbase.thirdparty.org.apache.commons.cli.Options;
053import org.apache.hbase.thirdparty.org.apache.commons.cli.ParseException;
054import org.apache.hbase.thirdparty.org.apache.commons.cli.PosixParser;
055
056/**
057 * WALPrettyPrinter prints the contents of a given WAL with a variety of
058 * options affecting formatting and extent of content.
059 *
060 * It targets two usage cases: pretty printing for ease of debugging directly by
061 * humans, and JSON output for consumption by monitoring and/or maintenance
062 * scripts.
063 *
064 * It can filter by row, region, or sequence id.
065 *
066 * It can also toggle output of values.
067 *
068 */
069@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
070@InterfaceStability.Evolving
071public class WALPrettyPrinter {
072  private static final Logger LOG = LoggerFactory.getLogger(WALPrettyPrinter.class);
073
074  // Output template for pretty printing.
075  private static final String outputTmpl =
076      "Sequence=%s, table=%s, region=%s, at write timestamp=%s";
077
078  private boolean outputValues;
079  private boolean outputJSON;
080  // The following enable filtering by sequence, region, and row, respectively
081  private long sequence;
082  private String table;
083  private String region;
084  private String row;
085  // enable in order to output a single list of transactions from several files
086  private boolean persistentOutput;
087  private boolean firstTxn;
088  // useful for programmatic capture of JSON output
089  private PrintStream out;
090  // for JSON encoding
091  private static final Gson GSON = GsonUtil.createGson().create();
092  //allows for jumping straight to a given portion of the file
093  private long position;
094
095  /**
096   * Basic constructor that simply initializes values to reasonable defaults.
097   */
098  public WALPrettyPrinter() {
099    outputValues = false;
100    outputJSON = false;
101    sequence = -1;
102    table = null;
103    region = null;
104    row = null;
105    persistentOutput = false;
106    firstTxn = true;
107    out = System.out;
108  }
109
110  /**
111   * turns value output on
112   */
113  public void enableValues() {
114    outputValues = true;
115  }
116
117  /**
118   * turns value output off
119   */
120  public void disableValues() {
121    outputValues = false;
122  }
123
124  /**
125   * turns JSON output on
126   */
127  public void enableJSON() {
128    outputJSON = true;
129  }
130
131  /**
132   * turns JSON output off, and turns on "pretty strings" for human consumption
133   */
134  public void disableJSON() {
135    outputJSON = false;
136  }
137
138  /**
139   * sets the region by which output will be filtered
140   *
141   * @param sequence
142   *          when nonnegative, serves as a filter; only log entries with this
143   *          sequence id will be printed
144   */
145  public void setSequenceFilter(long sequence) {
146    this.sequence = sequence;
147  }
148
149  /**
150   * Sets the table filter. Only log entries for this table are printed.
151   * @param table table name to set.
152   */
153  public void setTableFilter(String table) {
154    this.table = table;
155  }
156  /**
157   * sets the region by which output will be filtered
158   *
159   * @param region
160   *          when not null, serves as a filter; only log entries from this
161   *          region will be printed
162   */
163  public void setRegionFilter(String region) {
164    this.region = region;
165  }
166
167  /**
168   * sets the region by which output will be filtered
169   *
170   * @param row
171   *          when not null, serves as a filter; only log entries from this row
172   *          will be printed
173   */
174  public void setRowFilter(String row) {
175    this.row = row;
176  }
177
178  /**
179   * sets the position to start seeking the WAL file
180   * @param position
181   *          initial position to start seeking the given WAL file
182   */
183  public void setPosition(long position) {
184    this.position = position;
185  }
186
187  /**
188   * enables output as a single, persistent list. at present, only relevant in
189   * the case of JSON output.
190   */
191  public void beginPersistentOutput() {
192    if (persistentOutput) {
193      return;
194    }
195    persistentOutput = true;
196    firstTxn = true;
197    if (outputJSON) {
198      out.print("[");
199    }
200  }
201
202  /**
203   * ends output of a single, persistent list. at present, only relevant in the
204   * case of JSON output.
205   */
206  public void endPersistentOutput() {
207    if (!persistentOutput) {
208      return;
209    }
210    persistentOutput = false;
211    if (outputJSON) {
212      out.print("]");
213    }
214  }
215
216  /**
217   * reads a log file and outputs its contents, one transaction at a time, as
218   * specified by the currently configured options
219   *
220   * @param conf
221   *          the HBase configuration relevant to this log file
222   * @param p
223   *          the path of the log file to be read
224   * @throws IOException
225   *           may be unable to access the configured filesystem or requested
226   *           file.
227   */
228  public void processFile(final Configuration conf, final Path p)
229      throws IOException {
230    FileSystem fs = p.getFileSystem(conf);
231    if (!fs.exists(p)) {
232      throw new FileNotFoundException(p.toString());
233    }
234    if (!fs.isFile(p)) {
235      throw new IOException(p + " is not a file");
236    }
237
238    WAL.Reader log = WALFactory.createReader(fs, p, conf);
239
240    if (log instanceof ProtobufLogReader) {
241      List<String> writerClsNames = ((ProtobufLogReader) log).getWriterClsNames();
242      if (writerClsNames != null && writerClsNames.size() > 0) {
243        out.print("Writer Classes: ");
244        for (int i = 0; i < writerClsNames.size(); i++) {
245          out.print(writerClsNames.get(i));
246          if (i != writerClsNames.size() - 1) {
247            out.print(" ");
248          }
249        }
250        out.println();
251      }
252
253      String cellCodecClsName = ((ProtobufLogReader) log).getCodecClsName();
254      if (cellCodecClsName != null) {
255        out.println("Cell Codec Class: " + cellCodecClsName);
256      }
257    }
258
259    if (outputJSON && !persistentOutput) {
260      out.print("[");
261      firstTxn = true;
262    }
263
264    if (position > 0) {
265      log.seek(position);
266    }
267
268    try {
269      WAL.Entry entry;
270      while ((entry = log.next()) != null) {
271        WALKey key = entry.getKey();
272        WALEdit edit = entry.getEdit();
273        // begin building a transaction structure
274        Map<String, Object> txn = key.toStringMap();
275        long writeTime = key.getWriteTime();
276        // check output filters
277        if (table != null && !((TableName) txn.get("table")).toString().equals(table)) {
278          continue;
279        }
280        if (sequence >= 0 && ((Long) txn.get("sequence")) != sequence) {
281          continue;
282        }
283        if (region != null && !txn.get("region").equals(region)) {
284          continue;
285        }
286        // initialize list into which we will store atomic actions
287        List<Map<String, Object>> actions = new ArrayList<>();
288        for (Cell cell : edit.getCells()) {
289          // add atomic operation to txn
290          Map<String, Object> op = new HashMap<>(toStringMap(cell));
291          if (outputValues) {
292            op.put("value", Bytes.toStringBinary(CellUtil.cloneValue(cell)));
293          }
294          // check row output filter
295          if (row == null || ((String) op.get("row")).equals(row)) {
296            actions.add(op);
297          }
298          op.put("total_size_sum", cell.heapSize());
299        }
300        if (actions.isEmpty()) {
301          continue;
302        }
303        txn.put("actions", actions);
304        if (outputJSON) {
305          // JSON output is a straightforward "toString" on the txn object
306          if (firstTxn) {
307            firstTxn = false;
308          } else {
309            out.print(",");
310          }
311          // encode and print JSON
312          out.print(GSON.toJson(txn));
313        } else {
314          // Pretty output, complete with indentation by atomic action
315          out.println(String.format(outputTmpl,
316              txn.get("sequence"), txn.get("table"), txn.get("region"), new Date(writeTime)));
317          for (int i = 0; i < actions.size(); i++) {
318            Map<String, Object> op = actions.get(i);
319            printCell(out, op, outputValues);
320          }
321        }
322        out.println("edit heap size: " + entry.getEdit().heapSize());
323        out.println("position: " + log.getPosition());
324      }
325    } finally {
326      log.close();
327    }
328    if (outputJSON && !persistentOutput) {
329      out.print("]");
330    }
331  }
332
333  public static void printCell(PrintStream out, Map<String, Object> op, boolean outputValues) {
334    out.println("row=" + op.get("row") + ", type=" + op.get("type") + ", column=" +
335      op.get("family") + ":" + op.get("qualifier"));
336    if (op.get("tag") != null) {
337      out.println("    tag: " + op.get("tag"));
338    }
339    if (outputValues) {
340      out.println("    value: " + op.get("value"));
341    }
342    out.println("cell total size sum: " + op.get("total_size_sum"));
343  }
344
345  public static Map<String, Object> toStringMap(Cell cell) {
346    Map<String, Object> stringMap = new HashMap<>();
347    stringMap.put("row",
348        Bytes.toStringBinary(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()));
349    stringMap.put("type", cell.getType());
350    stringMap.put("family", Bytes.toStringBinary(cell.getFamilyArray(), cell.getFamilyOffset(),
351                cell.getFamilyLength()));
352    stringMap.put("qualifier",
353        Bytes.toStringBinary(cell.getQualifierArray(), cell.getQualifierOffset(),
354            cell.getQualifierLength()));
355    stringMap.put("timestamp", cell.getTimestamp());
356    stringMap.put("vlen", cell.getValueLength());
357    if (cell.getTagsLength() > 0) {
358      List<String> tagsString = new ArrayList<>();
359      Iterator<Tag> tagsIterator = PrivateCellUtil.tagsIterator(cell);
360      while (tagsIterator.hasNext()) {
361        Tag tag = tagsIterator.next();
362        tagsString
363            .add((tag.getType()) + ":" + Bytes.toStringBinary(Tag.cloneValue(tag)));
364      }
365      stringMap.put("tag", tagsString);
366    }
367    return stringMap;
368  }
369
370  public static void main(String[] args) throws IOException {
371    run(args);
372  }
373
374  /**
375   * Pass one or more log file names and formatting options and it will dump out
376   * a text version of the contents on <code>stdout</code>.
377   *
378   * @param args
379   *          Command line arguments
380   * @throws IOException
381   *           Thrown upon file system errors etc.
382   */
383  public static void run(String[] args) throws IOException {
384    // create options
385    Options options = new Options();
386    options.addOption("h", "help", false, "Output help message");
387    options.addOption("j", "json", false, "Output JSON");
388    options.addOption("p", "printvals", false, "Print values");
389    options.addOption("t", "table", true, "Table name to filter by.");
390    options.addOption("r", "region", true,
391        "Region to filter by. Pass encoded region name; e.g. '9192caead6a5a20acb4454ffbc79fa14'");
392    options.addOption("s", "sequence", true,
393        "Sequence to filter by. Pass sequence number.");
394    options.addOption("w", "row", true, "Row to filter by. Pass row name.");
395    options.addOption("g", "goto", true, "Position to seek to in the file");
396
397    WALPrettyPrinter printer = new WALPrettyPrinter();
398    CommandLineParser parser = new PosixParser();
399    List<?> files = null;
400    try {
401      CommandLine cmd = parser.parse(options, args);
402      files = cmd.getArgList();
403      if (files.isEmpty() || cmd.hasOption("h")) {
404        HelpFormatter formatter = new HelpFormatter();
405        formatter.printHelp("WAL <filename...>", options, true);
406        System.exit(-1);
407      }
408      // configure the pretty printer using command line options
409      if (cmd.hasOption("p")) {
410        printer.enableValues();
411      }
412      if (cmd.hasOption("j")) {
413        printer.enableJSON();
414      }
415      if (cmd.hasOption("t")) {
416        printer.setTableFilter(cmd.getOptionValue("t"));
417      }
418      if (cmd.hasOption("r")) {
419        printer.setRegionFilter(cmd.getOptionValue("r"));
420      }
421      if (cmd.hasOption("s")) {
422        printer.setSequenceFilter(Long.parseLong(cmd.getOptionValue("s")));
423      }
424      if (cmd.hasOption("w")) {
425        printer.setRowFilter(cmd.getOptionValue("w"));
426      }
427      if (cmd.hasOption("g")) {
428        printer.setPosition(Long.parseLong(cmd.getOptionValue("g")));
429      }
430    } catch (ParseException e) {
431      LOG.error("Failed to parse commandLine arguments", e);
432      HelpFormatter formatter = new HelpFormatter();
433      formatter.printHelp("HFile filename(s) ", options, true);
434      System.exit(-1);
435    }
436    // get configuration, file system, and process the given files
437    Configuration conf = HBaseConfiguration.create();
438    CommonFSUtils.setFsDefault(conf, CommonFSUtils.getRootDir(conf));
439
440    // begin output
441    printer.beginPersistentOutput();
442    for (Object f : files) {
443      Path file = new Path((String) f);
444      FileSystem fs = file.getFileSystem(conf);
445      if (!fs.exists(file)) {
446        System.err.println("ERROR, file doesnt exist: " + file);
447        return;
448      }
449      printer.processFile(conf, file);
450    }
451    printer.endPersistentOutput();
452  }
453}