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