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