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
85
86
87 public WALPrettyPrinter() {
88 outputValues = false;
89 outputJSON = false;
90 sequence = -1;
91 region = null;
92 row = null;
93 persistentOutput = false;
94 firstTxn = true;
95 out = System.out;
96 }
97
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 public WALPrettyPrinter(boolean outputValues, boolean outputJSON,
124 long sequence, String region, String row, boolean persistentOutput,
125 PrintStream out) {
126 this.outputValues = outputValues;
127 this.outputJSON = outputJSON;
128 this.sequence = sequence;
129 this.region = region;
130 this.row = row;
131 this.persistentOutput = persistentOutput;
132 if (persistentOutput) {
133 beginPersistentOutput();
134 }
135 this.out = out;
136 this.firstTxn = true;
137 }
138
139
140
141
142 public void enableValues() {
143 outputValues = true;
144 }
145
146
147
148
149 public void disableValues() {
150 outputValues = false;
151 }
152
153
154
155
156 public void enableJSON() {
157 outputJSON = true;
158 }
159
160
161
162
163 public void disableJSON() {
164 outputJSON = false;
165 }
166
167
168
169
170
171
172
173
174 public void setSequenceFilter(long sequence) {
175 this.sequence = sequence;
176 }
177
178
179
180
181
182
183
184
185 public void setRegionFilter(String region) {
186 this.region = region;
187 }
188
189
190
191
192
193
194
195
196 public void setRowFilter(String row) {
197 this.row = row;
198 }
199
200
201
202
203
204 public void beginPersistentOutput() {
205 if (persistentOutput)
206 return;
207 persistentOutput = true;
208 firstTxn = true;
209 if (outputJSON)
210 out.print("[");
211 }
212
213
214
215
216
217 public void endPersistentOutput() {
218 if (!persistentOutput)
219 return;
220 persistentOutput = false;
221 if (outputJSON)
222 out.print("]");
223 }
224
225
226
227
228
229
230
231
232
233
234
235
236
237 public void processFile(final Configuration conf, final Path p)
238 throws IOException {
239 FileSystem fs = p.getFileSystem(conf);
240 if (!fs.exists(p)) {
241 throw new FileNotFoundException(p.toString());
242 }
243 if (!fs.isFile(p)) {
244 throw new IOException(p + " is not a file");
245 }
246
247 WAL.Reader log = WALFactory.createReader(fs, p, conf);
248
249 if (log instanceof ProtobufLogReader) {
250 List<String> writerClsNames = ((ProtobufLogReader) log).getWriterClsNames();
251 if (writerClsNames != null && writerClsNames.size() > 0) {
252 out.print("Writer Classes: ");
253 for (int i = 0; i < writerClsNames.size(); i++) {
254 out.print(writerClsNames.get(i));
255 if (i != writerClsNames.size() - 1) {
256 out.print(" ");
257 }
258 }
259 out.println();
260 }
261
262 String cellCodecClsName = ((ProtobufLogReader) log).getCodecClsName();
263 if (cellCodecClsName != null) {
264 out.println("Cell Codec Class: " + cellCodecClsName);
265 }
266 }
267
268 if (outputJSON && !persistentOutput) {
269 out.print("[");
270 firstTxn = true;
271 }
272
273 try {
274 WAL.Entry entry;
275 while ((entry = log.next()) != null) {
276 WALKey key = entry.getKey();
277 WALEdit edit = entry.getEdit();
278
279 Map<String, Object> txn = key.toStringMap();
280 long writeTime = key.getWriteTime();
281
282 if (sequence >= 0 && ((Long) txn.get("sequence")) != sequence)
283 continue;
284 if (region != null && !((String) txn.get("region")).equals(region))
285 continue;
286
287 List<Map> actions = new ArrayList<Map>();
288 for (Cell cell : edit.getCells()) {
289
290 Map<String, Object> op = new HashMap<String, Object>(toStringMap(cell));
291 if (outputValues) op.put("value", Bytes.toStringBinary(cell.getValue()));
292
293 if (row == null || ((String) op.get("row")).equals(row)) {
294 actions.add(op);
295 }
296 }
297 if (actions.size() == 0)
298 continue;
299 txn.put("actions", actions);
300 if (outputJSON) {
301
302 if (firstTxn)
303 firstTxn = false;
304 else
305 out.print(",");
306
307 out.print(MAPPER.writeValueAsString(txn));
308 } else {
309
310 out.println("Sequence=" + txn.get("sequence") + " "
311 + ", region=" + txn.get("region") + " at write timestamp=" + new Date(writeTime));
312 for (int i = 0; i < actions.size(); i++) {
313 Map op = actions.get(i);
314 out.println("row=" + op.get("row") +
315 ", column=" + op.get("family") + ":" + op.get("qualifier"));
316 if (op.get("tag") != null) {
317 out.println(" tag: " + op.get("tag"));
318 }
319 if (outputValues) out.println(" value: " + op.get("value"));
320 }
321 }
322 }
323 } finally {
324 log.close();
325 }
326 if (outputJSON && !persistentOutput) {
327 out.print("]");
328 }
329 }
330
331 private static Map<String, Object> toStringMap(Cell cell) {
332 Map<String, Object> stringMap = new HashMap<String, Object>();
333 stringMap.put("row",
334 Bytes.toStringBinary(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()));
335 stringMap.put("family", Bytes.toStringBinary(cell.getFamilyArray(), cell.getFamilyOffset(),
336 cell.getFamilyLength()));
337 stringMap.put("qualifier",
338 Bytes.toStringBinary(cell.getQualifierArray(), cell.getQualifierOffset(),
339 cell.getQualifierLength()));
340 stringMap.put("timestamp", cell.getTimestamp());
341 stringMap.put("vlen", cell.getValueLength());
342 if (cell.getTagsLength() > 0) {
343 List<String> tagsString = new ArrayList<String>();
344 Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
345 cell.getTagsLength());
346 while (tagsIterator.hasNext()) {
347 Tag tag = tagsIterator.next();
348 tagsString.add((tag.getType()) + ":"
349 + Bytes.toStringBinary(tag.getBuffer(), tag.getTagOffset(), tag.getTagLength()));
350 }
351 stringMap.put("tag", tagsString);
352 }
353 return stringMap;
354 }
355
356 public static void main(String[] args) throws IOException {
357 run(args);
358 }
359
360
361
362
363
364
365
366
367
368
369 public static void run(String[] args) throws IOException {
370
371 Options options = new Options();
372 options.addOption("h", "help", false, "Output help message");
373 options.addOption("j", "json", false, "Output JSON");
374 options.addOption("p", "printvals", false, "Print values");
375 options.addOption("r", "region", true,
376 "Region to filter by. Pass encoded region name; e.g. '9192caead6a5a20acb4454ffbc79fa14'");
377 options.addOption("s", "sequence", true,
378 "Sequence to filter by. Pass sequence number.");
379 options.addOption("w", "row", true, "Row to filter by. Pass row name.");
380
381 WALPrettyPrinter printer = new WALPrettyPrinter();
382 CommandLineParser parser = new PosixParser();
383 List<?> files = null;
384 try {
385 CommandLine cmd = parser.parse(options, args);
386 files = cmd.getArgList();
387 if (files.size() == 0 || cmd.hasOption("h")) {
388 HelpFormatter formatter = new HelpFormatter();
389 formatter.printHelp("WAL <filename...>", options, true);
390 System.exit(-1);
391 }
392
393 if (cmd.hasOption("p"))
394 printer.enableValues();
395 if (cmd.hasOption("j"))
396 printer.enableJSON();
397 if (cmd.hasOption("r"))
398 printer.setRegionFilter(cmd.getOptionValue("r"));
399 if (cmd.hasOption("s"))
400 printer.setSequenceFilter(Long.parseLong(cmd.getOptionValue("s")));
401 if (cmd.hasOption("w"))
402 printer.setRowFilter(cmd.getOptionValue("w"));
403 } catch (ParseException e) {
404 e.printStackTrace();
405 HelpFormatter formatter = new HelpFormatter();
406 formatter.printHelp("HFile filename(s) ", options, true);
407 System.exit(-1);
408 }
409
410 Configuration conf = HBaseConfiguration.create();
411 FSUtils.setFsDefault(conf, FSUtils.getRootDir(conf));
412
413
414 printer.beginPersistentOutput();
415 for (Object f : files) {
416 Path file = new Path((String) f);
417 FileSystem fs = file.getFileSystem(conf);
418 if (!fs.exists(file)) {
419 System.err.println("ERROR, file doesnt exist: " + file);
420 return;
421 }
422 printer.processFile(conf, file);
423 }
424 printer.endPersistentOutput();
425 }
426 }