HBASE-18899 Make Fileinfo more readable in HFilePrettyPrinter
Signed-off-by: Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
parent
9cd7619b5c
commit
9ae2750588
|
@ -59,10 +59,12 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
import org.apache.hadoop.hbase.HRegionInfo;
|
import org.apache.hadoop.hbase.HRegionInfo;
|
||||||
|
import org.apache.hadoop.hbase.KeyValue;
|
||||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.Tag;
|
import org.apache.hadoop.hbase.Tag;
|
||||||
import org.apache.hadoop.hbase.TagUtil;
|
import org.apache.hadoop.hbase.TagUtil;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.HStoreFile;
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
import org.apache.yetus.audience.InterfaceStability;
|
import org.apache.yetus.audience.InterfaceStability;
|
||||||
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
|
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
|
||||||
|
@ -529,15 +531,27 @@ public class HFilePrettyPrinter extends Configured implements Tool {
|
||||||
out.println("Fileinfo:");
|
out.println("Fileinfo:");
|
||||||
for (Map.Entry<byte[], byte[]> e : fileInfo.entrySet()) {
|
for (Map.Entry<byte[], byte[]> e : fileInfo.entrySet()) {
|
||||||
out.print(FOUR_SPACES + Bytes.toString(e.getKey()) + " = ");
|
out.print(FOUR_SPACES + Bytes.toString(e.getKey()) + " = ");
|
||||||
if (Bytes.compareTo(e.getKey(), Bytes.toBytes("MAX_SEQ_ID_KEY")) == 0) {
|
if (Bytes.equals(e.getKey(), HStoreFile.MAX_SEQ_ID_KEY)
|
||||||
long seqid = Bytes.toLong(e.getValue());
|
|| Bytes.equals(e.getKey(), HStoreFile.DELETE_FAMILY_COUNT)
|
||||||
out.println(seqid);
|
|| Bytes.equals(e.getKey(), HStoreFile.EARLIEST_PUT_TS)
|
||||||
} else if (Bytes.compareTo(e.getKey(), Bytes.toBytes("TIMERANGE")) == 0) {
|
|| Bytes.equals(e.getKey(), HFileWriterImpl.MAX_MEMSTORE_TS_KEY)
|
||||||
|
|| Bytes.equals(e.getKey(), FileInfo.CREATE_TIME_TS)
|
||||||
|
|| Bytes.equals(e.getKey(), HStoreFile.BULKLOAD_TIME_KEY)) {
|
||||||
|
out.println(Bytes.toLong(e.getValue()));
|
||||||
|
} else if (Bytes.equals(e.getKey(), HStoreFile.TIMERANGE_KEY)) {
|
||||||
TimeRangeTracker timeRangeTracker = TimeRangeTracker.getTimeRangeTracker(e.getValue());
|
TimeRangeTracker timeRangeTracker = TimeRangeTracker.getTimeRangeTracker(e.getValue());
|
||||||
out.println(timeRangeTracker.getMin() + "...." + timeRangeTracker.getMax());
|
out.println(timeRangeTracker.getMin() + "...." + timeRangeTracker.getMax());
|
||||||
} else if (Bytes.compareTo(e.getKey(), FileInfo.AVG_KEY_LEN) == 0
|
} else if (Bytes.equals(e.getKey(), FileInfo.AVG_KEY_LEN)
|
||||||
|| Bytes.compareTo(e.getKey(), FileInfo.AVG_VALUE_LEN) == 0) {
|
|| Bytes.equals(e.getKey(), FileInfo.AVG_VALUE_LEN)
|
||||||
|
|| Bytes.equals(e.getKey(), HFileWriterImpl.KEY_VALUE_VERSION)
|
||||||
|
|| Bytes.equals(e.getKey(), FileInfo.MAX_TAGS_LEN)) {
|
||||||
out.println(Bytes.toInt(e.getValue()));
|
out.println(Bytes.toInt(e.getValue()));
|
||||||
|
} else if (Bytes.equals(e.getKey(), HStoreFile.MAJOR_COMPACTION_KEY)
|
||||||
|
|| Bytes.equals(e.getKey(), FileInfo.TAGS_COMPRESSED)
|
||||||
|
|| Bytes.equals(e.getKey(), HStoreFile.EXCLUDE_FROM_MINOR_COMPACTION_KEY)) {
|
||||||
|
out.println(Bytes.toBoolean(e.getValue()));
|
||||||
|
} else if (Bytes.equals(e.getKey(), FileInfo.LASTKEY)) {
|
||||||
|
out.println(new KeyValue.KeyOnlyKeyValue(e.getValue()).toString());
|
||||||
} else {
|
} else {
|
||||||
out.println(Bytes.toStringBinary(e.getValue()));
|
out.println(Bytes.toStringBinary(e.getValue()));
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue