Author: jgray
Date: Thu Oct 21 00:28:22 2010
New Revision: 1025798

URL: http://svn.apache.org/viewvc?rev=1025798&view=rev
Log:
HBASE-3132  Print TimestampRange and BloomFilters in HFile pretty print

Modified:
    hbase/trunk/CHANGES.txt
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java

Modified: hbase/trunk/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/hbase/trunk/CHANGES.txt?rev=1025798&r1=1025797&r2=1025798&view=diff
==============================================================================
--- hbase/trunk/CHANGES.txt (original)
+++ hbase/trunk/CHANGES.txt Thu Oct 21 00:28:22 2010
@@ -1021,6 +1021,7 @@ Release 0.21.0 - Unreleased
    HBASE-3128  On assign, if ConnectException, reassign another server
    HBASE-3133  Only log compaction requests when a request is actually added
                to the queue
+   HBASE-3132  Print TimestampRange and BloomFilters in HFile pretty print
 
   NEW FEATURES
    HBASE-1961  HBase EC2 scripts

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
URL: 
http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java?rev=1025798&r1=1025797&r2=1025798&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java 
(original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java Thu 
Oct 21 00:28:22 2010
@@ -53,9 +53,13 @@ import org.apache.hadoop.hbase.HRegionIn
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.io.HbaseMapWritable;
 import org.apache.hadoop.hbase.io.HeapSize;
+import org.apache.hadoop.hbase.regionserver.TimeRangeTracker;
+import org.apache.hadoop.hbase.util.BloomFilter;
+import org.apache.hadoop.hbase.util.ByteBloomFilter;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.RawComparator;
 import org.apache.hadoop.io.Writable;
@@ -1944,10 +1948,32 @@ public class HFile {
             if (Bytes.compareTo(e.getKey(), 
Bytes.toBytes("MAX_SEQ_ID_KEY"))==0) {
               long seqid = Bytes.toLong(e.getValue());
               System.out.println(seqid);
+            } else if (Bytes.compareTo(e.getKey(),
+                Bytes.toBytes("TIMERANGE")) == 0) {
+              TimeRangeTracker timeRangeTracker = new TimeRangeTracker();
+              Writables.copyWritable(e.getValue(), timeRangeTracker);
+              System.out.println(timeRangeTracker.getMinimumTimestamp() +
+                  "...." + timeRangeTracker.getMaximumTimestamp());
+            } else if (Bytes.compareTo(e.getKey(), FileInfo.AVG_KEY_LEN) == 0 
||
+                Bytes.compareTo(e.getKey(), FileInfo.AVG_VALUE_LEN) == 0) {
+              System.out.println(Bytes.toInt(e.getValue()));
             } else {
               System.out.println(Bytes.toStringBinary(e.getValue()));
             }
           }
+
+          //Printing bloom information
+          ByteBuffer b = reader.getMetaBlock("BLOOM_FILTER_META", false);
+          if (b!= null) {
+            BloomFilter bloomFilter = new ByteBloomFilter(b);
+            System.out.println("BloomSize: " + bloomFilter.getByteSize());
+            System.out.println("No of Keys in bloom: " +
+                bloomFilter.getKeyCount());
+            System.out.println("Max Keys for bloom: " +
+                bloomFilter.getMaxKeys());
+          } else {
+            System.out.println("Could not get bloom data from meta block");
+          }
         }
         reader.close();
       }


Reply via email to