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:13:49 UTC
svn commit: r1181497 - in /hbase/branches/0.89/src:
main/java/org/apache/hadoop/hbase/KeyValue.java
test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
Author: nspiegelberg
Date: Tue Oct 11 02:13:47 2011
New Revision: 1181497
URL: http://svn.apache.org/viewvc?rev=1181497&view=rev
Log:
Fix a bloomfilter bug Summary: The previous diff is generated by arc diff The diff has been accepted.
This revision is generated by svn commit.
Trac Bug: #
Blame Rev:
Reviewed By:
Test Plan:
Passing unit tests
Revert Plan:
Database Impact:
Memcache Impact:
Other Notes:
EImportant:
- begin *PUBLIC* platform impact section -
Bugzilla: #
- end platform impact -
Modified:
hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/KeyValue.java
hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/KeyValue.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/KeyValue.java?rev=1181497&r1=1181496&r2=1181497&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/KeyValue.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/KeyValue.java Tue Oct 11 02:13:47 2011
@@ -1429,7 +1429,8 @@ public class KeyValue implements Writabl
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;
}
Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java?rev=1181497&r1=1181496&r2=1181497&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java Tue Oct 11 02:13:47 2011
@@ -91,7 +91,7 @@ public class TestHRegion extends HBaseTe
protected final byte[] qual3 = Bytes.toBytes("qual3");
protected final byte[] value1 = Bytes.toBytes("value1");
protected final byte[] value2 = Bytes.toBytes("value2");
- protected final byte [] row = Bytes.toBytes("rowA");
+ protected final byte[] row = Bytes.toBytes("rowA");
/**
* @see org.apache.hadoop.hbase.HBaseTestCase#setUp()
@@ -2658,6 +2658,62 @@ public class TestHRegion extends HBaseTe
//////////////////////////////////////////////////////////////////////////////
// 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 * duplicate_multiplier, reader.getEntries());
+ 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 * duplicate_multiplier * num_storefiles,
+ reader.getEntries());
+ assertEquals(10, reader.getFilterEntries());
+ }
+
+ }
public void testAllColumnsWithBloomFilter() throws IOException {
byte [] TABLE = Bytes.toBytes("testAllColumnsWithBloomFilter");