1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
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.Date;
24 import java.util.List;
25
26 import org.apache.commons.cli.CommandLine;
27 import org.apache.commons.cli.CommandLineParser;
28 import org.apache.commons.cli.HelpFormatter;
29 import org.apache.commons.cli.Options;
30 import org.apache.commons.cli.ParseException;
31 import org.apache.commons.cli.PosixParser;
32 import org.apache.hadoop.conf.Configuration;
33 import org.apache.hadoop.fs.FileSystem;
34 import org.apache.hadoop.fs.Path;
35 import org.apache.hadoop.hbase.HBaseConfiguration;
36 import org.apache.hadoop.hbase.KeyValue;
37 import org.apache.hadoop.hbase.regionserver.wal.HLog.Reader;
38 import org.apache.hadoop.hbase.util.Bytes;
39 import org.codehaus.jettison.json.JSONArray;
40 import org.codehaus.jettison.json.JSONException;
41 import org.codehaus.jettison.json.JSONObject;
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56 public class HLogPrettyPrinter {
57 private boolean outputValues;
58 private boolean outputJSON;
59
60 private long sequence;
61 private String region;
62 private String row;
63
64 private boolean persistentOutput;
65 private boolean firstTxn;
66
67 private PrintStream out;
68
69
70
71
72 public HLogPrettyPrinter() {
73 outputValues = false;
74 outputJSON = false;
75 sequence = -1;
76 region = null;
77 row = null;
78 persistentOutput = false;
79 firstTxn = true;
80 out = System.out;
81 }
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108 public HLogPrettyPrinter(boolean outputValues, boolean outputJSON,
109 long sequence, String region, String row, boolean persistentOutput,
110 PrintStream out) {
111 this.outputValues = outputValues;
112 this.outputJSON = outputJSON;
113 this.sequence = sequence;
114 this.region = region;
115 this.row = row;
116 this.persistentOutput = persistentOutput;
117 if (persistentOutput) {
118 beginPersistentOutput();
119 }
120 this.out = out;
121 this.firstTxn = true;
122 }
123
124
125
126
127 public void enableValues() {
128 outputValues = true;
129 }
130
131
132
133
134 public void disableValues() {
135 outputValues = false;
136 }
137
138
139
140
141 public void enableJSON() {
142 outputJSON = true;
143 }
144
145
146
147
148 public void disableJSON() {
149 outputJSON = false;
150 }
151
152
153
154
155
156
157
158
159 public void setSequenceFilter(long sequence) {
160 this.sequence = sequence;
161 }
162
163
164
165
166
167
168
169
170 public void setRegionFilter(String region) {
171 this.region = region;
172 }
173
174
175
176
177
178
179
180
181 public void setRowFilter(String row) {
182 this.row = row;
183 }
184
185
186
187
188
189 public void beginPersistentOutput() {
190 if (persistentOutput)
191 return;
192 persistentOutput = true;
193 firstTxn = true;
194 if (outputJSON)
195 out.print("[");
196 }
197
198
199
200
201
202 public void endPersistentOutput() {
203 if (!persistentOutput)
204 return;
205 persistentOutput = false;
206 if (outputJSON)
207 out.print("]");
208 }
209
210
211
212
213
214
215
216
217
218
219
220
221
222 public void processFile(final Configuration conf, final Path p)
223 throws IOException {
224 FileSystem fs = FileSystem.get(conf);
225 if (!fs.exists(p)) {
226 throw new FileNotFoundException(p.toString());
227 }
228 if (!fs.isFile(p)) {
229 throw new IOException(p + " is not a file");
230 }
231 if (outputJSON && !persistentOutput) {
232 out.print("[");
233 firstTxn = true;
234 }
235 Reader log = HLog.getReader(fs, p, conf);
236 try {
237 HLog.Entry entry;
238 while ((entry = log.next()) != null) {
239 HLogKey key = entry.getKey();
240 WALEdit edit = entry.getEdit();
241
242 JSONObject txn = new JSONObject(key.toStringMap());
243
244 if (sequence >= 0 && ((Long) txn.get("sequence")) != sequence)
245 continue;
246 if (region != null && !((String) txn.get("region")).equals(region))
247 continue;
248
249 JSONArray actions = new JSONArray();
250 for (KeyValue kv : edit.getKeyValues()) {
251
252 JSONObject op = new JSONObject(kv.toStringMap());
253 if (outputValues)
254 op.put("value", Bytes.toStringBinary(kv.getValue()));
255 if (row == null || ((String) op.get("row")).equals(row))
256 actions.put(op);
257 }
258 if (actions.length() == 0)
259 continue;
260 txn.put("actions", actions);
261 if (outputJSON) {
262
263 if (firstTxn)
264 firstTxn = false;
265 else
266 out.print(",");
267 out.print(txn);
268 } else {
269
270 out.println("Sequence " + txn.getLong("sequence") + " "
271 + "from region " + txn.getString("region") + " " + "in table "
272 + txn.getString("table"));
273 for (int i = 0; i < actions.length(); i++) {
274 JSONObject op = actions.getJSONObject(i);
275 out.println(" Action:");
276 out.println(" row: " + op.getString("row"));
277 out.println(" column: " + op.getString("family") + ":"
278 + op.getString("qualifier"));
279 out.println(" at time: "
280 + (new Date(op.getLong("timestamp"))));
281 if (outputValues)
282 out.println(" value: " + op.get("value"));
283 }
284 }
285 }
286 } catch (JSONException e) {
287 e.printStackTrace();
288 } finally {
289 log.close();
290 }
291 if (outputJSON && !persistentOutput) {
292 out.print("]");
293 }
294 }
295
296
297
298
299
300
301
302
303
304
305
306
307 public static void run(String[] args) throws IOException {
308
309 Options options = new Options();
310 options.addOption("h", "help", false, "Output help message");
311 options.addOption("j", "json", false, "Output JSON");
312 options.addOption("p", "printvals", false, "Print values");
313 options.addOption("r", "region", true,
314 "Region to filter by. Pass region name; e.g. '.META.,,1'");
315 options.addOption("s", "sequence", true,
316 "Sequence to filter by. Pass sequence number.");
317 options.addOption("w", "row", true, "Row to filter by. Pass row name.");
318
319 HLogPrettyPrinter printer = new HLogPrettyPrinter();
320 CommandLineParser parser = new PosixParser();
321 List files = null;
322 try {
323 CommandLine cmd = parser.parse(options, args);
324 files = cmd.getArgList();
325 if (files.size() == 0 || cmd.hasOption("h")) {
326 HelpFormatter formatter = new HelpFormatter();
327 formatter.printHelp("HFile filename(s) ", options, true);
328 System.exit(-1);
329 }
330
331 if (cmd.hasOption("p"))
332 printer.enableValues();
333 if (cmd.hasOption("j"))
334 printer.enableJSON();
335 if (cmd.hasOption("r"))
336 printer.setRegionFilter(cmd.getOptionValue("r"));
337 if (cmd.hasOption("s"))
338 printer.setSequenceFilter(Long.parseLong(cmd.getOptionValue("s")));
339 if (cmd.hasOption("w"))
340 printer.setRowFilter(cmd.getOptionValue("w"));
341 } catch (ParseException e) {
342 e.printStackTrace();
343 HelpFormatter formatter = new HelpFormatter();
344 formatter.printHelp("HFile filename(s) ", options, true);
345 System.exit(-1);
346 }
347
348 Configuration conf = HBaseConfiguration.create();
349 conf.set("fs.defaultFS",
350 conf.get(org.apache.hadoop.hbase.HConstants.HBASE_DIR));
351 conf.set("fs.default.name",
352 conf.get(org.apache.hadoop.hbase.HConstants.HBASE_DIR));
353
354 printer.beginPersistentOutput();
355 for (Object f : files) {
356 Path file = new Path((String) f);
357 FileSystem fs = file.getFileSystem(conf);
358 if (!fs.exists(file)) {
359 System.err.println("ERROR, file doesnt exist: " + file);
360 return;
361 }
362 printer.processFile(conf, file);
363 }
364 printer.endPersistentOutput();
365 }
366 }