HBASE-24974: Provide a flexibility to print only row key and filter for multiple tables in the WALPrettyPrinter (#2387)

Signed-off-by: Bharath Vissapragada <bharathv@apache.org>
This commit is contained in:
Sandeep Pal 2020-09-11 12:44:34 -07:00 committed by GitHub
parent cd66d8cba5
commit b39199f43b
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
1 changed files with 45 additions and 17 deletions

View File

@ -21,11 +21,14 @@ import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.PrintStream;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -34,7 +37,6 @@ import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
import org.apache.hadoop.hbase.util.Bytes;
@ -79,9 +81,12 @@ public class WALPrettyPrinter {
private boolean outputJSON;
// The following enable filtering by sequence, region, and row, respectively
private long sequence;
private String table;
// List of tables for filter
private final Set<String> tableSet;
private String region;
private String row;
private boolean outputOnlyRowKey;
// enable in order to output a single list of transactions from several files
private boolean persistentOutput;
private boolean firstTxn;
@ -99,9 +104,10 @@ public class WALPrettyPrinter {
outputValues = false;
outputJSON = false;
sequence = -1;
table = null;
tableSet = new HashSet<>();
region = null;
row = null;
outputOnlyRowKey = false;
persistentOutput = false;
firstTxn = true;
out = System.out;
@ -192,11 +198,11 @@ public class WALPrettyPrinter {
}
/**
* Sets the table filter. Only log entries for this table are printed.
* @param table table name to set.
* Sets the tables filter. Only log entries for these tables are printed.
* @param tablesWithDelimiter table names separated with comma.
*/
public void setTableFilter(String table) {
this.table = table;
public void setTableFilter(String tablesWithDelimiter) {
Collections.addAll(tableSet, tablesWithDelimiter.split(","));
}
/**
* sets the region by which output will be filtered
@ -210,7 +216,7 @@ public class WALPrettyPrinter {
}
/**
* sets the region by which output will be filtered
* sets the row key by which output will be filtered
*
* @param row
* when not null, serves as a filter; only log entries from this row
@ -220,6 +226,13 @@ public class WALPrettyPrinter {
this.row = row;
}
/**
* Option to print the row key only in case you just need the row keys from the WAL
*/
public void setOutputOnlyRowKey() {
this.outputOnlyRowKey = true;
}
/**
* sets the position to start seeking the WAL file
* @param position
@ -319,7 +332,8 @@ public class WALPrettyPrinter {
Map<String, Object> txn = key.toStringMap();
long writeTime = key.getWriteTime();
// check output filters
if (table != null && !((TableName) txn.get("table")).toString().equals(table)) {
if (!tableSet.isEmpty() &&
!tableSet.contains(txn.get("table").toString())) {
continue;
}
if (sequence >= 0 && ((Long) txn.get("sequence")) != sequence) {
@ -332,7 +346,7 @@ public class WALPrettyPrinter {
List<Map<String, Object>> actions = new ArrayList<>();
for (Cell cell : edit.getCells()) {
// add atomic operation to txn
Map<String, Object> op = new HashMap<>(toStringMap(cell));
Map<String, Object> op = new HashMap<>(toStringMap(cell, outputOnlyRowKey));
if (outputValues) {
op.put("value", Bytes.toStringBinary(CellUtil.cloneValue(cell)));
}
@ -387,16 +401,20 @@ public class WALPrettyPrinter {
out.println("cell total size sum: " + op.get("total_size_sum"));
}
public static Map<String, Object> toStringMap(Cell cell) {
public static Map<String, Object> toStringMap(Cell cell, boolean printRowKeyOnly) {
Map<String, Object> stringMap = new HashMap<>();
stringMap.put("row",
Bytes.toStringBinary(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()));
Bytes.toStringBinary(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()));
if (printRowKeyOnly) {
return stringMap;
}
stringMap.put("type", cell.getType());
stringMap.put("family", Bytes.toStringBinary(cell.getFamilyArray(), cell.getFamilyOffset(),
cell.getFamilyLength()));
cell.getFamilyLength()));
stringMap.put("qualifier",
Bytes.toStringBinary(cell.getQualifierArray(), cell.getQualifierOffset(),
cell.getQualifierLength()));
Bytes.toStringBinary(cell.getQualifierArray(), cell.getQualifierOffset(),
cell.getQualifierLength()));
stringMap.put("timestamp", cell.getTimestamp());
stringMap.put("vlen", cell.getValueLength());
if (cell.getTagsLength() > 0) {
@ -405,13 +423,17 @@ public class WALPrettyPrinter {
while (tagsIterator.hasNext()) {
Tag tag = tagsIterator.next();
tagsString
.add((tag.getType()) + ":" + Bytes.toStringBinary(Tag.cloneValue(tag)));
.add((tag.getType()) + ":" + Bytes.toStringBinary(Tag.cloneValue(tag)));
}
stringMap.put("tag", tagsString);
}
return stringMap;
}
public static Map<String, Object> toStringMap(Cell cell) {
return toStringMap(cell, false);
}
public static void main(String[] args) throws IOException {
run(args);
}
@ -431,11 +453,14 @@ public class WALPrettyPrinter {
options.addOption("h", "help", false, "Output help message");
options.addOption("j", "json", false, "Output JSON");
options.addOption("p", "printvals", false, "Print values");
options.addOption("t", "table", true, "Table name to filter by.");
options.addOption("t", "tables", true,
"Table names (comma separated) to filter by; eg: test1,test2,test3 ");
options.addOption("r", "region", true,
"Region to filter by. Pass encoded region name; e.g. '9192caead6a5a20acb4454ffbc79fa14'");
options.addOption("s", "sequence", true,
"Sequence to filter by. Pass sequence number.");
options.addOption("k", "outputOnlyRowKey", false,
"Print only row keys");
options.addOption("w", "row", true, "Row to filter by. Pass row name.");
options.addOption("g", "goto", true, "Position to seek to in the file");
@ -457,6 +482,9 @@ public class WALPrettyPrinter {
if (cmd.hasOption("j")) {
printer.enableJSON();
}
if (cmd.hasOption("k")) {
printer.setOutputOnlyRowKey();
}
if (cmd.hasOption("t")) {
printer.setTableFilter(cmd.getOptionValue("t"));
}