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