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