HBASE-3636 a bug about deciding whether this key is a new key for the ROWCOL bloomfilter
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1081577 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
7c8f6406a5
commit
856065dee8
|
@ -157,6 +157,8 @@ Release 0.90.2 - Unreleased
|
|||
HBASE-3608 MemstoreFlusher error message doesnt include exception!
|
||||
HBASE-3633 ZKUtil::createSetData should only create a node when it
|
||||
nonexists (Guanpeng Xu via Stack)
|
||||
HBASE-3636 a bug about deciding whether this key is a new key for the ROWCOL
|
||||
bloomfilter (Liyin Tang via Stack)
|
||||
|
||||
IMPROVEMENTS
|
||||
HBASE-3542 MultiGet methods in Thrift
|
||||
|
|
|
@ -1493,7 +1493,8 @@ public class KeyValue implements Writable, HeapSize {
|
|||
short lrowlength = left.getRowLength();
|
||||
short rrowlength = right.getRowLength();
|
||||
// TsOffset = end of column data. just comparing Row+CF length of each
|
||||
return left.getTimestampOffset() == right.getTimestampOffset() &&
|
||||
return ((left.getTimestampOffset() - left.getOffset()) ==
|
||||
(right.getTimestampOffset() - right.getOffset())) &&
|
||||
matchingRows(left, lrowlength, right, rrowlength) &&
|
||||
compareColumns(left, lrowlength, right, rrowlength) == 0;
|
||||
}
|
||||
|
|
|
@ -2773,7 +2773,59 @@ public class TestHRegion extends HBaseTestCase {
|
|||
//////////////////////////////////////////////////////////////////////////////
|
||||
// Bloom filter test
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
public void testBloomFilterSize() throws IOException {
|
||||
byte [] tableName = Bytes.toBytes("testBloomFilterSize");
|
||||
byte [] row1 = Bytes.toBytes("row1");
|
||||
byte [] fam1 = Bytes.toBytes("fam1");
|
||||
byte [] qf1 = Bytes.toBytes("col");
|
||||
byte [] val1 = Bytes.toBytes("value1");
|
||||
// Create Table
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(fam1, Integer.MAX_VALUE,
|
||||
HColumnDescriptor.DEFAULT_COMPRESSION, false, true,
|
||||
HColumnDescriptor.DEFAULT_TTL, "rowcol");
|
||||
|
||||
HTableDescriptor htd = new HTableDescriptor(tableName);
|
||||
htd.addFamily(hcd);
|
||||
HRegionInfo info = new HRegionInfo(htd, null, null, false);
|
||||
Path path = new Path(DIR + "testBloomFilterSize");
|
||||
region = HRegion.createHRegion(info, path, conf);
|
||||
|
||||
int num_unique_rows = 10;
|
||||
int duplicate_multiplier =2;
|
||||
int num_storefiles = 4;
|
||||
|
||||
for (int f =0 ; f < num_storefiles; f++) {
|
||||
for (int i = 0; i < duplicate_multiplier; i ++) {
|
||||
for (int j = 0; j < num_unique_rows; j++) {
|
||||
Put put = new Put(Bytes.toBytes("row" + j));
|
||||
put.add(fam1, qf1, val1);
|
||||
region.put(put);
|
||||
}
|
||||
}
|
||||
region.flushcache();
|
||||
}
|
||||
//before compaction
|
||||
Store store = region.getStore(fam1);
|
||||
List<StoreFile> storeFiles = store.getStorefiles();
|
||||
for (StoreFile storefile : storeFiles) {
|
||||
StoreFile.Reader reader = storefile.getReader();
|
||||
reader.loadFileInfo();
|
||||
reader.loadBloomfilter();
|
||||
assertEquals(num_unique_rows, reader.getFilterEntries());
|
||||
}
|
||||
|
||||
region.compactStores(true);
|
||||
|
||||
//after compaction
|
||||
storeFiles = store.getStorefiles();
|
||||
for (StoreFile storefile : storeFiles) {
|
||||
StoreFile.Reader reader = storefile.getReader();
|
||||
reader.loadFileInfo();
|
||||
reader.loadBloomfilter();
|
||||
assertEquals(num_unique_rows, reader.getFilterEntries());
|
||||
}
|
||||
}
|
||||
|
||||
public void testAllColumnsWithBloomFilter() throws IOException {
|
||||
byte [] TABLE = Bytes.toBytes("testAllColumnsWithBloomFilter");
|
||||
byte [] FAMILY = Bytes.toBytes("family");
|
||||
|
|
Loading…
Reference in New Issue