001/** 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.wal; 019 020import java.io.FileNotFoundException; 021import java.io.IOException; 022import java.io.PrintStream; 023import java.util.ArrayList; 024import java.util.Collections; 025import java.util.Date; 026import java.util.HashMap; 027import java.util.HashSet; 028import java.util.Iterator; 029import java.util.List; 030import java.util.Map; 031import java.util.Set; 032import org.apache.hadoop.conf.Configuration; 033import org.apache.hadoop.fs.FileSystem; 034import org.apache.hadoop.fs.Path; 035import org.apache.hadoop.hbase.Cell; 036import org.apache.hadoop.hbase.CellUtil; 037import org.apache.hadoop.hbase.HBaseConfiguration; 038import org.apache.hadoop.hbase.HBaseInterfaceAudience; 039import org.apache.hadoop.hbase.PrivateCellUtil; 040import org.apache.hadoop.hbase.Tag; 041import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader; 042import org.apache.hadoop.hbase.util.Bytes; 043import org.apache.hadoop.hbase.util.CommonFSUtils; 044import org.apache.hadoop.hbase.util.GsonUtil; 045import org.apache.yetus.audience.InterfaceAudience; 046import org.apache.yetus.audience.InterfaceStability; 047import org.slf4j.Logger; 048import org.slf4j.LoggerFactory; 049import org.apache.hbase.thirdparty.com.google.common.base.Strings; 050import org.apache.hbase.thirdparty.com.google.gson.Gson; 051import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; 052import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLineParser; 053import org.apache.hbase.thirdparty.org.apache.commons.cli.HelpFormatter; 054import org.apache.hbase.thirdparty.org.apache.commons.cli.Options; 055import org.apache.hbase.thirdparty.org.apache.commons.cli.ParseException; 056import org.apache.hbase.thirdparty.org.apache.commons.cli.PosixParser; 057 058/** 059 * WALPrettyPrinter prints the contents of a given WAL with a variety of 060 * options affecting formatting and extent of content. 061 * 062 * It targets two usage cases: pretty printing for ease of debugging directly by 063 * humans, and JSON output for consumption by monitoring and/or maintenance 064 * scripts. 065 * 066 * It can filter by row, region, or sequence id. 067 * 068 * It can also toggle output of values. 069 * 070 */ 071@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS) 072@InterfaceStability.Evolving 073public class WALPrettyPrinter { 074 private static final Logger LOG = LoggerFactory.getLogger(WALPrettyPrinter.class); 075 076 // Output template for pretty printing. 077 private static final String outputTmpl = 078 "Sequence=%s, table=%s, region=%s, at write timestamp=%s"; 079 080 private boolean outputValues; 081 private boolean outputJSON; 082 // The following enable filtering by sequence, region, and row, respectively 083 private long sequence; 084 085 // List of tables for filter 086 private final Set<String> tableSet; 087 private String region; 088 089 // exact row which needs to be filtered 090 private String row; 091 // prefix of rows which needs to be filtered 092 private String rowPrefix; 093 094 private boolean outputOnlyRowKey; 095 // enable in order to output a single list of transactions from several files 096 private boolean persistentOutput; 097 private boolean firstTxn; 098 // useful for programmatic capture of JSON output 099 private PrintStream out; 100 // for JSON encoding 101 private static final Gson GSON = GsonUtil.createGson().create(); 102 //allows for jumping straight to a given portion of the file 103 private long position; 104 105 /** 106 * Basic constructor that simply initializes values to reasonable defaults. 107 */ 108 public WALPrettyPrinter() { 109 this(false, false, -1, new HashSet<>(), null, 110 null, null, false, false, System.out); 111 } 112 113 /** 114 * Fully specified constructor. 115 * 116 * @param outputValues 117 * when true, enables output of values along with other log 118 * information 119 * @param outputJSON 120 * when true, enables output in JSON format rather than a 121 * "pretty string" 122 * @param sequence 123 * when nonnegative, serves as a filter; only log entries with this 124 * sequence id will be printed 125 * @param tableSet 126 * when non null, serves as a filter. only entries corresponding to tables 127 * in the tableSet are printed 128 * @param region 129 * when not null, serves as a filter; only log entries from this 130 * region will be printed 131 * @param row 132 * when not null, serves as a filter; only log entries from this row 133 * will be printed 134 * @param rowPrefix 135 * when not null, serves as a filter; only log entries with row key 136 * having this prefix will be printed 137 * @param persistentOutput 138 * keeps a single list running for multiple files. if enabled, the 139 * endPersistentOutput() method must be used! 140 * @param out 141 * Specifies an alternative to stdout for the destination of this 142 * PrettyPrinter's output. 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 * turns value output on 165 */ 166 public void enableValues() { 167 outputValues = true; 168 } 169 170 /** 171 * turns value output off 172 */ 173 public void disableValues() { 174 outputValues = false; 175 } 176 177 /** 178 * turns JSON output on 179 */ 180 public void enableJSON() { 181 outputJSON = true; 182 } 183 184 /** 185 * turns JSON output off, and turns on "pretty strings" for human consumption 186 */ 187 public void disableJSON() { 188 outputJSON = false; 189 } 190 191 /** 192 * sets the region by which output will be filtered 193 * 194 * @param sequence 195 * when nonnegative, serves as a filter; only log entries with this 196 * sequence id will be printed 197 */ 198 public void setSequenceFilter(long sequence) { 199 this.sequence = sequence; 200 } 201 202 /** 203 * Sets the tables filter. Only log entries for these tables are printed. 204 * @param tablesWithDelimiter table names separated with comma. 205 */ 206 public void setTableFilter(String tablesWithDelimiter) { 207 Collections.addAll(tableSet, tablesWithDelimiter.split(",")); 208 } 209 /** 210 * sets the region by which output will be filtered 211 * 212 * @param region 213 * when not null, serves as a filter; only log entries from this 214 * region will be printed 215 */ 216 public void setRegionFilter(String region) { 217 this.region = region; 218 } 219 220 /** 221 * sets the row key by which output will be filtered 222 * 223 * @param row 224 * when not null, serves as a filter; only log entries from this row 225 * will be printed 226 */ 227 public void setRowFilter(String row) { 228 this.row = row; 229 } 230 231 /** 232 * sets the rowPrefix key prefix by which output will be filtered 233 * 234 * @param rowPrefix 235 * when not null, serves as a filter; only log entries with rows 236 * having this prefix will be printed 237 */ 238 public void setRowPrefixFilter(String rowPrefix) { 239 this.rowPrefix = rowPrefix; 240 } 241 242 /** 243 * Option to print the row key only in case you just need the row keys from the WAL 244 */ 245 public void setOutputOnlyRowKey() { 246 this.outputOnlyRowKey = true; 247 } 248 249 /** 250 * sets the position to start seeking the WAL file 251 * @param position 252 * initial position to start seeking the given WAL file 253 */ 254 public void setPosition(long position) { 255 this.position = position; 256 } 257 258 /** 259 * enables output as a single, persistent list. at present, only relevant in 260 * the case of JSON output. 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 * ends output of a single, persistent list. at present, only relevant in the 275 * case of JSON output. 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 * reads a log file and outputs its contents, one transaction at a time, as 289 * specified by the currently configured options 290 * 291 * @param conf 292 * the HBase configuration relevant to this log file 293 * @param p 294 * the path of the log file to be read 295 * @throws IOException 296 * may be unable to access the configured filesystem or requested 297 * file. 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 // begin building a transaction structure 345 Map<String, Object> txn = key.toStringMap(); 346 long writeTime = key.getWriteTime(); 347 // check output filters 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 // initialize list into which we will store atomic actions 359 List<Map<String, Object>> actions = new ArrayList<>(); 360 for (Cell cell : edit.getCells()) { 361 // add atomic operation to txn 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 // JSON output is a straightforward "toString" on the txn object 375 if (firstTxn) { 376 firstTxn = false; 377 } else { 378 out.print(","); 379 } 380 // encode and print JSON 381 out.print(GSON.toJson(txn)); 382 } else { 383 // Pretty output, complete with indentation by atomic action 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 } finally { 399 log.close(); 400 } 401 if (outputJSON && !persistentOutput) { 402 out.print("]"); 403 } 404 } 405 406 public static void printCell(PrintStream out, Map<String, Object> op, 407 boolean outputValues, boolean outputOnlyRowKey) { 408 String rowDetails = "row=" + op.get("row"); 409 if (outputOnlyRowKey) { 410 out.println(rowDetails); 411 return; 412 } 413 414 rowDetails += ", column=" + op.get("family") + ":" + op.get("qualifier"); 415 rowDetails += ", type=" + op.get("type"); 416 out.println(rowDetails); 417 if (op.get("tag") != null) { 418 out.println(" tag: " + op.get("tag")); 419 } 420 if (outputValues) { 421 out.println(" value: " + op.get("value")); 422 } 423 out.println("cell total size sum: " + op.get("total_size_sum")); 424 } 425 426 public static Map<String, Object> toStringMap(Cell cell, 427 boolean printRowKeyOnly, String rowPrefix, String row, boolean outputValues) { 428 Map<String, Object> stringMap = new HashMap<>(); 429 String rowKey = Bytes.toStringBinary(cell.getRowArray(), 430 cell.getRowOffset(), cell.getRowLength()); 431 // Row and row prefix are mutually options so both cannot be true at the 432 // same time. We can include checks in the same condition 433 // Check if any of the filters are satisfied by the row, if not return empty map 434 if ((!Strings.isNullOrEmpty(rowPrefix) && !rowKey.startsWith(rowPrefix)) || 435 (!Strings.isNullOrEmpty(row) && !rowKey.equals(row))) { 436 return stringMap; 437 } 438 439 stringMap.put("row", rowKey); 440 if (printRowKeyOnly) { 441 return stringMap; 442 } 443 stringMap.put("type", cell.getType()); 444 stringMap.put("family", Bytes.toStringBinary(cell.getFamilyArray(), cell.getFamilyOffset(), 445 cell.getFamilyLength())); 446 stringMap.put("qualifier", 447 Bytes.toStringBinary(cell.getQualifierArray(), cell.getQualifierOffset(), 448 cell.getQualifierLength())); 449 stringMap.put("timestamp", cell.getTimestamp()); 450 stringMap.put("vlen", cell.getValueLength()); 451 stringMap.put("total_size_sum", cell.heapSize()); 452 if (cell.getTagsLength() > 0) { 453 List<String> tagsString = new ArrayList<>(); 454 Iterator<Tag> tagsIterator = PrivateCellUtil.tagsIterator(cell); 455 while (tagsIterator.hasNext()) { 456 Tag tag = tagsIterator.next(); 457 tagsString 458 .add((tag.getType()) + ":" + Bytes.toStringBinary(Tag.cloneValue(tag))); 459 } 460 stringMap.put("tag", tagsString); 461 } 462 if (outputValues) { 463 stringMap.put("value", Bytes.toStringBinary(CellUtil.cloneValue(cell))); 464 } 465 return stringMap; 466 } 467 468 public static Map<String, Object> toStringMap(Cell cell) { 469 return toStringMap(cell, false, null, null, false); 470 } 471 472 public static void main(String[] args) throws IOException { 473 run(args); 474 } 475 476 /** 477 * Pass one or more log file names and formatting options and it will dump out 478 * a text version of the contents on <code>stdout</code>. 479 * 480 * @param args 481 * Command line arguments 482 * @throws IOException 483 * Thrown upon file system errors etc. 484 */ 485 public static void run(String[] args) throws IOException { 486 // create options 487 Options options = new Options(); 488 options.addOption("h", "help", false, "Output help message"); 489 options.addOption("j", "json", false, "Output JSON"); 490 options.addOption("p", "printvals", false, "Print values"); 491 options.addOption("t", "tables", true, 492 "Table names (comma separated) to filter by; eg: test1,test2,test3 "); 493 options.addOption("r", "region", true, 494 "Region to filter by. Pass encoded region name; e.g. '9192caead6a5a20acb4454ffbc79fa14'"); 495 options.addOption("s", "sequence", true, 496 "Sequence to filter by. Pass sequence number."); 497 options.addOption("k", "outputOnlyRowKey", false, 498 "Print only row keys"); 499 options.addOption("w", "row", true, "Row to filter by. Pass row name."); 500 options.addOption("f", "rowPrefix", true, "Row prefix to filter by."); 501 options.addOption("g", "goto", true, "Position to seek to in the file"); 502 503 WALPrettyPrinter printer = new WALPrettyPrinter(); 504 CommandLineParser parser = new PosixParser(); 505 List<?> files = null; 506 try { 507 CommandLine cmd = parser.parse(options, args); 508 files = cmd.getArgList(); 509 if (files.isEmpty() || cmd.hasOption("h")) { 510 HelpFormatter formatter = new HelpFormatter(); 511 formatter.printHelp("WAL <filename...>", options, true); 512 System.exit(-1); 513 } 514 // configure the pretty printer using command line options 515 if (cmd.hasOption("p")) { 516 printer.enableValues(); 517 } 518 if (cmd.hasOption("j")) { 519 printer.enableJSON(); 520 } 521 if (cmd.hasOption("k")) { 522 printer.setOutputOnlyRowKey(); 523 } 524 if (cmd.hasOption("t")) { 525 printer.setTableFilter(cmd.getOptionValue("t")); 526 } 527 if (cmd.hasOption("r")) { 528 printer.setRegionFilter(cmd.getOptionValue("r")); 529 } 530 if (cmd.hasOption("s")) { 531 printer.setSequenceFilter(Long.parseLong(cmd.getOptionValue("s"))); 532 } 533 if (cmd.hasOption("w")) { 534 if (cmd.hasOption("f")) { 535 throw new ParseException("Row and Row-prefix cannot be supplied together"); 536 } 537 printer.setRowFilter(cmd.getOptionValue("w")); 538 } 539 if (cmd.hasOption("f")) { 540 if (cmd.hasOption("w")) { 541 throw new ParseException("Row and Row-prefix cannot be supplied together"); 542 } 543 printer.setRowPrefixFilter(cmd.getOptionValue("f")); 544 } 545 if (cmd.hasOption("g")) { 546 printer.setPosition(Long.parseLong(cmd.getOptionValue("g"))); 547 } 548 } catch (ParseException e) { 549 LOG.error("Failed to parse commandLine arguments", e); 550 HelpFormatter formatter = new HelpFormatter(); 551 formatter.printHelp("HFile filename(s) ", options, true); 552 System.exit(-1); 553 } 554 // get configuration, file system, and process the given files 555 Configuration conf = HBaseConfiguration.create(); 556 CommonFSUtils.setFsDefault(conf, CommonFSUtils.getRootDir(conf)); 557 558 // begin output 559 printer.beginPersistentOutput(); 560 for (Object f : files) { 561 Path file = new Path((String) f); 562 FileSystem fs = file.getFileSystem(conf); 563 if (!fs.exists(file)) { 564 System.err.println("ERROR, file doesnt exist: " + file); 565 return; 566 } 567 printer.processFile(conf, file); 568 } 569 printer.endPersistentOutput(); 570 } 571}