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