hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From chia7...@apache.org
Subject hbase git commit: HBASE-18899 Make Fileinfo more readable in HFilePrettyPrinter
Date Sat, 07 Oct 2017 19:43:32 GMT
Repository: hbase
Updated Branches:
  refs/heads/branch-2 9cd7619b5 -> 9ae275058


HBASE-18899 Make Fileinfo more readable in HFilePrettyPrinter

Signed-off-by: Chia-Ping Tsai <chia7712@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/9ae27505
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/9ae27505
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/9ae27505

Branch: refs/heads/branch-2
Commit: 9ae2750588594a8d02e63d00008e1f61967314b0
Parents: 9cd7619
Author: Guangxu Cheng <guangxucheng@gmail.com>
Authored: Fri Sep 29 01:10:59 2017 +0800
Committer: Chia-Ping Tsai <chia7712@gmail.com>
Committed: Sun Oct 8 03:27:38 2017 +0800

----------------------------------------------------------------------
 .../hbase/io/hfile/HFilePrettyPrinter.java      | 26 +++++++++++++++-----
 1 file changed, 20 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/9ae27505/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
----------------------------------------------------------------------
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 4524350..eed73df 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,27 @@ public class HFilePrettyPrinter extends Configured implements Tool {
     out.println("Fileinfo:");
     for (Map.Entry<byte[], byte[]> 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.equals(e.getKey(), HStoreFile.MAX_SEQ_ID_KEY)
+          || Bytes.equals(e.getKey(), HStoreFile.DELETE_FAMILY_COUNT)
+          || Bytes.equals(e.getKey(), HStoreFile.EARLIEST_PUT_TS)
+          || 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());
         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) {
+      } else if (Bytes.equals(e.getKey(), FileInfo.AVG_KEY_LEN)
+          || 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()));
+      } 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 {
         out.println(Bytes.toStringBinary(e.getValue()));
       }


Mime
View raw message