Index: src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java =================================================================== --- src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java (revision 1081528) +++ src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java (working copy) @@ -2773,7 +2773,59 @@ ////////////////////////////////////////////////////////////////////////////// // 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 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"); Index: src/main/java/org/apache/hadoop/hbase/KeyValue.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/KeyValue.java (revision 1081528) +++ src/main/java/org/apache/hadoop/hbase/KeyValue.java (working copy) @@ -1493,7 +1493,8 @@ 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; }