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