1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.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.Iterator;
27 import java.util.List;
28 import java.util.Map;
29
30 import org.apache.commons.cli.CommandLine;
31 import org.apache.commons.cli.CommandLineParser;
32 import org.apache.commons.cli.HelpFormatter;
33 import org.apache.commons.cli.Options;
34 import org.apache.commons.cli.ParseException;
35 import org.apache.commons.cli.PosixParser;
36 import org.apache.hadoop.hbase.classification.InterfaceAudience;
37 import org.apache.hadoop.hbase.classification.InterfaceStability;
38 import org.apache.hadoop.conf.Configuration;
39 import org.apache.hadoop.fs.FileSystem;
40 import org.apache.hadoop.fs.Path;
41 import org.apache.hadoop.hbase.Cell;
42 import org.apache.hadoop.hbase.CellUtil;
43 import org.apache.hadoop.hbase.HBaseConfiguration;
44 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
45 import org.apache.hadoop.hbase.Tag;
46 import org.apache.hadoop.hbase.util.Bytes;
47 import org.apache.hadoop.hbase.util.FSUtils;
48 import org.codehaus.jackson.map.ObjectMapper;
49
50 import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
51
52 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
68 @InterfaceStability.Evolving
69 public class WALPrettyPrinter {
70 private boolean outputValues;
71 private boolean outputJSON;
72
73 private long sequence;
74 private String region;
75 private String row;
76
77 private boolean persistentOutput;
78 private boolean firstTxn;
79
80 private PrintStream out;
81
82 private static final ObjectMapper MAPPER = new ObjectMapper();
83
84 private long position;
85
86
87
88
89 public WALPrettyPrinter() {
90 outputValues = false;
91 outputJSON = false;
92 sequence = -1;
93 region = null;
94 row = null;
95 persistentOutput = false;
96 firstTxn = true;
97 out = System.out;
98 }
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125 public WALPrettyPrinter(boolean outputValues, boolean outputJSON,
126 long sequence, String region, String row, boolean
127 persistentOutput,
128 PrintStream out) {
129 this.outputValues = outputValues;
130 this.outputJSON = outputJSON;
131 this.sequence = sequence;
132 this.region = region;
133 this.row = row;
134 this.persistentOutput = persistentOutput;
135 if (persistentOutput) {
136 beginPersistentOutput();
137 }
138 this.out = out;
139 this.firstTxn = true;
140 }
141
142
143
144
145 public void enableValues() {
146 outputValues = true;
147 }
148
149
150
151
152 public void disableValues() {
153 outputValues = false;
154 }
155
156
157
158
159 public void enableJSON() {
160 outputJSON = true;
161 }
162
163
164
165
166 public void disableJSON() {
167 outputJSON = false;
168 }
169
170
171
172
173
174
175
176
177 public void setSequenceFilter(long sequence) {
178 this.sequence = sequence;
179 }
180
181
182
183
184
185
186
187
188 public void setRegionFilter(String region) {
189 this.region = region;
190 }
191
192
193
194
195
196
197
198
199 public void setRowFilter(String row) {
200 this.row = row;
201 }
202
203
204
205
206
207 public void beginPersistentOutput() {
208 if (persistentOutput)
209 return;
210 persistentOutput = true;
211 firstTxn = true;
212 if (outputJSON)
213 out.print("[");
214 }
215
216
217
218
219
220 public void endPersistentOutput() {
221 if (!persistentOutput)
222 return;
223 persistentOutput = false;
224 if (outputJSON)
225 out.print("]");
226 }
227
228
229
230
231
232
233
234
235
236
237
238
239
240 public void processFile(final Configuration conf, final Path p)
241 throws IOException {
242 FileSystem fs = p.getFileSystem(conf);
243 if (!fs.exists(p)) {
244 throw new FileNotFoundException(p.toString());
245 }
246 if (!fs.isFile(p)) {
247 throw new IOException(p + " is not a file");
248 }
249
250 WAL.Reader log = WALFactory.createReader(fs, p, conf);
251
252 if (log instanceof ProtobufLogReader) {
253 List<String> writerClsNames = ((ProtobufLogReader) log).getWriterClsNames();
254 if (writerClsNames != null && writerClsNames.size() > 0) {
255 out.print("Writer Classes: ");
256 for (int i = 0; i < writerClsNames.size(); i++) {
257 out.print(writerClsNames.get(i));
258 if (i != writerClsNames.size() - 1) {
259 out.print(" ");
260 }
261 }
262 out.println();
263 }
264
265 String cellCodecClsName = ((ProtobufLogReader) log).getCodecClsName();
266 if (cellCodecClsName != null) {
267 out.println("Cell Codec Class: " + cellCodecClsName);
268 }
269 }
270
271 if (outputJSON && !persistentOutput) {
272 out.print("[");
273 firstTxn = true;
274 }
275
276 if (position > 0) {
277 log.seek(position);
278 }
279
280 try {
281 WAL.Entry entry;
282 while ((entry = log.next()) != null) {
283 WALKey key = entry.getKey();
284 WALEdit edit = entry.getEdit();
285
286 Map<String, Object> txn = key.toStringMap();
287 long writeTime = key.getWriteTime();
288
289 if (sequence >= 0 && ((Long) txn.get("sequence")) != sequence)
290 continue;
291 if (region != null && !((String) txn.get("region")).equals(region))
292 continue;
293
294 List<Map> actions = new ArrayList<Map>();
295 for (Cell cell : edit.getCells()) {
296
297 Map<String, Object> op = new HashMap<String, Object>(toStringMap(cell));
298 if (outputValues) op.put("value", Bytes.toStringBinary(cell.getValue()));
299
300 if (row == null || ((String) op.get("row")).equals(row)) {
301 actions.add(op);
302 }
303 op.put("total_size_sum", CellUtil.estimatedHeapSizeOf(cell));
304
305 }
306 if (actions.size() == 0)
307 continue;
308 txn.put("actions", actions);
309 if (outputJSON) {
310
311 if (firstTxn)
312 firstTxn = false;
313 else
314 out.print(",");
315
316 out.print(MAPPER.writeValueAsString(txn));
317 } else {
318
319 out.println("Sequence=" + txn.get("sequence") + " "
320 + ", region=" + txn.get("region") + " at write timestamp=" + new Date(writeTime));
321 for (int i = 0; i < actions.size(); i++) {
322 Map op = actions.get(i);
323 out.println("row=" + op.get("row") +
324 ", column=" + op.get("family") + ":" + op.get("qualifier"));
325 if (op.get("tag") != null) {
326 out.println(" tag: " + op.get("tag"));
327 }
328 if (outputValues) out.println(" value: " + op.get("value"));
329 out.println("cell total size sum: " + op.get("total_size_sum"));
330 }
331 }
332 out.println("edit heap size: " + entry.getEdit().heapSize());
333 out.println("position: " + log.getPosition());
334 }
335 } finally {
336 log.close();
337 }
338 if (outputJSON && !persistentOutput) {
339 out.print("]");
340 }
341 }
342
343 private static Map<String, Object> toStringMap(Cell cell) {
344 Map<String, Object> stringMap = new HashMap<String, Object>();
345 stringMap.put("row",
346 Bytes.toStringBinary(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()));
347 stringMap.put("family", Bytes.toStringBinary(cell.getFamilyArray(), cell.getFamilyOffset(),
348 cell.getFamilyLength()));
349 stringMap.put("qualifier",
350 Bytes.toStringBinary(cell.getQualifierArray(), cell.getQualifierOffset(),
351 cell.getQualifierLength()));
352 stringMap.put("timestamp", cell.getTimestamp());
353 stringMap.put("vlen", cell.getValueLength());
354 if (cell.getTagsLength() > 0) {
355 List<String> tagsString = new ArrayList<String>();
356 Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
357 cell.getTagsLength());
358 while (tagsIterator.hasNext()) {
359 Tag tag = tagsIterator.next();
360 tagsString.add((tag.getType()) + ":"
361 + Bytes.toStringBinary(tag.getBuffer(), tag.getTagOffset(), tag.getTagLength()));
362 }
363 stringMap.put("tag", tagsString);
364 }
365 return stringMap;
366 }
367
368 public static void main(String[] args) throws IOException {
369 run(args);
370 }
371
372
373
374
375
376
377
378
379
380
381 public static void run(String[] args) throws IOException {
382
383 Options options = new Options();
384 options.addOption("h", "help", false, "Output help message");
385 options.addOption("j", "json", false, "Output JSON");
386 options.addOption("p", "printvals", false, "Print values");
387 options.addOption("r", "region", true,
388 "Region to filter by. Pass encoded region name; e.g. '9192caead6a5a20acb4454ffbc79fa14'");
389 options.addOption("s", "sequence", true,
390 "Sequence to filter by. Pass sequence number.");
391 options.addOption("w", "row", true, "Row to filter by. Pass row name.");
392 options.addOption("g", "goto", true, "Position to seek to in the file");
393
394 WALPrettyPrinter printer = new WALPrettyPrinter();
395 CommandLineParser parser = new PosixParser();
396 List<?> files = null;
397 try {
398 CommandLine cmd = parser.parse(options, args);
399 files = cmd.getArgList();
400 if (files.size() == 0 || cmd.hasOption("h")) {
401 HelpFormatter formatter = new HelpFormatter();
402 formatter.printHelp("WAL <filename...>", options, true);
403 System.exit(-1);
404 }
405
406 if (cmd.hasOption("p"))
407 printer.enableValues();
408 if (cmd.hasOption("j"))
409 printer.enableJSON();
410 if (cmd.hasOption("r"))
411 printer.setRegionFilter(cmd.getOptionValue("r"));
412 if (cmd.hasOption("s"))
413 printer.setSequenceFilter(Long.parseLong(cmd.getOptionValue("s")));
414 if (cmd.hasOption("w"))
415 printer.setRowFilter(cmd.getOptionValue("w"));
416 if (cmd.hasOption("g")) {
417 printer.setPosition(Long.parseLong(cmd.getOptionValue("g")));
418 }
419 } catch (ParseException e) {
420 e.printStackTrace();
421 HelpFormatter formatter = new HelpFormatter();
422 formatter.printHelp("HFile filename(s) ", options, true);
423 System.exit(-1);
424 }
425
426 Configuration conf = HBaseConfiguration.create();
427 FSUtils.setFsDefault(conf, FSUtils.getRootDir(conf));
428
429
430 printer.beginPersistentOutput();
431 for (Object f : files) {
432 Path file = new Path((String) f);
433 FileSystem fs = file.getFileSystem(conf);
434 if (!fs.exists(file)) {
435 System.err.println("ERROR, file doesnt exist: " + file);
436 return;
437 }
438 printer.processFile(conf, file);
439 }
440 printer.endPersistentOutput();
441 }
442
443 public void setPosition(long position) {
444 this.position = position;
445 }
446 }