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