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.AbstractProtobufWALReader; 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; 049 050import org.apache.hbase.thirdparty.com.google.common.base.Strings; 051import org.apache.hbase.thirdparty.com.google.gson.Gson; 052import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; 053import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLineParser; 054import org.apache.hbase.thirdparty.org.apache.commons.cli.HelpFormatter; 055import org.apache.hbase.thirdparty.org.apache.commons.cli.Options; 056import org.apache.hbase.thirdparty.org.apache.commons.cli.ParseException; 057import org.apache.hbase.thirdparty.org.apache.commons.cli.PosixParser; 058 059/** 060 * WALPrettyPrinter prints the contents of a given WAL with a variety of options affecting 061 * formatting and extent of content. It targets two usage cases: pretty printing for ease of 062 * debugging directly by humans, and JSON output for consumption by monitoring and/or maintenance 063 * scripts. It can filter by row, region, or sequence id. It can also toggle output of values. 064 */ 065@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS) 066@InterfaceStability.Evolving 067public class WALPrettyPrinter { 068 private static final Logger LOG = LoggerFactory.getLogger(WALPrettyPrinter.class); 069 070 // Output template for pretty printing. 071 private static final String outputTmpl = 072 "Sequence=%s, table=%s, region=%s, at write timestamp=%s"; 073 074 private boolean outputValues; 075 private boolean outputJSON; 076 // The following enable filtering by sequence, region, and row, respectively 077 private long sequence; 078 079 // List of tables for filter 080 private final Set<String> tableSet; 081 private String region; 082 083 // exact row which needs to be filtered 084 private String row; 085 // prefix of rows which needs to be filtered 086 private String rowPrefix; 087 088 private boolean outputOnlyRowKey; 089 // enable in order to output a single list of transactions from several files 090 private boolean persistentOutput; 091 private boolean firstTxn; 092 // useful for programmatic capture of JSON output 093 private PrintStream out; 094 // for JSON encoding 095 private static final Gson GSON = GsonUtil.createGson().create(); 096 // allows for jumping straight to a given portion of the file 097 private long position; 098 099 /** 100 * Basic constructor that simply initializes values to reasonable defaults. 101 */ 102 public WALPrettyPrinter() { 103 this(false, false, -1, new HashSet<>(), null, null, null, false, false, System.out); 104 } 105 106 /** 107 * Fully specified constructor. 108 * @param outputValues when true, enables output of values along with other log information 109 * @param outputJSON when true, enables output in JSON format rather than a "pretty string" 110 * @param sequence when nonnegative, serves as a filter; only log entries with this 111 * sequence id will be printed 112 * @param tableSet when non null, serves as a filter. only entries corresponding to tables 113 * in the tableSet are printed 114 * @param region when not null, serves as a filter; only log entries from this region 115 * will be printed 116 * @param row when not null, serves as a filter; only log entries from this row will 117 * be printed 118 * @param rowPrefix when not null, serves as a filter; only log entries with row key having 119 * this prefix will be printed 120 * @param persistentOutput keeps a single list running for multiple files. if enabled, the 121 * endPersistentOutput() method must be used! 122 * @param out Specifies an alternative to stdout for the destination of this 123 * PrettyPrinter's output. 124 */ 125 public WALPrettyPrinter(boolean outputValues, boolean outputJSON, long sequence, 126 Set<String> tableSet, String region, String row, String rowPrefix, boolean outputOnlyRowKey, 127 boolean persistentOutput, PrintStream out) { 128 this.outputValues = outputValues; 129 this.outputJSON = outputJSON; 130 this.sequence = sequence; 131 this.tableSet = tableSet; 132 this.region = region; 133 this.row = row; 134 this.rowPrefix = rowPrefix; 135 this.outputOnlyRowKey = outputOnlyRowKey; 136 this.persistentOutput = persistentOutput; 137 if (persistentOutput) { 138 beginPersistentOutput(); 139 } 140 this.out = out; 141 this.firstTxn = true; 142 } 143 144 /** 145 * turns value output on 146 */ 147 public void enableValues() { 148 outputValues = true; 149 } 150 151 /** 152 * turns value output off 153 */ 154 public void disableValues() { 155 outputValues = false; 156 } 157 158 /** 159 * turns JSON output on 160 */ 161 public void enableJSON() { 162 outputJSON = true; 163 } 164 165 /** 166 * turns JSON output off, and turns on "pretty strings" for human consumption 167 */ 168 public void disableJSON() { 169 outputJSON = false; 170 } 171 172 /** 173 * sets the region by which output will be filtered when nonnegative, serves as a filter; only log 174 * entries with this sequence id will be printed 175 */ 176 public void setSequenceFilter(long sequence) { 177 this.sequence = sequence; 178 } 179 180 /** 181 * Sets the tables filter. Only log entries for these tables are printed. 182 * @param tablesWithDelimiter table names separated with comma. 183 */ 184 public void setTableFilter(String tablesWithDelimiter) { 185 Collections.addAll(tableSet, tablesWithDelimiter.split(",")); 186 } 187 188 /** 189 * sets the region by which output will be filtered when not null, serves as a filter; only log 190 * entries from this region will be printed 191 */ 192 public void setRegionFilter(String region) { 193 this.region = region; 194 } 195 196 /** 197 * sets the row key by which output will be filtered when not null, serves as a filter; only log 198 * entries from this row will be printed 199 */ 200 public void setRowFilter(String row) { 201 this.row = row; 202 } 203 204 /** 205 * sets the rowPrefix key prefix by which output will be filtered when not null, serves as a 206 * filter; only log entries with rows having this prefix will be printed 207 */ 208 public void setRowPrefixFilter(String rowPrefix) { 209 this.rowPrefix = rowPrefix; 210 } 211 212 /** 213 * Option to print the row key only in case you just need the row keys from the WAL 214 */ 215 public void setOutputOnlyRowKey() { 216 this.outputOnlyRowKey = true; 217 } 218 219 /** 220 * sets the position to start seeking the WAL file initial position to start seeking the given WAL 221 * file 222 */ 223 public void setPosition(long position) { 224 this.position = position; 225 } 226 227 /** 228 * enables output as a single, persistent list. at present, only relevant in the case of JSON 229 * output. 230 */ 231 public void beginPersistentOutput() { 232 if (persistentOutput) { 233 return; 234 } 235 persistentOutput = true; 236 firstTxn = true; 237 if (outputJSON) { 238 out.print("["); 239 } 240 } 241 242 /** 243 * ends output of a single, persistent list. at present, only relevant in the case of JSON output. 244 */ 245 public void endPersistentOutput() { 246 if (!persistentOutput) { 247 return; 248 } 249 persistentOutput = false; 250 if (outputJSON) { 251 out.print("]"); 252 } 253 } 254 255 /** 256 * reads a log file and outputs its contents, one transaction at a time, as specified by the 257 * currently configured options the HBase configuration relevant to this log file the path of the 258 * log file to be read may be unable to access the configured filesystem or requested file. 259 */ 260 public void processFile(final Configuration conf, final Path p) throws IOException { 261 FileSystem fs = p.getFileSystem(conf); 262 if (!fs.exists(p)) { 263 throw new FileNotFoundException(p.toString()); 264 } 265 if (!fs.isFile(p)) { 266 throw new IOException(p + " is not a file"); 267 } 268 269 WALStreamReader log = WALFactory.createStreamReader(fs, p, conf, position > 0 ? position : -1); 270 271 if (log instanceof AbstractProtobufWALReader) { 272 List<String> writerClsNames = ((AbstractProtobufWALReader) log).getWriterClsNames(); 273 if (writerClsNames != null && writerClsNames.size() > 0) { 274 out.print("Writer Classes: "); 275 for (int i = 0; i < writerClsNames.size(); i++) { 276 out.print(writerClsNames.get(i)); 277 if (i != writerClsNames.size() - 1) { 278 out.print(" "); 279 } 280 } 281 out.println(); 282 } 283 284 String cellCodecClsName = ((AbstractProtobufWALReader) log).getCodecClsName(); 285 if (cellCodecClsName != null) { 286 out.println("Cell Codec Class: " + cellCodecClsName); 287 } 288 } 289 290 if (outputJSON && !persistentOutput) { 291 out.print("["); 292 firstTxn = true; 293 } 294 295 try { 296 WAL.Entry entry; 297 while ((entry = log.next()) != null) { 298 WALKey key = entry.getKey(); 299 WALEdit edit = entry.getEdit(); 300 // begin building a transaction structure 301 Map<String, Object> txn = key.toStringMap(); 302 long writeTime = key.getWriteTime(); 303 // check output filters 304 if (!tableSet.isEmpty() && !tableSet.contains(txn.get("table").toString())) { 305 continue; 306 } 307 if (sequence >= 0 && ((Long) txn.get("sequence")) != sequence) { 308 continue; 309 } 310 if (region != null && !txn.get("region").equals(region)) { 311 continue; 312 } 313 // initialize list into which we will store atomic actions 314 List<Map<String, Object>> actions = new ArrayList<>(); 315 for (Cell cell : edit.getCells()) { 316 // add atomic operation to txn 317 Map<String, Object> op = 318 new HashMap<>(toStringMap(cell, outputOnlyRowKey, rowPrefix, row, outputValues)); 319 if (op.isEmpty()) { 320 continue; 321 } 322 actions.add(op); 323 } 324 if (actions.isEmpty()) { 325 continue; 326 } 327 txn.put("actions", actions); 328 if (outputJSON) { 329 // JSON output is a straightforward "toString" on the txn object 330 if (firstTxn) { 331 firstTxn = false; 332 } else { 333 out.print(","); 334 } 335 // encode and print JSON 336 out.print(GSON.toJson(txn)); 337 } else { 338 // Pretty output, complete with indentation by atomic action 339 if (!outputOnlyRowKey) { 340 out.println(String.format(outputTmpl, txn.get("sequence"), txn.get("table"), 341 txn.get("region"), new Date(writeTime))); 342 } 343 for (int i = 0; i < actions.size(); i++) { 344 Map<String, Object> op = actions.get(i); 345 printCell(out, op, outputValues, outputOnlyRowKey); 346 } 347 } 348 if (!outputOnlyRowKey) { 349 out.println("edit heap size: " + entry.getEdit().heapSize()); 350 out.println("position: " + log.getPosition()); 351 } 352 } 353 } finally { 354 log.close(); 355 } 356 if (outputJSON && !persistentOutput) { 357 out.print("]"); 358 } 359 } 360 361 public static void printCell(PrintStream out, Map<String, Object> op, boolean outputValues, 362 boolean outputOnlyRowKey) { 363 String rowDetails = "row=" + op.get("row"); 364 if (outputOnlyRowKey) { 365 out.println(rowDetails); 366 return; 367 } 368 369 rowDetails += ", column=" + op.get("family") + ":" + op.get("qualifier"); 370 rowDetails += ", timestamp=" + op.get("timestamp"); 371 rowDetails += ", type=" + op.get("type"); 372 out.println(rowDetails); 373 if (op.get("tag") != null) { 374 out.println(" tag: " + op.get("tag")); 375 } 376 if (outputValues) { 377 out.println(" value: " + op.get("value")); 378 } 379 out.println("cell total size sum: " + op.get("total_size_sum")); 380 } 381 382 public static Map<String, Object> toStringMap(Cell cell, boolean printRowKeyOnly, 383 String rowPrefix, String row, boolean outputValues) { 384 Map<String, Object> stringMap = new HashMap<>(); 385 String rowKey = 386 Bytes.toStringBinary(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()); 387 // Row and row prefix are mutually options so both cannot be true at the 388 // same time. We can include checks in the same condition 389 // Check if any of the filters are satisfied by the row, if not return empty map 390 if ( 391 (!Strings.isNullOrEmpty(rowPrefix) && !rowKey.startsWith(rowPrefix)) 392 || (!Strings.isNullOrEmpty(row) && !rowKey.equals(row)) 393 ) { 394 return stringMap; 395 } 396 397 stringMap.put("row", rowKey); 398 if (printRowKeyOnly) { 399 return stringMap; 400 } 401 stringMap.put("type", cell.getType()); 402 stringMap.put("family", 403 Bytes.toStringBinary(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength())); 404 stringMap.put("qualifier", Bytes.toStringBinary(cell.getQualifierArray(), 405 cell.getQualifierOffset(), cell.getQualifierLength())); 406 stringMap.put("timestamp", cell.getTimestamp()); 407 stringMap.put("vlen", cell.getValueLength()); 408 stringMap.put("total_size_sum", cell.heapSize()); 409 if (cell.getTagsLength() > 0) { 410 List<String> tagsString = new ArrayList<>(); 411 Iterator<Tag> tagsIterator = PrivateCellUtil.tagsIterator(cell); 412 while (tagsIterator.hasNext()) { 413 Tag tag = tagsIterator.next(); 414 tagsString.add((tag.getType()) + ":" + Bytes.toStringBinary(Tag.cloneValue(tag))); 415 } 416 stringMap.put("tag", tagsString); 417 } 418 if (outputValues) { 419 stringMap.put("value", Bytes.toStringBinary(CellUtil.cloneValue(cell))); 420 } 421 return stringMap; 422 } 423 424 public static Map<String, Object> toStringMap(Cell cell) { 425 return toStringMap(cell, false, null, null, false); 426 } 427 428 public static void main(String[] args) throws IOException { 429 run(args); 430 } 431 432 /** 433 * Pass one or more log file names and formatting options and it will dump out a text version of 434 * the contents on <code>stdout</code>. Command line arguments Thrown upon file system errors etc. 435 */ 436 public static void run(String[] args) throws IOException { 437 // create options 438 Options options = new Options(); 439 options.addOption("h", "help", false, "Output help message"); 440 options.addOption("j", "json", false, "Output JSON"); 441 options.addOption("p", "printvals", false, "Print values"); 442 options.addOption("t", "tables", true, 443 "Table names (comma separated) to filter by; eg: test1,test2,test3 "); 444 options.addOption("r", "region", true, 445 "Region to filter by. Pass encoded region name; e.g. '9192caead6a5a20acb4454ffbc79fa14'"); 446 options.addOption("s", "sequence", true, "Sequence to filter by. Pass sequence number."); 447 options.addOption("k", "outputOnlyRowKey", false, "Print only row keys"); 448 options.addOption("w", "row", true, "Row to filter by. Pass row name."); 449 options.addOption("f", "rowPrefix", true, "Row prefix to filter by."); 450 options.addOption("g", "goto", true, "Position to seek to in the file"); 451 452 WALPrettyPrinter printer = new WALPrettyPrinter(); 453 CommandLineParser parser = new PosixParser(); 454 List<?> files = null; 455 try { 456 CommandLine cmd = parser.parse(options, args); 457 files = cmd.getArgList(); 458 if (files.isEmpty() || cmd.hasOption("h")) { 459 HelpFormatter formatter = new HelpFormatter(); 460 formatter.printHelp("WAL <filename...>", options, true); 461 System.exit(-1); 462 } 463 // configure the pretty printer using command line options 464 if (cmd.hasOption("p")) { 465 printer.enableValues(); 466 } 467 if (cmd.hasOption("j")) { 468 printer.enableJSON(); 469 } 470 if (cmd.hasOption("k")) { 471 printer.setOutputOnlyRowKey(); 472 } 473 if (cmd.hasOption("t")) { 474 printer.setTableFilter(cmd.getOptionValue("t")); 475 } 476 if (cmd.hasOption("r")) { 477 printer.setRegionFilter(cmd.getOptionValue("r")); 478 } 479 if (cmd.hasOption("s")) { 480 printer.setSequenceFilter(Long.parseLong(cmd.getOptionValue("s"))); 481 } 482 if (cmd.hasOption("w")) { 483 if (cmd.hasOption("f")) { 484 throw new ParseException("Row and Row-prefix cannot be supplied together"); 485 } 486 printer.setRowFilter(cmd.getOptionValue("w")); 487 } 488 if (cmd.hasOption("f")) { 489 if (cmd.hasOption("w")) { 490 throw new ParseException("Row and Row-prefix cannot be supplied together"); 491 } 492 printer.setRowPrefixFilter(cmd.getOptionValue("f")); 493 } 494 if (cmd.hasOption("g")) { 495 printer.setPosition(Long.parseLong(cmd.getOptionValue("g"))); 496 } 497 } catch (ParseException e) { 498 LOG.error("Failed to parse commandLine arguments", e); 499 HelpFormatter formatter = new HelpFormatter(); 500 formatter.printHelp("HFile filename(s) ", options, true); 501 System.exit(-1); 502 } 503 // get configuration, file system, and process the given files 504 Configuration conf = HBaseConfiguration.create(); 505 CommonFSUtils.setFsDefault(conf, CommonFSUtils.getRootDir(conf)); 506 507 // begin output 508 printer.beginPersistentOutput(); 509 for (Object f : files) { 510 Path file = new Path((String) f); 511 FileSystem fs = file.getFileSystem(conf); 512 if (!fs.exists(file)) { 513 System.err.println("ERROR, file doesnt exist: " + file); 514 return; 515 } 516 printer.processFile(conf, file); 517 } 518 printer.endPersistentOutput(); 519 } 520}