You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ns...@apache.org on 2011/10/11 04:02:30 UTC

svn commit: r1181368 - /hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java

Author: nspiegelberg
Date: Tue Oct 11 02:02:26 2011
New Revision: 1181368

URL: http://svn.apache.org/viewvc?rev=1181368&view=rev
Log:
Debugging information while printing HFiles

Summary:
Added code to print TimestampRange and BloomFilter information while pretty
printing HFiles

Test Plan:
Ran the program for a few HFiles and the output was as expected

DiffCamp Revision: 152877
Reviewed By: kannan
CC: kannan, kranganathan, pkhaitan, hbase@lists
Tasks:
#360606: More debugging information while printing HFiles

Revert Plan:
OK

Modified:
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java?rev=1181368&r1=1181367&r2=1181368&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java Tue Oct 11 02:02:26 2011
@@ -51,9 +51,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;
@@ -1914,10 +1918,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();
       }