From 76a85b946733b84144505551e753691eea367b25 Mon Sep 17 00:00:00 2001 From: Guangxu Cheng Date: Thu, 28 Sep 2017 21:07:17 +0800 Subject: [PATCH] HBASE-18899 Make Fileinfo more readable in HFilePrettyPrinter --- .../hadoop/hbase/io/hfile/HFilePrettyPrinter.java | 22 +++++++++++++++++----- 1 file changed, 17 insertions(+), 5 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java index 45243509807..9fc6958d4a2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java @@ -59,10 +59,12 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.TagUtil; +import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper; @@ -529,15 +531,25 @@ public class HFilePrettyPrinter extends Configured implements Tool { out.println("Fileinfo:"); for (Map.Entry e : fileInfo.entrySet()) { out.print(FOUR_SPACES + Bytes.toString(e.getKey()) + " = "); - if (Bytes.compareTo(e.getKey(), Bytes.toBytes("MAX_SEQ_ID_KEY")) == 0) { - long seqid = Bytes.toLong(e.getValue()); - out.println(seqid); - } else if (Bytes.compareTo(e.getKey(), Bytes.toBytes("TIMERANGE")) == 0) { + if (Bytes.compareTo(e.getKey(), HStoreFile.MAX_SEQ_ID_KEY) == 0 + || Bytes.compareTo(e.getKey(), HStoreFile.DELETE_FAMILY_COUNT) == 0 + || Bytes.compareTo(e.getKey(), HStoreFile.EARLIEST_PUT_TS) == 0 + || Bytes.compareTo(e.getKey(), HFileWriterImpl.MAX_MEMSTORE_TS_KEY) == 0 + || Bytes.compareTo(e.getKey(), FileInfo.CREATE_TIME_TS) == 0) { + out.println(Bytes.toLong(e.getValue())); + } else if (Bytes.compareTo(e.getKey(), HStoreFile.TIMERANGE_KEY) == 0) { TimeRangeTracker timeRangeTracker = TimeRangeTracker.getTimeRangeTracker(e.getValue()); out.println(timeRangeTracker.getMin() + "...." + timeRangeTracker.getMax()); } else if (Bytes.compareTo(e.getKey(), FileInfo.AVG_KEY_LEN) == 0 - || Bytes.compareTo(e.getKey(), FileInfo.AVG_VALUE_LEN) == 0) { + || Bytes.compareTo(e.getKey(), FileInfo.AVG_VALUE_LEN) == 0 + || Bytes.compareTo(e.getKey(), HFileWriterImpl.KEY_VALUE_VERSION) == 0) { out.println(Bytes.toInt(e.getValue())); + } else if (Bytes.compareTo(e.getKey(), HStoreFile.MAJOR_COMPACTION_KEY) == 0 + || Bytes.compareTo(e.getKey(), HStoreFile.BULKLOAD_TIME_KEY) == 0 + || Bytes.compareTo(e.getKey(), HStoreFile.EXCLUDE_FROM_MINOR_COMPACTION_KEY) == 0) { + out.println(Bytes.toBoolean(e.getValue())); + } else if (Bytes.compareTo(e.getKey(), FileInfo.LASTKEY) == 0) { + out.println(new KeyValue.KeyOnlyKeyValue(e.getValue()).toString()); } else { out.println(Bytes.toStringBinary(e.getValue())); } -- 2.13.0