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.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
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 ObjectMapper mapper;
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 mapper = new ObjectMapper();
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
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
135
136 public void enableValues() {
137 outputValues = true;
138 }
139
140
141
142
143 public void disableValues() {
144 outputValues = false;
145 }
146
147
148
149
150 public void enableJSON() {
151 outputJSON = true;
152 }
153
154
155
156
157 public void disableJSON() {
158 outputJSON = false;
159 }
160
161
162
163
164
165
166
167
168 public void setSequenceFilter(long sequence) {
169 this.sequence = sequence;
170 }
171
172
173
174
175
176
177
178
179 public void setRegionFilter(String region) {
180 this.region = region;
181 }
182
183
184
185
186
187
188
189
190 public void setRowFilter(String row) {
191 this.row = row;
192 }
193
194
195
196
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
209
210
211 public void endPersistentOutput() {
212 if (!persistentOutput)
213 return;
214 persistentOutput = false;
215 if (outputJSON)
216 out.print("]");
217 }
218
219
220
221
222
223
224
225
226
227
228
229
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
251 Map<String, Object> txn = key.toStringMap();
252 long writeTime = key.getWriteTime();
253
254 if (sequence >= 0 && ((Long) txn.get("sequence")) != sequence)
255 continue;
256 if (region != null && !((String) txn.get("region")).equals(region))
257 continue;
258
259 List<Map> actions = new ArrayList<Map>();
260 for (KeyValue kv : edit.getKeyValues()) {
261
262 Map<String, Object> op =
263 new HashMap<String, Object>(kv.toStringMap());
264 if (outputValues)
265 op.put("value", Bytes.toStringBinary(kv.getValue()));
266
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
275 if (firstTxn)
276 firstTxn = false;
277 else
278 out.print(",");
279
280 out.print(mapper.writeValueAsString(txn));
281 } else {
282
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
313
314
315
316
317
318
319
320
321
322 public static void run(String[] args) throws IOException {
323
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
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
363 Configuration conf = HBaseConfiguration.create();
364 FSUtils.setFsDefault(conf, FSUtils.getRootDir(conf));
365
366
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 }