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