Return-Path: Delivered-To: apmail-lucene-hadoop-commits-archive@locus.apache.org Received: (qmail 48232 invoked from network); 7 Jul 2007 06:09:50 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.2) by minotaur.apache.org with SMTP; 7 Jul 2007 06:09:50 -0000 Received: (qmail 70027 invoked by uid 500); 7 Jul 2007 06:09:52 -0000 Delivered-To: apmail-lucene-hadoop-commits-archive@lucene.apache.org Received: (qmail 69935 invoked by uid 500); 7 Jul 2007 06:09:52 -0000 Mailing-List: contact hadoop-commits-help@lucene.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hadoop-dev@lucene.apache.org Delivered-To: mailing list hadoop-commits@lucene.apache.org Received: (qmail 69920 invoked by uid 99); 7 Jul 2007 06:09:52 -0000 Received: from herse.apache.org (HELO herse.apache.org) (140.211.11.133) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 06 Jul 2007 23:09:52 -0700 X-ASF-Spam-Status: No, hits=-99.5 required=10.0 tests=ALL_TRUSTED,NO_REAL_NAME X-Spam-Check-By: apache.org Received: from [140.211.11.3] (HELO eris.apache.org) (140.211.11.3) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 06 Jul 2007 23:09:48 -0700 Received: by eris.apache.org (Postfix, from userid 65534) id 839FC1A981A; Fri, 6 Jul 2007 23:09:27 -0700 (PDT) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r554144 - in /lucene/hadoop/trunk/src/contrib/hbase: ./ src/java/org/apache/hadoop/hbase/ src/java/org/onelab/filter/ src/test/org/apache/hadoop/hbase/ Date: Sat, 07 Jul 2007 06:09:27 -0000 To: hadoop-commits@lucene.apache.org From: jimk@apache.org X-Mailer: svnmailer-1.1.0 Message-Id: <20070707060927.839FC1A981A@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: jimk Date: Fri Jul 6 23:09:25 2007 New Revision: 554144 URL: http://svn.apache.org/viewvc?view=rev&rev=554144 Log: HADOOP-1415 Provide configurable per-column bloom filters. HADOOP-1466 Clean up visibility and javadoc issues in HBase. Added: lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestBloomFilters.java Modified: lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegiondirReader.java lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStore.java lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/BloomFilter.java lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/CountingBloomFilter.java lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/RetouchedBloomFilter.java Modified: lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt?view=diff&rev=554144&r1=554143&r2=554144 ============================================================================== --- lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt (original) +++ lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt Fri Jul 6 23:09:25 2007 @@ -51,3 +51,6 @@ Adds a row filtering interface and two implemenentations: A page scanner, and a regex row/column-data matcher. (James Kennedy via Stack) 31. HADOOP-1566 Key-making utility + 32. HADOOP-1415 Provide configurable per-column bloom filters. + HADOOP-1466 Clean up visibility and javadoc issues in HBase. + Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegiondirReader.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegiondirReader.java?view=diff&rev=554144&r1=554143&r2=554144 ============================================================================== --- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegiondirReader.java (original) +++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegiondirReader.java Fri Jul 6 23:09:25 2007 @@ -58,7 +58,7 @@ if (!fs.exists(parentdir)) { throw new FileNotFoundException(parentdirName); } - if (!fs.isDirectory(parentdir)) { + if (!fs.getFileStatus(parentdir).isDir()) { throw new IOException(parentdirName + " not a directory"); } // Look for regions in parentdir. Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStore.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStore.java?view=diff&rev=554144&r1=554143&r2=554144 ============================================================================== --- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStore.java (original) +++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStore.java Fri Jul 6 23:09:25 2007 @@ -29,6 +29,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FSDataInputStream; @@ -52,7 +53,7 @@ * be called directly by any writer, but rather by an HRegion manager. */ class HStore implements HConstants { - private static final Log LOG = LogFactory.getLog(HStore.class); + static final Log LOG = LogFactory.getLog(HStore.class); static final String COMPACTION_DIR = "compaction.tmp"; static final String WORKING_COMPACTION = "compaction.inprogress"; @@ -299,6 +300,10 @@ private void loadOrCreateBloomFilter() throws IOException { Path filterFile = new Path(filterDir, BLOOMFILTER_FILE_NAME); if(fs.exists(filterFile)) { + if (LOG.isDebugEnabled()) { + LOG.debug("loading bloom filter for " + family.getName()); + } + switch(family.bloomFilter.filterType) { case BloomFilterDescriptor.BLOOMFILTER: @@ -317,6 +322,10 @@ fs.close(); } else { + if (LOG.isDebugEnabled()) { + LOG.debug("creating bloom filter for " + family.getName()); + } + switch(family.bloomFilter.filterType) { case BloomFilterDescriptor.BLOOMFILTER: @@ -342,18 +351,33 @@ * @throws IOException */ private void flushBloomFilter() throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("flushing bloom filter for " + family.getName()); + } FSDataOutputStream out = fs.create(new Path(filterDir, BLOOMFILTER_FILE_NAME)); bloomFilter.write(out); out.close(); + if (LOG.isDebugEnabled()) { + LOG.debug("flushed bloom filter for " + family.getName()); + } } /** Generates a bloom filter key from the row and column keys */ Key getBloomFilterKey(HStoreKey k) { StringBuilder s = new StringBuilder(k.getRow().toString()); s.append(k.getColumn().toString()); - return new Key(s.toString().getBytes()); + + byte[] bytes = null; + try { + bytes = s.toString().getBytes(HConstants.UTF8_ENCODING); + + } catch (UnsupportedEncodingException e) { + e.printStackTrace(); + assert(false); + } + return new Key(bytes); } /** @@ -372,8 +396,14 @@ // Note - the key being passed to us is always a HStoreKey if(bloomFilter.membershipTest(getBloomFilterKey((HStoreKey)key))) { + if (LOG.isDebugEnabled()) { + LOG.debug("bloom filter reported that key exists"); + } return super.get(key, val); } + if (LOG.isDebugEnabled()) { + LOG.debug("bloom filter reported that key does not exist"); + } return null; } @@ -383,8 +413,14 @@ // Note - the key being passed to us is always a HStoreKey if(bloomFilter.membershipTest(getBloomFilterKey((HStoreKey)key))) { + if (LOG.isDebugEnabled()) { + LOG.debug("bloom filter reported that key exists"); + } return super.getClosest(key, val); } + if (LOG.isDebugEnabled()) { + LOG.debug("bloom filter reported that key does not exist"); + } return null; } } @@ -1083,8 +1119,8 @@ // Iterate through all the MapFiles for(Map.Entry e: mapFiles.entrySet()) { HStoreFile curHSF = e.getValue(); - long size = fs.getLength( - new Path(curHSF.getMapFilePath(), MapFile.DATA_FILE_NAME)); + long size = fs.getFileStatus( + new Path(curHSF.getMapFilePath(), MapFile.DATA_FILE_NAME)).getLen(); if(size > maxSize) { // This is the largest one so far maxSize = size; mapIndex = e.getKey(); Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/BloomFilter.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/BloomFilter.java?view=diff&rev=554144&r1=554143&r2=554144 ============================================================================== --- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/BloomFilter.java (original) +++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/BloomFilter.java Fri Jul 6 23:09:25 2007 @@ -79,6 +79,7 @@ } int[] h = hash.hash(key); + hash.clear(); for(int i = 0; i < nbHash; i++) { vector[h[i]] = true; @@ -108,6 +109,7 @@ } int[] h = hash.hash(key); + hash.clear(); for(int i = 0; i < nbHash; i++) { if(!vector[h[i]]) { return false; Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/CountingBloomFilter.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/CountingBloomFilter.java?view=diff&rev=554144&r1=554143&r2=554144 ============================================================================== --- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/CountingBloomFilter.java (original) +++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/CountingBloomFilter.java Fri Jul 6 23:09:25 2007 @@ -74,6 +74,7 @@ } int[] h = hash.hash(key); + hash.clear(); for(int i = 0; i < nbHash; i++) { vector[h[i]]++; @@ -95,6 +96,7 @@ } int[] h = hash.hash(key); + hash.clear(); for(int i = 0; i < nbHash; i++) { if(vector[h[i]] >= 1) { @@ -125,6 +127,7 @@ } int[] h = hash.hash(key); + hash.clear(); for(int i = 0; i < nbHash; i++) { if(vector[h[i]] == 0) { Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/RetouchedBloomFilter.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/RetouchedBloomFilter.java?view=diff&rev=554144&r1=554143&r2=554144 ============================================================================== --- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/RetouchedBloomFilter.java (original) +++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/RetouchedBloomFilter.java Fri Jul 6 23:09:25 2007 @@ -95,6 +95,7 @@ } int[] h = hash.hash(key); + hash.clear(); for(int i = 0; i < nbHash; i++) { vector[h[i]] = true; @@ -114,6 +115,7 @@ } int[] h = hash.hash(key); + hash.clear(); for(int i = 0; i < nbHash; i++) { fpVector[h[i]].add(key); @@ -328,6 +330,7 @@ } int[] h = hash.hash(k); + hash.clear(); for(int i = 0; i < nbHash; i++) { vector[h[i]].remove(k); Added: lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestBloomFilters.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestBloomFilters.java?view=auto&rev=554144 ============================================================================== --- lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestBloomFilters.java (added) +++ lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestBloomFilters.java Fri Jul 6 23:09:25 2007 @@ -0,0 +1,205 @@ +/** + * Copyright 2007 The Apache Software Foundation + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase; + +import org.apache.log4j.Level; +import org.apache.log4j.Logger; + +import org.apache.hadoop.io.Text; + +/** Tests per-column bloom filters */ +public class TestBloomFilters extends HBaseClusterTestCase { + private static final Text CONTENTS = new Text("contents:"); + + private HTableDescriptor desc = null; + private HClient client = null; + + private static final Text[] rows = { + new Text("wmjwjzyv"), + new Text("baietibz"), + new Text("guhsgxnv"), + new Text("mhnqycto"), + new Text("xcyqafgz"), + new Text("zidoamgb"), + new Text("tftfirzd"), + new Text("okapqlrg"), + new Text("yccwzwsq"), + new Text("qmonufqu"), + new Text("wlsctews"), + new Text("mksdhqri"), + new Text("wxxllokj"), + new Text("eviuqpls"), + new Text("bavotqmj"), + new Text("yibqzhdl"), + new Text("csfqmsyr"), + new Text("guxliyuh"), + new Text("pzicietj"), + new Text("qdwgrqwo"), + new Text("ujfzecmi"), + new Text("dzeqfvfi"), + new Text("phoegsij"), + new Text("bvudfcou"), + new Text("dowzmciz"), + new Text("etvhkizp"), + new Text("rzurqycg"), + new Text("krqfxuge"), + new Text("gflcohtd"), + new Text("fcrcxtps"), + new Text("qrtovxdq"), + new Text("aypxwrwi"), + new Text("dckpyznr"), + new Text("mdaawnpz"), + new Text("pakdfvca"), + new Text("xjglfbez"), + new Text("xdsecofi"), + new Text("sjlrfcab"), + new Text("ebcjawxv"), + new Text("hkafkjmy"), + new Text("oimmwaxo"), + new Text("qcuzrazo"), + new Text("nqydfkwk"), + new Text("frybvmlb"), + new Text("amxmaqws"), + new Text("gtkovkgx"), + new Text("vgwxrwss"), + new Text("xrhzmcep"), + new Text("tafwziil"), + new Text("erjmncnv"), + new Text("heyzqzrn"), + new Text("sowvyhtu"), + new Text("heeixgzy"), + new Text("ktcahcob"), + new Text("ljhbybgg"), + new Text("jiqfcksl"), + new Text("anjdkjhm"), + new Text("uzcgcuxp"), + new Text("vzdhjqla"), + new Text("svhgwwzq"), + new Text("zhswvhbp"), + new Text("ueceybwy"), + new Text("czkqykcw"), + new Text("ctisayir"), + new Text("hppbgciu"), + new Text("nhzgljfk"), + new Text("vaziqllf"), + new Text("narvrrij"), + new Text("kcevbbqi"), + new Text("qymuaqnp"), + new Text("pwqpfhsr"), + new Text("peyeicuk"), + new Text("kudlwihi"), + new Text("pkmqejlm"), + new Text("ylwzjftl"), + new Text("rhqrlqar"), + new Text("xmftvzsp"), + new Text("iaemtihk"), + new Text("ymsbrqcu"), + new Text("yfnlcxto"), + new Text("nluqopqh"), + new Text("wmrzhtox"), + new Text("qnffhqbl"), + new Text("zypqpnbw"), + new Text("oiokhatd"), + new Text("mdraddiu"), + new Text("zqoatltt"), + new Text("ewhulbtm"), + new Text("nmswpsdf"), + new Text("xsjeteqe"), + new Text("ufubcbma"), + new Text("phyxvrds"), + new Text("vhnfldap"), + new Text("zrrlycmg"), + new Text("becotcjx"), + new Text("wvbubokn"), + new Text("avkgiopr"), + new Text("mbqqxmrv"), + new Text("ibplgvuu"), + new Text("dghvpkgc") + }; + + private static final Text[] testKeys = { + new Text("abcdefgh"), + new Text("ijklmnop"), + new Text("qrstuvwx"), + new Text("yzabcdef") + }; + + /** constructor */ + public TestBloomFilters() { + super(); + conf.set("hbase.hregion.maxunflushed", "90"); // flush cache every 100 writes + conf.set("hbase.regionserver.maxlogentries", "90"); // and roll log too + Logger.getLogger(HRegion.class).setLevel(Level.DEBUG); + Logger.getLogger(HStore.class).setLevel(Level.DEBUG); + } + + @Override + public void setUp() { + try { + super.setUp(); + this.client = new HClient(conf); + this.desc = new HTableDescriptor("test"); + desc.addFamily( + new HColumnDescriptor(CONTENTS, 1, HColumnDescriptor.CompressionType.NONE, + false, Integer.MAX_VALUE, + new BloomFilterDescriptor( // if we insert 1000 values + BloomFilterDescriptor.BLOOMFILTER, // plain old bloom filter + 12499, // number of bits + 4 // number of hash functions + ))); // false positive = 0.0000001 + client.createTable(desc); + client.openTable(desc.getName()); + + // Store some values + + for(int i = 0; i < 100; i++) { + Text row = rows[i]; + String value = row.toString(); + long lockid = client.startUpdate(rows[i]); + client.put(lockid, CONTENTS, value.getBytes(HConstants.UTF8_ENCODING)); + client.commit(lockid); + } + } catch (Exception e) { + e.printStackTrace(); + fail(); + } + } + + /** the test */ + public void testBloomFilters() { + try { + // Give cache flusher and log roller a chance to run + // Otherwise we'll never hit the bloom filter, just the memcache + Thread.sleep(conf.getLong(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000) * 2); + + } catch (InterruptedException e) { + // ignore + } + + try { + for(int i = 0; i < testKeys.length; i++) { + byte[] value = client.get(testKeys[i], CONTENTS); + if(value != null && value.length != 0) { + System.err.println("non existant key: " + testKeys[i] + + " returned value: " + new String(value, HConstants.UTF8_ENCODING)); + } + } + } catch (Exception e) { + e.printStackTrace(); + fail(); + } + } +}