HBASE-24974: Provide a flexibility to print only row key and filter for multiple tables in the WALPrettyPrinter (#2389)
Signed-off-by: Bharath Vissapragada <bharathv@apache.org>
This commit is contained in:
parent
c8d6d7a780
commit
d6f0128c83
|
@ -21,6 +21,7 @@ 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.List;
|
||||
|
@ -101,7 +102,8 @@ public class HLogPrettyPrinter extends WALPrettyPrinter {
|
|||
public HLogPrettyPrinter(boolean outputValues, boolean outputJSON,
|
||||
long sequence, String table, String region, String row, boolean persistentOutput,
|
||||
PrintStream out) {
|
||||
super(outputValues, outputJSON, sequence, table, region, row, persistentOutput, out);
|
||||
super(outputValues, outputJSON, sequence, Collections.singleton(table), region, row,
|
||||
false, persistentOutput, out);
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws IOException {
|
||||
|
|
|
@ -21,21 +21,20 @@ 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.commons.cli.CommandLine;
|
||||
import org.apache.commons.cli.CommandLineParser;
|
||||
import org.apache.commons.cli.HelpFormatter;
|
||||
import org.apache.commons.cli.Options;
|
||||
import org.apache.commons.cli.ParseException;
|
||||
import org.apache.commons.cli.PosixParser;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.util.GsonUtil;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
@ -43,14 +42,14 @@ import org.apache.hadoop.hbase.Cell;
|
|||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
|
||||
import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
|
||||
// imports for things that haven't moved yet.
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.util.GsonUtil;
|
||||
import org.apache.hbase.thirdparty.com.google.gson.Gson;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
@ -82,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;
|
||||
|
@ -92,22 +94,14 @@ public class WALPrettyPrinter {
|
|||
private PrintStream out;
|
||||
// for JSON encoding
|
||||
private static final Gson GSON = GsonUtil.createGson().create();
|
||||
|
||||
//allows for jumping straight to a given portion of the file
|
||||
private long position;
|
||||
|
||||
/**
|
||||
* Basic constructor that simply initializes values to reasonable defaults.
|
||||
*/
|
||||
public WALPrettyPrinter() {
|
||||
outputValues = false;
|
||||
outputJSON = false;
|
||||
sequence = -1;
|
||||
table = null;
|
||||
region = null;
|
||||
row = null;
|
||||
persistentOutput = false;
|
||||
firstTxn = true;
|
||||
out = System.out;
|
||||
this(false, false, -1, new HashSet<String>(), null, null, false, false, System.out);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -122,9 +116,9 @@ public class WALPrettyPrinter {
|
|||
* @param sequence
|
||||
* when nonnegative, serves as a filter; only log entries with this
|
||||
* sequence id will be printed
|
||||
* @param table
|
||||
* when non null, serves as a filter. only entries corresponding to this
|
||||
* table will be printed.
|
||||
* @param tableSet
|
||||
* when non null, serves as a filter. only entries corresponding to tables
|
||||
* in the tableSet are printed
|
||||
* @param region
|
||||
* when not null, serves as a filter; only log entries from this
|
||||
* region will be printed
|
||||
|
@ -138,15 +132,16 @@ public class WALPrettyPrinter {
|
|||
* Specifies an alternative to stdout for the destination of this
|
||||
* PrettyPrinter's output.
|
||||
*/
|
||||
public WALPrettyPrinter(boolean outputValues, boolean outputJSON,
|
||||
long sequence, String table, String region, String row, boolean persistentOutput,
|
||||
PrintStream out) {
|
||||
public WALPrettyPrinter(boolean outputValues, boolean outputJSON, long sequence,
|
||||
Set<String> tableSet, String region, String row, boolean outputOnlyRowKey,
|
||||
boolean persistentOutput, PrintStream out) {
|
||||
this.outputValues = outputValues;
|
||||
this.outputJSON = outputJSON;
|
||||
this.sequence = sequence;
|
||||
this.table = table;
|
||||
this.tableSet = tableSet;
|
||||
this.region = region;
|
||||
this.row = row;
|
||||
this.outputOnlyRowKey = outputOnlyRowKey;
|
||||
this.persistentOutput = persistentOutput;
|
||||
if (persistentOutput) {
|
||||
beginPersistentOutput();
|
||||
|
@ -195,11 +190,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
|
||||
|
@ -213,7 +208,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
|
||||
|
@ -223,6 +218,22 @@ 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
|
||||
* initial position to start seeking the given WAL file
|
||||
*/
|
||||
public void setPosition(long position) {
|
||||
this.position = position;
|
||||
}
|
||||
|
||||
/**
|
||||
* enables output as a single, persistent list. at present, only relevant in
|
||||
* the case of JSON output.
|
||||
|
@ -313,7 +324,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) {
|
||||
|
@ -323,10 +335,10 @@ public class WALPrettyPrinter {
|
|||
continue;
|
||||
}
|
||||
// initialize list into which we will store atomic actions
|
||||
List<Map<String, Object>> actions = new ArrayList<Map<String, Object>>();
|
||||
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)));
|
||||
}
|
||||
|
@ -335,7 +347,6 @@ public class WALPrettyPrinter {
|
|||
actions.add(op);
|
||||
}
|
||||
op.put("total_size_sum", CellUtil.estimatedHeapSizeOf(cell));
|
||||
|
||||
}
|
||||
if (actions.isEmpty()) {
|
||||
continue;
|
||||
|
@ -356,15 +367,7 @@ public class WALPrettyPrinter {
|
|||
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);
|
||||
out.println("row=" + op.get("row") +
|
||||
", column=" + op.get("family") + ":" + op.get("qualifier"));
|
||||
if (op.get("tag") != null) {
|
||||
out.println(" tag: " + op.get("tag"));
|
||||
}
|
||||
if (outputValues) {
|
||||
out.println(" value: " + op.get("value"));
|
||||
}
|
||||
out.println("cell total size sum: " + op.get("total_size_sum"));
|
||||
printCell(out, op, outputValues);
|
||||
}
|
||||
}
|
||||
out.println("edit heap size: " + entry.getEdit().heapSize());
|
||||
|
@ -378,31 +381,51 @@ public class WALPrettyPrinter {
|
|||
}
|
||||
}
|
||||
|
||||
private static Map<String, Object> toStringMap(Cell cell) {
|
||||
Map<String, Object> stringMap = new HashMap<String, Object>();
|
||||
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"));
|
||||
if (op.get("tag") != null) {
|
||||
out.println(" tag: " + op.get("tag"));
|
||||
}
|
||||
if (outputValues) {
|
||||
out.println(" value: " + op.get("value"));
|
||||
}
|
||||
out.println("cell total size sum: " + op.get("total_size_sum"));
|
||||
}
|
||||
|
||||
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()));
|
||||
stringMap.put("family", Bytes.toStringBinary(cell.getFamilyArray(), cell.getFamilyOffset(),
|
||||
cell.getFamilyLength()));
|
||||
stringMap.put("qualifier",
|
||||
Bytes.toStringBinary(cell.getQualifierArray(), cell.getQualifierOffset(),
|
||||
cell.getQualifierLength()));
|
||||
Bytes.toStringBinary(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()));
|
||||
|
||||
if (printRowKeyOnly) {
|
||||
return stringMap;
|
||||
}
|
||||
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());
|
||||
if (cell.getTagsLength() > 0) {
|
||||
List<String> tagsString = new ArrayList<String>();
|
||||
Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
|
||||
cell.getTagsLength());
|
||||
List<String> tagsString = new ArrayList<>();
|
||||
Iterator<Tag> tagsIterator =
|
||||
CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength());
|
||||
while (tagsIterator.hasNext()) {
|
||||
Tag tag = tagsIterator.next();
|
||||
tagsString.add((tag.getType()) + ":"
|
||||
+ Bytes.toStringBinary(tag.getBuffer(), tag.getTagOffset(), tag.getTagLength()));
|
||||
tagsString.add((tag.getType()) + ":" + Bytes
|
||||
.toStringBinary(tag.getBuffer(), tag.getTagOffset(), tag.getTagLength()));
|
||||
}
|
||||
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);
|
||||
}
|
||||
|
@ -422,11 +445,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");
|
||||
|
||||
|
@ -448,6 +474,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"));
|
||||
}
|
||||
|
@ -486,8 +515,4 @@ public class WALPrettyPrinter {
|
|||
}
|
||||
printer.endPersistentOutput();
|
||||
}
|
||||
|
||||
public void setPosition(long position) {
|
||||
this.position = position;
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue