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.Collections;
25 import java.util.Date;
26 import java.util.HashMap;
27 import java.util.HashSet;
28 import java.util.Iterator;
29 import java.util.List;
30 import java.util.Map;
31 import java.util.Set;
32 import org.apache.commons.cli.CommandLine;
33 import org.apache.commons.cli.CommandLineParser;
34 import org.apache.commons.cli.HelpFormatter;
35 import org.apache.commons.cli.Options;
36 import org.apache.commons.cli.ParseException;
37 import org.apache.commons.cli.PosixParser;
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.KeyValue;
46 import org.apache.hadoop.hbase.Tag;
47 import org.apache.hadoop.hbase.classification.InterfaceAudience;
48 import org.apache.hadoop.hbase.classification.InterfaceStability;
49 import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
50 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
51 import org.apache.hadoop.hbase.util.Bytes;
52 import org.apache.hadoop.hbase.util.FSUtils;
53 import org.apache.hadoop.hbase.util.GsonUtil;
54 import org.apache.hadoop.hbase.util.Strings;
55 import org.apache.hbase.thirdparty.com.google.gson.Gson;
56 import org.slf4j.Logger;
57 import org.slf4j.LoggerFactory;
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
74 @InterfaceStability.Evolving
75 public class WALPrettyPrinter {
76 private static final Logger LOG = LoggerFactory.getLogger(WALPrettyPrinter.class);
77
78
79 private static final String outputTmpl =
80 "Sequence=%s, table=%s, region=%s, at write timestamp=%s";
81
82 private boolean outputValues;
83 private boolean outputJSON;
84
85 private long sequence;
86
87
88 private final Set<String> tableSet;
89 private String region;
90
91
92 private String row;
93
94 private String rowPrefix;
95 private boolean outputOnlyRowKey;
96
97 private boolean persistentOutput;
98 private boolean firstTxn;
99
100 private PrintStream out;
101
102 private static final Gson GSON = GsonUtil.createGson().create();
103
104 private long position;
105
106
107
108
109 public WALPrettyPrinter() {
110 this(false, false, -1, new HashSet<String>(), null, null, null, false, false, System.out);
111 }
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144 public WALPrettyPrinter(boolean outputValues, boolean outputJSON, long sequence,
145 Set<String> tableSet, String region, String row, String rowPrefix, boolean outputOnlyRowKey,
146 boolean persistentOutput, PrintStream out) {
147 this.outputValues = outputValues;
148 this.outputJSON = outputJSON;
149 this.sequence = sequence;
150 this.tableSet = tableSet;
151 this.region = region;
152 this.row = row;
153 this.rowPrefix = rowPrefix;
154 this.outputOnlyRowKey = outputOnlyRowKey;
155 this.persistentOutput = persistentOutput;
156 if (persistentOutput) {
157 beginPersistentOutput();
158 }
159 this.out = out;
160 this.firstTxn = true;
161 }
162
163
164
165
166 public void enableValues() {
167 outputValues = true;
168 }
169
170
171
172
173 public void disableValues() {
174 outputValues = false;
175 }
176
177
178
179
180 public void enableJSON() {
181 outputJSON = true;
182 }
183
184
185
186
187 public void disableJSON() {
188 outputJSON = false;
189 }
190
191
192
193
194
195
196
197
198 public void setSequenceFilter(long sequence) {
199 this.sequence = sequence;
200 }
201
202
203
204
205
206 public void setTableFilter(String tablesWithDelimiter) {
207 Collections.addAll(tableSet, tablesWithDelimiter.split(","));
208 }
209
210
211
212
213
214
215
216 public void setRegionFilter(String region) {
217 this.region = region;
218 }
219
220
221
222
223
224
225
226
227 public void setRowFilter(String row) {
228 this.row = row;
229 }
230
231
232
233
234
235
236
237
238 public void setRowPrefixFilter(String rowPrefix) {
239 this.rowPrefix = rowPrefix;
240 }
241
242
243
244
245 public void setOutputOnlyRowKey() {
246 this.outputOnlyRowKey = true;
247 }
248
249
250
251
252
253
254 public void setPosition(long position) {
255 this.position = position;
256 }
257
258
259
260
261
262 public void beginPersistentOutput() {
263 if (persistentOutput) {
264 return;
265 }
266 persistentOutput = true;
267 firstTxn = true;
268 if (outputJSON) {
269 out.print("[");
270 }
271 }
272
273
274
275
276
277 public void endPersistentOutput() {
278 if (!persistentOutput) {
279 return;
280 }
281 persistentOutput = false;
282 if (outputJSON) {
283 out.print("]");
284 }
285 }
286
287
288
289
290
291
292
293
294
295
296
297
298
299 public void processFile(final Configuration conf, final Path p)
300 throws IOException {
301 FileSystem fs = p.getFileSystem(conf);
302 if (!fs.exists(p)) {
303 throw new FileNotFoundException(p.toString());
304 }
305 if (!fs.isFile(p)) {
306 throw new IOException(p + " is not a file");
307 }
308
309 WAL.Reader log = WALFactory.createReader(fs, p, conf);
310
311 if (log instanceof ProtobufLogReader) {
312 List<String> writerClsNames = ((ProtobufLogReader) log).getWriterClsNames();
313 if (writerClsNames != null && writerClsNames.size() > 0) {
314 out.print("Writer Classes: ");
315 for (int i = 0; i < writerClsNames.size(); i++) {
316 out.print(writerClsNames.get(i));
317 if (i != writerClsNames.size() - 1) {
318 out.print(" ");
319 }
320 }
321 out.println();
322 }
323
324 String cellCodecClsName = ((ProtobufLogReader) log).getCodecClsName();
325 if (cellCodecClsName != null) {
326 out.println("Cell Codec Class: " + cellCodecClsName);
327 }
328 }
329
330 if (outputJSON && !persistentOutput) {
331 out.print("[");
332 firstTxn = true;
333 }
334
335 if (position > 0) {
336 log.seek(position);
337 }
338
339 try {
340 WAL.Entry entry;
341 while ((entry = log.next()) != null) {
342 WALKey key = entry.getKey();
343 WALEdit edit = entry.getEdit();
344
345 Map<String, Object> txn = key.toStringMap();
346 long writeTime = key.getWriteTime();
347
348 if (!tableSet.isEmpty() &&
349 !tableSet.contains(txn.get("table").toString())) {
350 continue;
351 }
352 if (sequence >= 0 && ((Long) txn.get("sequence")) != sequence) {
353 continue;
354 }
355 if (region != null && !txn.get("region").equals(region)) {
356 continue;
357 }
358
359 List<Map<String, Object>> actions = new ArrayList<>();
360 for (Cell cell : edit.getCells()) {
361
362 Map<String, Object> op =
363 new HashMap<>(toStringMap(cell, outputOnlyRowKey, rowPrefix, row, outputValues));
364 if (op.isEmpty()) {
365 continue;
366 }
367 actions.add(op);
368 }
369 if (actions.isEmpty()) {
370 continue;
371 }
372 txn.put("actions", actions);
373 if (outputJSON) {
374
375 if (firstTxn) {
376 firstTxn = false;
377 } else {
378 out.print(",");
379 }
380
381 out.print(GSON.toJson(txn));
382 } else {
383
384 if (!outputOnlyRowKey) {
385 out.println(String.format(outputTmpl,
386 txn.get("sequence"), txn.get("table"), txn.get("region"), new Date(writeTime)));
387 }
388 for (int i = 0; i < actions.size(); i++) {
389 Map<String, Object> op = actions.get(i);
390 printCell(out, op, outputValues, outputOnlyRowKey);
391 }
392 }
393 if (!outputOnlyRowKey) {
394 out.println("edit heap size: " + entry.getEdit().heapSize());
395 out.println("position: " + log.getPosition());
396
397 }
398 }
399 } finally {
400 log.close();
401 }
402 if (outputJSON && !persistentOutput) {
403 out.print("]");
404 }
405 }
406
407 public static void printCell(PrintStream out, Map<String, Object> op,
408 boolean outputValues, boolean outputOnlyRowKey) {
409 String rowDetails = "row=" + op.get("row");
410 if (outputOnlyRowKey) {
411 out.println(rowDetails);
412 return;
413 }
414
415 rowDetails += ", column=" + op.get("family") + ":" + op.get("qualifier");
416 rowDetails += ", type=" + op.get("type");
417 out.println(rowDetails);
418 if (op.get("tag") != null) {
419 out.println(" tag: " + op.get("tag"));
420 }
421 if (outputValues) {
422 out.println(" value: " + op.get("value"));
423 }
424 out.println("cell total size sum: " + op.get("total_size_sum"));
425 }
426
427 public static Map<String, Object> toStringMap(Cell cell,
428 boolean printRowKeyOnly, String rowPrefix, String row, boolean outputValues) {
429 Map<String, Object> stringMap = new HashMap<>();
430 String rowKey = Bytes.toStringBinary(cell.getRowArray(),
431 cell.getRowOffset(), cell.getRowLength());
432
433
434
435 if ((!Strings.isEmpty(rowPrefix) && !rowKey.startsWith(rowPrefix)) ||
436 (!Strings.isEmpty(row) && !rowKey.equals(row))) {
437 return stringMap;
438 }
439
440 stringMap.put("row", rowKey);
441 if (printRowKeyOnly) {
442 return stringMap;
443 }
444 stringMap.put("type", KeyValue.Type.codeToType(cell.getTypeByte()));
445 stringMap.put("family",
446 Bytes.toStringBinary(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength()));
447 stringMap.put("qualifier", Bytes
448 .toStringBinary(cell.getQualifierArray(), cell.getQualifierOffset(),
449 cell.getQualifierLength()));
450 stringMap.put("timestamp", cell.getTimestamp());
451 stringMap.put("vlen", cell.getValueLength());
452 stringMap.put("total_size_sum", CellUtil.estimatedHeapSizeOf(cell));
453 if (cell.getTagsLength() > 0) {
454 List<String> tagsString = new ArrayList<>();
455 Iterator<Tag> tagsIterator =
456 CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength());
457 while (tagsIterator.hasNext()) {
458 Tag tag = tagsIterator.next();
459 tagsString.add((tag.getType()) + ":" + Bytes
460 .toStringBinary(tag.getBuffer(), tag.getTagOffset(), tag.getTagLength()));
461 }
462 stringMap.put("tag", tagsString);
463 }
464 if (outputValues) {
465 stringMap.put("value", Bytes.toStringBinary(CellUtil.cloneValue(cell)));
466 }
467 return stringMap;
468 }
469
470 public static Map<String, Object> toStringMap(Cell cell) {
471 return toStringMap(cell, false, null, null, false);
472 }
473
474 public static void main(String[] args) throws IOException {
475 run(args);
476 }
477
478
479
480
481
482
483
484
485
486
487 public static void run(String[] args) throws IOException {
488
489 Options options = new Options();
490 options.addOption("h", "help", false, "Output help message");
491 options.addOption("j", "json", false, "Output JSON");
492 options.addOption("p", "printvals", false, "Print values");
493 options.addOption("t", "tables", true,
494 "Table names (comma separated) to filter by; eg: test1,test2,test3 ");
495 options.addOption("r", "region", true,
496 "Region to filter by. Pass encoded region name; e.g. '9192caead6a5a20acb4454ffbc79fa14'");
497 options.addOption("s", "sequence", true,
498 "Sequence to filter by. Pass sequence number.");
499 options.addOption("k", "outputOnlyRowKey", false,
500 "Print only row keys");
501 options.addOption("w", "row", true, "Row to filter by. Pass row name.");
502 options.addOption("f", "rowPrefix", true, "Row prefix to filter by.");
503 options.addOption("g", "goto", true, "Position to seek to in the file");
504
505 WALPrettyPrinter printer = new WALPrettyPrinter();
506 CommandLineParser parser = new PosixParser();
507 List<?> files = null;
508 try {
509 CommandLine cmd = parser.parse(options, args);
510 files = cmd.getArgList();
511 if (files.size() == 0 || cmd.hasOption("h")) {
512 HelpFormatter formatter = new HelpFormatter();
513 formatter.printHelp("WAL <filename...>", options, true);
514 System.exit(-1);
515 }
516
517 if (cmd.hasOption("p")) {
518 printer.enableValues();
519 }
520 if (cmd.hasOption("j")) {
521 printer.enableJSON();
522 }
523 if (cmd.hasOption("k")) {
524 printer.setOutputOnlyRowKey();
525 }
526 if (cmd.hasOption("t")) {
527 printer.setTableFilter(cmd.getOptionValue("t"));
528 }
529 if (cmd.hasOption("r")) {
530 printer.setRegionFilter(cmd.getOptionValue("r"));
531 }
532 if (cmd.hasOption("s")) {
533 printer.setSequenceFilter(Long.parseLong(cmd.getOptionValue("s")));
534 }
535 if (cmd.hasOption("w")) {
536 if (cmd.hasOption("f")) {
537 throw new ParseException("Row and Row-prefix cannot be supplied together");
538 }
539 printer.setRowFilter(cmd.getOptionValue("w"));
540 }
541 if (cmd.hasOption("f")) {
542 if (cmd.hasOption("w")) {
543 throw new ParseException("Row and Row-prefix cannot be supplied together");
544 }
545 printer.setRowPrefixFilter(cmd.getOptionValue("f"));
546 }
547 if (cmd.hasOption("g")) {
548 printer.setPosition(Long.parseLong(cmd.getOptionValue("g")));
549 }
550 } catch (ParseException e) {
551 LOG.error("Failed to parse commandLine arguments", e);
552 HelpFormatter formatter = new HelpFormatter();
553 formatter.printHelp("HFile filename(s) ", options, true);
554 System.exit(-1);
555 }
556
557 Configuration conf = HBaseConfiguration.create();
558 FSUtils.setFsDefault(conf, FSUtils.getRootDir(conf));
559
560
561 printer.beginPersistentOutput();
562 for (Object f : files) {
563 Path file = new Path((String) f);
564 FileSystem fs = file.getFileSystem(conf);
565 if (!fs.exists(file)) {
566 System.err.println("ERROR, file doesnt exist: " + file);
567 return;
568 }
569 printer.processFile(conf, file);
570 }
571 printer.endPersistentOutput();
572 }
573 }