Return-Path: Delivered-To: apmail-hbase-commits-archive@www.apache.org Received: (qmail 2405 invoked from network); 14 Mar 2011 21:54:31 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.3) by minotaur.apache.org with SMTP; 14 Mar 2011 21:54:31 -0000 Received: (qmail 54811 invoked by uid 500); 14 Mar 2011 21:54:31 -0000 Delivered-To: apmail-hbase-commits-archive@hbase.apache.org Received: (qmail 54777 invoked by uid 500); 14 Mar 2011 21:54:31 -0000 Mailing-List: contact commits-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@hbase.apache.org Delivered-To: mailing list commits@hbase.apache.org Received: (qmail 54769 invoked by uid 99); 14 Mar 2011 21:54:31 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 14 Mar 2011 21:54:31 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 14 Mar 2011 21:54:30 +0000 Received: by eris.apache.org (Postfix, from userid 65534) id 6705A238897A; Mon, 14 Mar 2011 21:54:08 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1081577 - in /hbase/trunk: CHANGES.txt src/main/java/org/apache/hadoop/hbase/KeyValue.java src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java Date: Mon, 14 Mar 2011 21:54:08 -0000 To: commits@hbase.apache.org From: stack@apache.org X-Mailer: svnmailer-1.0.8 Message-Id: <20110314215408.6705A238897A@eris.apache.org> Author: stack Date: Mon Mar 14 21:54:07 2011 New Revision: 1081577 URL: http://svn.apache.org/viewvc?rev=1081577&view=rev Log: HBASE-3636 a bug about deciding whether this key is a new key for the ROWCOL bloomfilter Modified: hbase/trunk/CHANGES.txt hbase/trunk/src/main/java/org/apache/hadoop/hbase/KeyValue.java hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java Modified: hbase/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/hbase/trunk/CHANGES.txt?rev=1081577&r1=1081576&r2=1081577&view=diff ============================================================================== --- hbase/trunk/CHANGES.txt (original) +++ hbase/trunk/CHANGES.txt Mon Mar 14 21:54:07 2011 @@ -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 Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/KeyValue.java URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/KeyValue.java?rev=1081577&r1=1081576&r2=1081577&view=diff ============================================================================== --- hbase/trunk/src/main/java/org/apache/hadoop/hbase/KeyValue.java (original) +++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/KeyValue.java Mon Mar 14 21:54:07 2011 @@ -1493,7 +1493,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/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java?rev=1081577&r1=1081576&r2=1081577&view=diff ============================================================================== --- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java (original) +++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java Mon Mar 14 21:54:07 2011 @@ -2773,7 +2773,59 @@ 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 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");