HBASE-3132 Print TimestampRange and BloomFilters in HFile pretty print

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1025798 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Jonathan Gray 2010-10-21 00:28:22 +00:00
parent 43dd80f935
commit 5200b27780
2 changed files with 27 additions and 0 deletions

View File

@ -1021,6 +1021,7 @@ Release 0.21.0 - Unreleased
HBASE-3128 On assign, if ConnectException, reassign another server HBASE-3128 On assign, if ConnectException, reassign another server
HBASE-3133 Only log compaction requests when a request is actually added HBASE-3133 Only log compaction requests when a request is actually added
to the queue to the queue
HBASE-3132 Print TimestampRange and BloomFilters in HFile pretty print
NEW FEATURES NEW FEATURES
HBASE-1961 HBase EC2 scripts HBASE-1961 HBase EC2 scripts

View File

@ -53,9 +53,13 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.io.HbaseMapWritable; import org.apache.hadoop.hbase.io.HbaseMapWritable;
import org.apache.hadoop.hbase.io.HeapSize; 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.Bytes;
import org.apache.hadoop.hbase.util.ClassSize; import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.hbase.util.FSUtils; 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.IOUtils;
import org.apache.hadoop.io.RawComparator; import org.apache.hadoop.io.RawComparator;
import org.apache.hadoop.io.Writable; 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) { if (Bytes.compareTo(e.getKey(), Bytes.toBytes("MAX_SEQ_ID_KEY"))==0) {
long seqid = Bytes.toLong(e.getValue()); long seqid = Bytes.toLong(e.getValue());
System.out.println(seqid); 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 { } else {
System.out.println(Bytes.toStringBinary(e.getValue())); 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(); reader.close();
} }