Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HBASE-25193: Add support for row prefix and type in the WAL Pretty Printer #2555

Closed
wants to merge 3 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Expand Up @@ -68,7 +68,7 @@ public class HLogPrettyPrinter extends WALPrettyPrinter {
* Basic constructor that simply initializes values to reasonable defaults.
*/
public HLogPrettyPrinter() {
this(false, false, -1l, null, null, null, false, System.out);
this(false, false, -1L, null, null, null, null, false, System.out);
}

/**
Expand All @@ -92,6 +92,9 @@ public HLogPrettyPrinter() {
* @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 @@ -100,10 +103,10 @@ public HLogPrettyPrinter() {
* PrettyPrinter's output.
*/
public HLogPrettyPrinter(boolean outputValues, boolean outputJSON,
long sequence, String table, String region, String row, boolean persistentOutput,
PrintStream out) {
super(outputValues, outputJSON, sequence, Collections.singleton(table), region, row,
false, persistentOutput, out);
long sequence, String table, String region, String row, String rowPrefix,
boolean persistentOutput, PrintStream out) {
super(outputValues, outputJSON, sequence, Collections.singleton(table), region,
row, rowPrefix,false, persistentOutput, out);
}

public static void main(String[] args) throws IOException {
Expand Down
Expand Up @@ -42,6 +42,7 @@
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
Expand All @@ -50,6 +51,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 +87,11 @@ public class WALPrettyPrinter {
// List of tables for filter
private final Set<String> tableSet;
private String region;

// exact row which needs to be filtered
private String row;
// prefix of rows which needs to be filtered
private String rowPrefix;
private boolean outputOnlyRowKey;
// enable in order to output a single list of transactions from several files
private boolean persistentOutput;
Expand All @@ -101,7 +107,7 @@ public class WALPrettyPrinter {
* Basic constructor that simply initializes values to reasonable defaults.
*/
public WALPrettyPrinter() {
this(false, false, -1, new HashSet<String>(), null, null, false, false, System.out);
this(false, false, -1, new HashSet<String>(), null, null, null, false, false, System.out);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think its time to use builder pattern for this, its getting annoyingly long (perhaps in another patch).

}

/**
Expand All @@ -125,6 +131,9 @@ public WALPrettyPrinter() {
* @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 +142,15 @@ 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 row, 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 @@ -218,6 +228,17 @@ public void setRowFilter(String row) {
this.row = row;
}

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

/**
* Option to print the row key only in case you just need the row keys from the WAL
*/
Expand Down Expand Up @@ -338,15 +359,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, row, outputValues));
if (op.isEmpty()) {
continue;
}
op.put("total_size_sum", CellUtil.estimatedHeapSizeOf(cell));
actions.add(op);
}
if (actions.isEmpty()) {
continue;
Expand All @@ -363,15 +381,20 @@ public void processFile(final Configuration conf, final Path p)
out.print(GSON.toJson(txn));
} else {
// Pretty output, complete with indentation by atomic action
out.println(String.format(outputTmpl,
if (!outputOnlyRowKey) {
out.println(String.format(outputTmpl,
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, outputValues, outputOnlyRowKey);
}
}
out.println("edit heap size: " + entry.getEdit().heapSize());
out.println("position: " + log.getPosition());
if (!outputOnlyRowKey) {
out.println("edit heap size: " + entry.getEdit().heapSize());
out.println("position: " + log.getPosition());

}
}
} finally {
log.close();
Expand All @@ -381,9 +404,17 @@ 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 outputValues, boolean outputOnlyRowKey) {
String rowDetails = "row=" + op.get("row");
if (outputOnlyRowKey) {
out.println(rowDetails);
return;
}

rowDetails += ", column=" + op.get("family") + ":" + op.get("qualifier");
rowDetails += ", type=" + op.get("type");
out.println(rowDetails);
if (op.get("tag") != null) {
out.println(" tag: " + op.get("tag"));
}
Expand All @@ -393,21 +424,32 @@ 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, String row, 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());
// Row and row prefix are mutually options so both cannot be true at the
// same time. We can include checks in the same condition
// Check if any of the filters are satisfied by the row, if not return empty map
if ((!Strings.isEmpty(rowPrefix) && !rowKey.startsWith(rowPrefix)) ||
(!Strings.isEmpty(row) && !rowKey.equals(row))) {
return stringMap;
}

stringMap.put("row", rowKey);
if (printRowKeyOnly) {
return stringMap;
}
stringMap.put("type", KeyValue.Type.codeToType(cell.getTypeByte()));
stringMap.put("family",
Bytes.toStringBinary(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength()));
stringMap.put("qualifier", Bytes
.toStringBinary(cell.getQualifierArray(), cell.getQualifierOffset(),
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 +461,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, null, false);
}

public static void main(String[] args) throws IOException {
Expand Down Expand Up @@ -454,6 +499,7 @@ public static void run(String[] args) throws IOException {
options.addOption("k", "outputOnlyRowKey", false,
"Print only row keys");
options.addOption("w", "row", true, "Row to filter by. Pass row name.");
options.addOption("f", "rowPrefix", true, "Row 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,8 +533,17 @@ public static void run(String[] args) throws IOException {
printer.setSequenceFilter(Long.parseLong(cmd.getOptionValue("s")));
}
if (cmd.hasOption("w")) {
if (cmd.hasOption("f")) {
throw new ParseException("Row and Row-prefix cannot be supplied together");
}
printer.setRowFilter(cmd.getOptionValue("w"));
}
if (cmd.hasOption("f")) {
if (cmd.hasOption("w")) {
throw new ParseException("Row and Row-prefix cannot be supplied together");
}
printer.setRowPrefixFilter(cmd.getOptionValue("f"));
}
if (cmd.hasOption("g")) {
printer.setPosition(Long.parseLong(cmd.getOptionValue("g")));
}
Expand Down