Skip to content

Commit

Permalink
Add support for row prefix in the WAL Pretty Printer
Browse files Browse the repository at this point in the history
  • Loading branch information
Sandeep Pal committed Oct 18, 2020
1 parent e066951 commit 26d9b46
Showing 1 changed file with 45 additions and 29 deletions.
Expand Up @@ -50,6 +50,7 @@
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.GsonUtil;
import org.apache.hadoop.hbase.util.Strings;
import org.apache.hbase.thirdparty.com.google.gson.Gson;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -85,7 +86,7 @@ public class WALPrettyPrinter {
// List of tables for filter
private final Set<String> tableSet;
private String region;
private String row;
private String rowPrefix;
private boolean outputOnlyRowKey;
// enable in order to output a single list of transactions from several files
private boolean persistentOutput;
Expand Down Expand Up @@ -122,9 +123,9 @@ public WALPrettyPrinter() {
* @param region
* when not null, serves as a filter; only log entries from this
* region will be printed
* @param row
* when not null, serves as a filter; only log entries from this row
* will be printed
* @param rowPrefix
* when not null, serves as a filter; only log entries with row key
* having this prefix will be printed
* @param persistentOutput
* keeps a single list running for multiple files. if enabled, the
* endPersistentOutput() method must be used!
Expand All @@ -133,14 +134,14 @@ public WALPrettyPrinter() {
* PrettyPrinter's output.
*/
public WALPrettyPrinter(boolean outputValues, boolean outputJSON, long sequence,
Set<String> tableSet, String region, String row, boolean outputOnlyRowKey,
Set<String> tableSet, String region, String rowPrefix, boolean outputOnlyRowKey,
boolean persistentOutput, PrintStream out) {
this.outputValues = outputValues;
this.outputJSON = outputJSON;
this.sequence = sequence;
this.tableSet = tableSet;
this.region = region;
this.row = row;
this.rowPrefix = rowPrefix;
this.outputOnlyRowKey = outputOnlyRowKey;
this.persistentOutput = persistentOutput;
if (persistentOutput) {
Expand Down Expand Up @@ -208,14 +209,14 @@ public void setRegionFilter(String region) {
}

/**
* sets the row key by which output will be filtered
* sets the rowPrefix key prefix by which output will be filtered
*
* @param row
* when not null, serves as a filter; only log entries from this row
* @param rowPrefix
* when not null, serves as a filter; only log entries from this rowPrefix
* will be printed
*/
public void setRowFilter(String row) {
this.row = row;
public void setRowPrefixFilter(String rowPrefix) {
this.rowPrefix = rowPrefix;
}

/**
Expand Down Expand Up @@ -338,15 +339,12 @@ public void processFile(final Configuration conf, final Path p)
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, outputOnlyRowKey));
if (outputValues) {
op.put("value", Bytes.toStringBinary(CellUtil.cloneValue(cell)));
}
// check row output filter
if (row == null || ((String) op.get("row")).equals(row)) {
actions.add(op);
Map<String, Object> op =
new HashMap<>(toStringMap(cell, outputOnlyRowKey, rowPrefix, outputValues));
if (op.isEmpty()) {
continue;
}
op.put("total_size_sum", CellUtil.estimatedHeapSizeOf(cell));
actions.add(op);
}
if (actions.isEmpty()) {
continue;
Expand All @@ -367,7 +365,7 @@ public void processFile(final Configuration conf, final Path p)
txn.get("sequence"), txn.get("table"), txn.get("region"), new Date(writeTime)));
for (int i = 0; i < actions.size(); i++) {
Map<String, Object> op = actions.get(i);
printCell(out, op, outputValues);
printCell(out, op, outputOnlyRowKey, outputValues);
}
}
out.println("edit heap size: " + entry.getEdit().heapSize());
Expand All @@ -381,9 +379,16 @@ public void processFile(final Configuration conf, final Path p)
}
}

public static void printCell(PrintStream out, Map<String, Object> op, boolean outputValues) {
out.println("row=" + op.get("row") + ", column=" +
op.get("family") + ":" + op.get("qualifier"));
public static void printCell(PrintStream out, Map<String, Object> op,
boolean outputOnlyRowKey, boolean outputValues) {
String rowDetails = "row=" + op.get("row");
if (outputOnlyRowKey) {
out.println(rowDetails);
return;
}

rowDetails += ", column=" + op.get("family") + ":" + op.get("qualifier");
out.println(rowDetails);
if (op.get("tag") != null) {
out.println(" tag: " + op.get("tag"));
}
Expand All @@ -393,11 +398,18 @@ public static void printCell(PrintStream out, Map<String, Object> op, boolean ou
out.println("cell total size sum: " + op.get("total_size_sum"));
}

public static Map<String, Object> toStringMap(Cell cell, boolean printRowKeyOnly) {
public static Map<String, Object> toStringMap(Cell cell,
boolean printRowKeyOnly, String rowPrefix, boolean outputValues) {
Map<String, Object> stringMap = new HashMap<>();
stringMap.put("row",
Bytes.toStringBinary(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()));
String rowKey = Bytes.toStringBinary(cell.getRowArray(),
cell.getRowOffset(), cell.getRowLength());
// If the row prefix is provided and the current row doesn't satisfy
// the prefix requirement return empty map
if (!Strings.isEmpty(rowPrefix) && !rowKey.startsWith(rowPrefix)) {
return stringMap;
}

stringMap.put("row", rowKey);
if (printRowKeyOnly) {
return stringMap;
}
Expand All @@ -408,6 +420,7 @@ public static Map<String, Object> toStringMap(Cell cell, boolean printRowKeyOnly
cell.getQualifierLength()));
stringMap.put("timestamp", cell.getTimestamp());
stringMap.put("vlen", cell.getValueLength());
stringMap.put("total_size_sum", CellUtil.estimatedHeapSizeOf(cell));
if (cell.getTagsLength() > 0) {
List<String> tagsString = new ArrayList<>();
Iterator<Tag> tagsIterator =
Expand All @@ -419,11 +432,14 @@ public static Map<String, Object> toStringMap(Cell cell, boolean printRowKeyOnly
}
stringMap.put("tag", tagsString);
}
if (outputValues) {
stringMap.put("value", Bytes.toStringBinary(CellUtil.cloneValue(cell)));
}
return stringMap;
}

public static Map<String, Object> toStringMap(Cell cell) {
return toStringMap(cell, false);
return toStringMap(cell, false, null, false);
}

public static void main(String[] args) throws IOException {
Expand Down Expand Up @@ -453,7 +469,7 @@ public static void run(String[] args) throws IOException {
"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("w", "row", true, "Row key prefix to filter by. ");
options.addOption("g", "goto", true, "Position to seek to in the file");

WALPrettyPrinter printer = new WALPrettyPrinter();
Expand Down Expand Up @@ -487,7 +503,7 @@ public static void run(String[] args) throws IOException {
printer.setSequenceFilter(Long.parseLong(cmd.getOptionValue("s")));
}
if (cmd.hasOption("w")) {
printer.setRowFilter(cmd.getOptionValue("w"));
printer.setRowPrefixFilter(cmd.getOptionValue("w"));
}
if (cmd.hasOption("g")) {
printer.setPosition(Long.parseLong(cmd.getOptionValue("g")));
Expand Down

0 comments on commit 26d9b46

Please sign in to comment.