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.regionserver.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.Tag;
45  import org.apache.hadoop.hbase.regionserver.wal.HLog.Reader;
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  /**
51   * HLogPrettyPrinter prints the contents of a given HLog with a variety of
52   * options affecting formatting and extent of content.
53   * 
54   * It targets two usage cases: pretty printing for ease of debugging directly by
55   * humans, and JSON output for consumption by monitoring and/or maintenance
56   * scripts.
57   * 
58   * It can filter by row, region, or sequence id.
59   * 
60   * It can also toggle output of values.
61   * 
62   */
63  @InterfaceAudience.Public
64  @InterfaceStability.Evolving
65  public class HLogPrettyPrinter {
66    private boolean outputValues;
67    private boolean outputJSON;
68    // The following enable filtering by sequence, region, and row, respectively
69    private long sequence;
70    private String region;
71    private String row;
72    // enable in order to output a single list of transactions from several files
73    private boolean persistentOutput;
74    private boolean firstTxn;
75    // useful for programatic capture of JSON output
76    private PrintStream out;
77    // for JSON encoding
78    private static final ObjectMapper MAPPER = new ObjectMapper();
79  
80    /**
81     * Basic constructor that simply initializes values to reasonable defaults.
82     */
83    public HLogPrettyPrinter() {
84      outputValues = false;
85      outputJSON = false;
86      sequence = -1;
87      region = null;
88      row = null;
89      persistentOutput = false;
90      firstTxn = true;
91      out = System.out;
92    }
93  
94    /**
95     * Fully specified constructor.
96     * 
97     * @param outputValues
98     *          when true, enables output of values along with other log
99     *          information
100    * @param outputJSON
101    *          when true, enables output in JSON format rather than a
102    *          "pretty string"
103    * @param sequence
104    *          when nonnegative, serves as a filter; only log entries with this
105    *          sequence id will be printed
106    * @param region
107    *          when not null, serves as a filter; only log entries from this
108    *          region will be printed
109    * @param row
110    *          when not null, serves as a filter; only log entries from this row
111    *          will be printed
112    * @param persistentOutput
113    *          keeps a single list running for multiple files. if enabled, the
114    *          endPersistentOutput() method must be used!
115    * @param out
116    *          Specifies an alternative to stdout for the destination of this 
117    *          PrettyPrinter's output.
118    */
119   public HLogPrettyPrinter(boolean outputValues, boolean outputJSON,
120       long sequence, String region, String row, boolean persistentOutput,
121       PrintStream out) {
122     this.outputValues = outputValues;
123     this.outputJSON = outputJSON;
124     this.sequence = sequence;
125     this.region = region;
126     this.row = row;
127     this.persistentOutput = persistentOutput;
128     if (persistentOutput) {
129       beginPersistentOutput();
130     }
131     this.out = out;
132     this.firstTxn = true;
133   }
134 
135   /**
136    * turns value output on
137    */
138   public void enableValues() {
139     outputValues = true;
140   }
141 
142   /**
143    * turns value output off
144    */
145   public void disableValues() {
146     outputValues = false;
147   }
148 
149   /**
150    * turns JSON output on
151    */
152   public void enableJSON() {
153     outputJSON = true;
154   }
155 
156   /**
157    * turns JSON output off, and turns on "pretty strings" for human consumption
158    */
159   public void disableJSON() {
160     outputJSON = false;
161   }
162 
163   /**
164    * sets the region by which output will be filtered
165    * 
166    * @param sequence
167    *          when nonnegative, serves as a filter; only log entries with this
168    *          sequence id will be printed
169    */
170   public void setSequenceFilter(long sequence) {
171     this.sequence = sequence;
172   }
173 
174   /**
175    * sets the region by which output will be filtered
176    * 
177    * @param region
178    *          when not null, serves as a filter; only log entries from this
179    *          region will be printed
180    */
181   public void setRegionFilter(String region) {
182     this.region = region;
183   }
184 
185   /**
186    * sets the region by which output will be filtered
187    * 
188    * @param row
189    *          when not null, serves as a filter; only log entries from this row
190    *          will be printed
191    */
192   public void setRowFilter(String row) {
193     this.row = row;
194   }
195 
196   /**
197    * enables output as a single, persistent list. at present, only relevant in
198    * the case of JSON output.
199    */
200   public void beginPersistentOutput() {
201     if (persistentOutput)
202       return;
203     persistentOutput = true;
204     firstTxn = true;
205     if (outputJSON)
206       out.print("[");
207   }
208 
209   /**
210    * ends output of a single, persistent list. at present, only relevant in the
211    * case of JSON output.
212    */
213   public void endPersistentOutput() {
214     if (!persistentOutput)
215       return;
216     persistentOutput = false;
217     if (outputJSON)
218       out.print("]");
219   }
220 
221   /**
222    * reads a log file and outputs its contents, one transaction at a time, as
223    * specified by the currently configured options
224    * 
225    * @param conf
226    *          the HBase configuration relevant to this log file
227    * @param p
228    *          the path of the log file to be read
229    * @throws IOException
230    *           may be unable to access the configured filesystem or requested
231    *           file.
232    */
233   public void processFile(final Configuration conf, final Path p)
234       throws IOException {
235     FileSystem fs = FileSystem.get(conf);
236     if (!fs.exists(p)) {
237       throw new FileNotFoundException(p.toString());
238     }
239     if (!fs.isFile(p)) {
240       throw new IOException(p + " is not a file");
241     }
242     if (outputJSON && !persistentOutput) {
243       out.print("[");
244       firstTxn = true;
245     }
246     Reader log = HLogFactory.createReader(fs, p, conf);
247     try {
248       FSHLog.Entry entry;
249       while ((entry = log.next()) != null) {
250         HLogKey key = entry.getKey();
251         WALEdit edit = entry.getEdit();
252         // begin building a transaction structure
253         Map<String, Object> txn = key.toStringMap();
254         long writeTime = key.getWriteTime();
255         // check output filters
256         if (sequence >= 0 && ((Long) txn.get("sequence")) != sequence)
257           continue;
258         if (region != null && !((String) txn.get("region")).equals(region))
259           continue;
260         // initialize list into which we will store atomic actions
261         List<Map> actions = new ArrayList<Map>();
262         for (Cell cell : edit.getCells()) {
263           // add atomic operation to txn
264           Map<String, Object> op = new HashMap<String, Object>(toStringMap(cell));
265           if (outputValues) op.put("value", Bytes.toStringBinary(cell.getValue()));
266           // check row output filter
267           if (row == null || ((String) op.get("row")).equals(row))
268             actions.add(op);
269         }
270         if (actions.size() == 0)
271           continue;
272         txn.put("actions", actions);
273         if (outputJSON) {
274           // JSON output is a straightforward "toString" on the txn object
275           if (firstTxn)
276             firstTxn = false;
277           else
278             out.print(",");
279           // encode and print JSON
280           out.print(MAPPER.writeValueAsString(txn));
281         } else {
282           // Pretty output, complete with indentation by atomic action
283           out.println("Sequence " + txn.get("sequence") + " "
284               + "from region " + txn.get("region") + " " + "in table "
285               + txn.get("table") + " at write timestamp: " + new Date(writeTime));
286           for (int i = 0; i < actions.size(); i++) {
287             Map op = actions.get(i);
288             out.println("  Action:");
289             out.println("    row: " + op.get("row"));
290             out.println("    column: " + op.get("family") + ":"
291                 + op.get("qualifier"));
292             out.println("    timestamp: "
293                 + (new Date((Long) op.get("timestamp"))));
294             if(op.get("tag") != null) {
295               out.println("    tag: " + op.get("tag"));
296             }
297             if (outputValues)
298               out.println("    value: " + op.get("value"));
299           }
300         }
301       }
302     } finally {
303       log.close();
304     }
305     if (outputJSON && !persistentOutput) {
306       out.print("]");
307     }
308   }
309 
310   private static Map<String, Object> toStringMap(Cell cell) {
311     Map<String, Object> stringMap = new HashMap<String, Object>();
312     stringMap.put("row",
313         Bytes.toStringBinary(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()));
314     stringMap.put("family", Bytes.toStringBinary(cell.getFamilyArray(), cell.getFamilyOffset(),
315                 cell.getFamilyLength()));
316     stringMap.put("qualifier",
317         Bytes.toStringBinary(cell.getQualifierArray(), cell.getQualifierOffset(),
318             cell.getQualifierLength()));
319     stringMap.put("timestamp", cell.getTimestamp());
320     stringMap.put("vlen", cell.getValueLength());
321     if (cell.getTagsLength() > 0) {
322       List<String> tagsString = new ArrayList<String>();
323       Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
324           cell.getTagsLength());
325       while (tagsIterator.hasNext()) {
326         Tag tag = tagsIterator.next();
327         tagsString.add((tag.getType()) + ":"
328             + Bytes.toStringBinary(tag.getBuffer(), tag.getTagOffset(), tag.getTagLength()));
329       }
330       stringMap.put("tag", tagsString);
331     }
332     return stringMap;
333   }
334 
335   public static void main(String[] args) throws IOException {
336     run(args);
337   }
338 
339   /**
340    * Pass one or more log file names and formatting options and it will dump out
341    * a text version of the contents on <code>stdout</code>.
342    * 
343    * @param args
344    *          Command line arguments
345    * @throws IOException
346    *           Thrown upon file system errors etc.
347    * @throws ParseException
348    *           Thrown if command-line parsing fails.
349    */
350   public static void run(String[] args) throws IOException {
351     // create options
352     Options options = new Options();
353     options.addOption("h", "help", false, "Output help message");
354     options.addOption("j", "json", false, "Output JSON");
355     options.addOption("p", "printvals", false, "Print values");
356     options.addOption("r", "region", true,
357         "Region to filter by. Pass region name; e.g. 'hbase:meta,,1'");
358     options.addOption("s", "sequence", true,
359         "Sequence to filter by. Pass sequence number.");
360     options.addOption("w", "row", true, "Row to filter by. Pass row name.");
361 
362     HLogPrettyPrinter printer = new HLogPrettyPrinter();
363     CommandLineParser parser = new PosixParser();
364     List files = null;
365     try {
366       CommandLine cmd = parser.parse(options, args);
367       files = cmd.getArgList();
368       if (files.size() == 0 || cmd.hasOption("h")) {
369         HelpFormatter formatter = new HelpFormatter();
370         formatter.printHelp("HLog <filename...>", options, true);
371         System.exit(-1);
372       }
373       // configure the pretty printer using command line options
374       if (cmd.hasOption("p"))
375         printer.enableValues();
376       if (cmd.hasOption("j"))
377         printer.enableJSON();
378       if (cmd.hasOption("r"))
379         printer.setRegionFilter(cmd.getOptionValue("r"));
380       if (cmd.hasOption("s"))
381         printer.setSequenceFilter(Long.parseLong(cmd.getOptionValue("s")));
382       if (cmd.hasOption("w"))
383         printer.setRowFilter(cmd.getOptionValue("w"));
384     } catch (ParseException e) {
385       e.printStackTrace();
386       HelpFormatter formatter = new HelpFormatter();
387       formatter.printHelp("HFile filename(s) ", options, true);
388       System.exit(-1);
389     }
390     // get configuration, file system, and process the given files
391     Configuration conf = HBaseConfiguration.create();
392     FSUtils.setFsDefault(conf, FSUtils.getRootDir(conf));
393 
394     // begin output
395     printer.beginPersistentOutput();
396     for (Object f : files) {
397       Path file = new Path((String) f);
398       FileSystem fs = file.getFileSystem(conf);
399       if (!fs.exists(file)) {
400         System.err.println("ERROR, file doesnt exist: " + file);
401         return;
402       }
403       printer.processFile(conf, file);
404     }
405     printer.endPersistentOutput();
406   }
407 }