View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.wal;
19  
20  import java.io.FileNotFoundException;
21  import java.io.IOException;
22  import java.io.PrintStream;
23  import java.util.ArrayList;
24  import java.util.Date;
25  import java.util.HashMap;
26  import java.util.Iterator;
27  import java.util.List;
28  import java.util.Map;
29  
30  import org.apache.commons.cli.CommandLine;
31  import org.apache.commons.cli.CommandLineParser;
32  import org.apache.commons.cli.HelpFormatter;
33  import org.apache.commons.cli.Options;
34  import org.apache.commons.cli.ParseException;
35  import org.apache.commons.cli.PosixParser;
36  import org.apache.hadoop.hbase.classification.InterfaceAudience;
37  import org.apache.hadoop.hbase.classification.InterfaceStability;
38  import org.apache.hadoop.conf.Configuration;
39  import org.apache.hadoop.fs.FileSystem;
40  import org.apache.hadoop.fs.Path;
41  import org.apache.hadoop.hbase.Cell;
42  import org.apache.hadoop.hbase.CellUtil;
43  import org.apache.hadoop.hbase.HBaseConfiguration;
44  import org.apache.hadoop.hbase.HBaseInterfaceAudience;
45  import org.apache.hadoop.hbase.Tag;
46  import org.apache.hadoop.hbase.util.Bytes;
47  import org.apache.hadoop.hbase.util.FSUtils;
48  import org.codehaus.jackson.map.ObjectMapper;
49  
50  import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
51  // imports for things that haven't moved yet.
52  import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
53  
54  /**
55   * WALPrettyPrinter prints the contents of a given WAL with a variety of
56   * options affecting formatting and extent of content.
57   *
58   * It targets two usage cases: pretty printing for ease of debugging directly by
59   * humans, and JSON output for consumption by monitoring and/or maintenance
60   * scripts.
61   *
62   * It can filter by row, region, or sequence id.
63   *
64   * It can also toggle output of values.
65   *
66   */
67  @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
68  @InterfaceStability.Evolving
69  public class WALPrettyPrinter {
70    private boolean outputValues;
71    private boolean outputJSON;
72    // The following enable filtering by sequence, region, and row, respectively
73    private long sequence;
74    private String region;
75    private String row;
76    // enable in order to output a single list of transactions from several files
77    private boolean persistentOutput;
78    private boolean firstTxn;
79    // useful for programmatic capture of JSON output
80    private PrintStream out;
81    // for JSON encoding
82    private static final ObjectMapper MAPPER = new ObjectMapper();
83  
84    private long position;
85  
86    /**
87     * Basic constructor that simply initializes values to reasonable defaults.
88     */
89    public WALPrettyPrinter() {
90      outputValues = false;
91      outputJSON = false;
92      sequence = -1;
93      region = null;
94      row = null;
95      persistentOutput = false;
96      firstTxn = true;
97      out = System.out;
98    }
99  
100   /**
101    * Fully specified constructor.
102    *
103    * @param outputValues
104    *          when true, enables output of values along with other log
105    *          information
106    * @param outputJSON
107    *          when true, enables output in JSON format rather than a
108    *          "pretty string"
109    * @param sequence
110    *          when nonnegative, serves as a filter; only log entries with this
111    *          sequence id will be printed
112    * @param region
113    *          when not null, serves as a filter; only log entries from this
114    *          region will be printed
115    * @param row
116    *          when not null, serves as a filter; only log entries from this row
117    *          will be printed
118    * @param persistentOutput
119    *          keeps a single list running for multiple files. if enabled, the
120    *          endPersistentOutput() method must be used!
121    * @param out
122    *          Specifies an alternative to stdout for the destination of this
123    *          PrettyPrinter's output.
124    */
125   public WALPrettyPrinter(boolean outputValues, boolean outputJSON,
126       long sequence, String region, String row, boolean
127       persistentOutput,
128       PrintStream out) {
129     this.outputValues = outputValues;
130     this.outputJSON = outputJSON;
131     this.sequence = sequence;
132     this.region = region;
133     this.row = row;
134     this.persistentOutput = persistentOutput;
135     if (persistentOutput) {
136       beginPersistentOutput();
137     }
138     this.out = out;
139     this.firstTxn = true;
140   }
141 
142   /**
143    * turns value output on
144    */
145   public void enableValues() {
146     outputValues = true;
147   }
148 
149   /**
150    * turns value output off
151    */
152   public void disableValues() {
153     outputValues = false;
154   }
155 
156   /**
157    * turns JSON output on
158    */
159   public void enableJSON() {
160     outputJSON = true;
161   }
162 
163   /**
164    * turns JSON output off, and turns on "pretty strings" for human consumption
165    */
166   public void disableJSON() {
167     outputJSON = false;
168   }
169 
170   /**
171    * sets the region by which output will be filtered
172    *
173    * @param sequence
174    *          when nonnegative, serves as a filter; only log entries with this
175    *          sequence id will be printed
176    */
177   public void setSequenceFilter(long sequence) {
178     this.sequence = sequence;
179   }
180 
181   /**
182    * sets the region by which output will be filtered
183    *
184    * @param region
185    *          when not null, serves as a filter; only log entries from this
186    *          region will be printed
187    */
188   public void setRegionFilter(String region) {
189     this.region = region;
190   }
191 
192   /**
193    * sets the region by which output will be filtered
194    *
195    * @param row
196    *          when not null, serves as a filter; only log entries from this row
197    *          will be printed
198    */
199   public void setRowFilter(String row) {
200     this.row = row;
201   }
202 
203   /**
204    * enables output as a single, persistent list. at present, only relevant in
205    * the case of JSON output.
206    */
207   public void beginPersistentOutput() {
208     if (persistentOutput)
209       return;
210     persistentOutput = true;
211     firstTxn = true;
212     if (outputJSON)
213       out.print("[");
214   }
215 
216   /**
217    * ends output of a single, persistent list. at present, only relevant in the
218    * case of JSON output.
219    */
220   public void endPersistentOutput() {
221     if (!persistentOutput)
222       return;
223     persistentOutput = false;
224     if (outputJSON)
225       out.print("]");
226   }
227 
228   /**
229    * reads a log file and outputs its contents, one transaction at a time, as
230    * specified by the currently configured options
231    *
232    * @param conf
233    *          the HBase configuration relevant to this log file
234    * @param p
235    *          the path of the log file to be read
236    * @throws IOException
237    *           may be unable to access the configured filesystem or requested
238    *           file.
239    */
240   public void processFile(final Configuration conf, final Path p)
241       throws IOException {
242     FileSystem fs = p.getFileSystem(conf);
243     if (!fs.exists(p)) {
244       throw new FileNotFoundException(p.toString());
245     }
246     if (!fs.isFile(p)) {
247       throw new IOException(p + " is not a file");
248     }
249 
250     WAL.Reader log = WALFactory.createReader(fs, p, conf);
251     
252     if (log instanceof ProtobufLogReader) {
253       List<String> writerClsNames = ((ProtobufLogReader) log).getWriterClsNames();
254       if (writerClsNames != null && writerClsNames.size() > 0) {
255         out.print("Writer Classes: ");
256         for (int i = 0; i < writerClsNames.size(); i++) {
257           out.print(writerClsNames.get(i));
258           if (i != writerClsNames.size() - 1) {
259             out.print(" ");
260           }
261         }
262         out.println();
263       }
264       
265       String cellCodecClsName = ((ProtobufLogReader) log).getCodecClsName();
266       if (cellCodecClsName != null) {
267         out.println("Cell Codec Class: " + cellCodecClsName);
268       }
269     }
270     
271     if (outputJSON && !persistentOutput) {
272       out.print("[");
273       firstTxn = true;
274     }
275 
276     if (position > 0) {
277       log.seek(position);
278     }
279 
280     try {
281       WAL.Entry entry;
282       while ((entry = log.next()) != null) {
283         WALKey key = entry.getKey();
284         WALEdit edit = entry.getEdit();
285         // begin building a transaction structure
286         Map<String, Object> txn = key.toStringMap();
287         long writeTime = key.getWriteTime();
288         // check output filters
289         if (sequence >= 0 && ((Long) txn.get("sequence")) != sequence)
290           continue;
291         if (region != null && !((String) txn.get("region")).equals(region))
292           continue;
293         // initialize list into which we will store atomic actions
294         List<Map> actions = new ArrayList<Map>();
295         for (Cell cell : edit.getCells()) {
296           // add atomic operation to txn
297           Map<String, Object> op = new HashMap<String, Object>(toStringMap(cell));
298           if (outputValues) op.put("value", Bytes.toStringBinary(cell.getValue()));
299           // check row output filter
300           if (row == null || ((String) op.get("row")).equals(row)) {
301             actions.add(op);
302           }
303           op.put("total_size_sum", CellUtil.estimatedHeapSizeOf(cell));
304 
305         }
306         if (actions.size() == 0)
307           continue;
308         txn.put("actions", actions);
309         if (outputJSON) {
310           // JSON output is a straightforward "toString" on the txn object
311           if (firstTxn)
312             firstTxn = false;
313           else
314             out.print(",");
315           // encode and print JSON
316           out.print(MAPPER.writeValueAsString(txn));
317         } else {
318           // Pretty output, complete with indentation by atomic action
319           out.println("Sequence=" + txn.get("sequence") + " "
320               + ", region=" + txn.get("region") + " at write timestamp=" + new Date(writeTime));
321           for (int i = 0; i < actions.size(); i++) {
322             Map op = actions.get(i);
323             out.println("row=" + op.get("row") +
324                 ", column=" + op.get("family") + ":" + op.get("qualifier"));
325             if (op.get("tag") != null) {
326               out.println("    tag: " + op.get("tag"));
327             }
328             if (outputValues) out.println("    value: " + op.get("value"));
329             out.println("cell total size sum: " + op.get("total_size_sum"));
330           }
331         }
332         out.println("edit heap size: " + entry.getEdit().heapSize());
333         out.println("position: " + log.getPosition());
334       }
335     } finally {
336       log.close();
337     }
338     if (outputJSON && !persistentOutput) {
339       out.print("]");
340     }
341   }
342 
343   private static Map<String, Object> toStringMap(Cell cell) {
344     Map<String, Object> stringMap = new HashMap<String, Object>();
345     stringMap.put("row",
346         Bytes.toStringBinary(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()));
347     stringMap.put("family", Bytes.toStringBinary(cell.getFamilyArray(), cell.getFamilyOffset(),
348                 cell.getFamilyLength()));
349     stringMap.put("qualifier",
350         Bytes.toStringBinary(cell.getQualifierArray(), cell.getQualifierOffset(),
351             cell.getQualifierLength()));
352     stringMap.put("timestamp", cell.getTimestamp());
353     stringMap.put("vlen", cell.getValueLength());
354     if (cell.getTagsLength() > 0) {
355       List<String> tagsString = new ArrayList<String>();
356       Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
357           cell.getTagsLength());
358       while (tagsIterator.hasNext()) {
359         Tag tag = tagsIterator.next();
360         tagsString.add((tag.getType()) + ":"
361             + Bytes.toStringBinary(tag.getBuffer(), tag.getTagOffset(), tag.getTagLength()));
362       }
363       stringMap.put("tag", tagsString);
364     }
365     return stringMap;
366   }
367 
368   public static void main(String[] args) throws IOException {
369     run(args);
370   }
371 
372   /**
373    * Pass one or more log file names and formatting options and it will dump out
374    * a text version of the contents on <code>stdout</code>.
375    *
376    * @param args
377    *          Command line arguments
378    * @throws IOException
379    *           Thrown upon file system errors etc.
380    */
381   public static void run(String[] args) throws IOException {
382     // create options
383     Options options = new Options();
384     options.addOption("h", "help", false, "Output help message");
385     options.addOption("j", "json", false, "Output JSON");
386     options.addOption("p", "printvals", false, "Print values");
387     options.addOption("r", "region", true,
388         "Region to filter by. Pass encoded region name; e.g. '9192caead6a5a20acb4454ffbc79fa14'");
389     options.addOption("s", "sequence", true,
390         "Sequence to filter by. Pass sequence number.");
391     options.addOption("w", "row", true, "Row to filter by. Pass row name.");
392     options.addOption("g", "goto", true, "Position to seek to in the file");
393 
394     WALPrettyPrinter printer = new WALPrettyPrinter();
395     CommandLineParser parser = new PosixParser();
396     List<?> files = null;
397     try {
398       CommandLine cmd = parser.parse(options, args);
399       files = cmd.getArgList();
400       if (files.size() == 0 || cmd.hasOption("h")) {
401         HelpFormatter formatter = new HelpFormatter();
402         formatter.printHelp("WAL <filename...>", options, true);
403         System.exit(-1);
404       }
405       // configure the pretty printer using command line options
406       if (cmd.hasOption("p"))
407         printer.enableValues();
408       if (cmd.hasOption("j"))
409         printer.enableJSON();
410       if (cmd.hasOption("r"))
411         printer.setRegionFilter(cmd.getOptionValue("r"));
412       if (cmd.hasOption("s"))
413         printer.setSequenceFilter(Long.parseLong(cmd.getOptionValue("s")));
414       if (cmd.hasOption("w"))
415         printer.setRowFilter(cmd.getOptionValue("w"));
416       if (cmd.hasOption("g")) {
417         printer.setPosition(Long.parseLong(cmd.getOptionValue("g")));
418       }
419     } catch (ParseException e) {
420       e.printStackTrace();
421       HelpFormatter formatter = new HelpFormatter();
422       formatter.printHelp("HFile filename(s) ", options, true);
423       System.exit(-1);
424     }
425     // get configuration, file system, and process the given files
426     Configuration conf = HBaseConfiguration.create();
427     FSUtils.setFsDefault(conf, FSUtils.getRootDir(conf));
428 
429     // begin output
430     printer.beginPersistentOutput();
431     for (Object f : files) {
432       Path file = new Path((String) f);
433       FileSystem fs = file.getFileSystem(conf);
434       if (!fs.exists(file)) {
435         System.err.println("ERROR, file doesnt exist: " + file);
436         return;
437       }
438       printer.processFile(conf, file);
439     }
440     printer.endPersistentOutput();
441   }
442 
443   public void setPosition(long position) {
444     this.position = position;
445   }
446 }