You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ji...@apache.org on 2007/06/14 09:40:51 UTC

svn commit: r547159 - in /lucene/hadoop/trunk/src/contrib/hbase: ./ src/java/org/onelab/ src/java/org/onelab/filter/ src/test/org/onelab/ src/test/org/onelab/test/

Author: jimk
Date: Thu Jun 14 00:40:49 2007
New Revision: 547159

URL: http://svn.apache.org/viewvc?view=rev&rev=547159
Log:
HADOOP-1415 Integrate BSD licensed bloom filter implementation.

Added:
    lucene/hadoop/trunk/src/contrib/hbase/NOTICE.txt
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/
    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/DynamicBloomFilter.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/Filter.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/HashFunction.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/Key.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/RemoveScheme.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/RetouchedBloomFilter.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/onelab/
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/onelab/test/
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/onelab/test/StringKey.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/onelab/test/TestFilter.java
Modified:
    lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt

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=547159&r1=547158&r2=547159
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt Thu Jun 14 00:40:49 2007
@@ -31,3 +31,4 @@
  16. HADOOP-1479 Fix NPE in HStore#get if store file only has keys < passed key.
  17. HADOOP-1476 Distributed version of 'Performance Evaluation' script
  18. HADOOP-1469 Asychronous table creation
+ 19. HADOOP-1415 Integrate BSD licensed bloom filter implementation.

Added: lucene/hadoop/trunk/src/contrib/hbase/NOTICE.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/NOTICE.txt?view=auto&rev=547159
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/NOTICE.txt (added)
+++ lucene/hadoop/trunk/src/contrib/hbase/NOTICE.txt Thu Jun 14 00:40:49 2007
@@ -0,0 +1,5 @@
+This product includes software developed by The Apache Software
+Foundation (http://www.apache.org/).
+
+In addition, this product includes software developed by European Commission
+project OneLab (http://www.one-lab.org)

Added: 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=auto&rev=547159
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/BloomFilter.java (added)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/BloomFilter.java Thu Jun 14 00:40:49 2007
@@ -0,0 +1,226 @@
+/**
+ *
+ * Copyright (c) 2005, European Commission project OneLab under contract 034819 (http://www.one-lab.org)
+ * All rights reserved.
+ * Redistribution and use in source and binary forms, with or 
+ * without modification, are permitted provided that the following 
+ * conditions are met:
+ *  - Redistributions of source code must retain the above copyright 
+ *    notice, this list of conditions and the following disclaimer.
+ *  - Redistributions in binary form must reproduce the above copyright 
+ *    notice, this list of conditions and the following disclaimer in 
+ *    the documentation and/or other materials provided with the distribution.
+ *  - Neither the name of the University Catholique de Louvain - UCL
+ *    nor the names of its contributors may be used to endorse or 
+ *    promote products derived from this software without specific prior 
+ *    written permission.
+ *    
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS 
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT 
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS 
+ * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE 
+ * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, 
+ * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, 
+ * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; 
+ * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER 
+ * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT 
+ * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN 
+ * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 
+ * POSSIBILITY OF SUCH DAMAGE.
+ */
+package org.onelab.filter;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+/**
+ * Implements a <i>Bloom filter</i>, as defined by Bloom in 1970.
+ * <p>
+ * The Bloom filter is a data structure that was introduced in 1970 and that has been adopted by 
+ * the networking research community in the past decade thanks to the bandwidth efficiencies that it
+ * offers for the transmission of set membership information between networked hosts.  A sender encodes 
+ * the information into a bit vector, the Bloom filter, that is more compact than a conventional 
+ * representation. Computation and space costs for construction are linear in the number of elements.  
+ * The receiver uses the filter to test whether various elements are members of the set. Though the 
+ * filter will occasionally return a false positive, it will never return a false negative. When creating 
+ * the filter, the sender can choose its desired point in a trade-off between the false positive rate and the size. 
+ * 
+ * @author <a href="mailto:donnet@ucl.ac.be">Benoit Donnet</a> - Universite Catholique de Louvain - Faculte des Sciences Appliquees - Departement d'Ingenierie Informatique.
+ * contract <a href="http://www.one-lab.org">European Commission One-Lab Project 034819</a>.
+ *
+ * @version 1.0 - 2 Feb. 07
+ * 
+ * @see org.onelab.filter.Filter The general behavior of a filter
+ * 
+ * @see <a href="http://portal.acm.org/citation.cfm?id=362692&dl=ACM&coll=portal">Space/Time Trade-Offs in Hash Coding with Allowable Errors</a>
+ */
+public class BloomFilter extends Filter {
+  /** The bit vector. */
+  boolean[] vector;
+
+  /** Default constructor - use with readFields */
+  public BloomFilter() {}
+  
+  /**
+   * Constructor
+   * @param vectorSize The vector size of <i>this</i> filter.
+   * @param nbHash The number of hash function to consider.
+   */
+  public BloomFilter(int vectorSize, int nbHash){
+    super(vectorSize, nbHash);
+
+    vector = new boolean[this.vectorSize];
+  }//end constructor
+
+  @Override
+  public void add(Key key) {
+    if(key == null) {
+      throw new NullPointerException("key cannot be null");
+    }
+
+    int[] h = hash.hash(key);
+
+    for(int i = 0; i < nbHash; i++) {
+      vector[h[i]] = true;
+    }
+  }//end add()
+
+  @Override
+  public void and(Filter filter){
+    if(filter == null
+        || !(filter instanceof BloomFilter)
+        || filter.vectorSize != this.vectorSize
+        || filter.nbHash != this.nbHash) {
+      throw new IllegalArgumentException("filters cannot be and-ed");
+    }
+
+    BloomFilter bf = (BloomFilter)filter;
+
+    for(int i = 0; i < vectorSize; i++) {
+      this.vector[i] &= bf.vector[i];
+    }
+  }//end and()
+
+  @Override
+  public boolean membershipTest(Key key){
+    if(key == null) {
+      throw new NullPointerException("key cannot be null");
+    }
+
+    int[] h = hash.hash(key);
+    for(int i = 0; i < nbHash; i++) {
+      if(!vector[h[i]]) {
+        return false;
+      }
+    }
+    return true;
+  }//end memberhsipTest()
+
+  @Override
+  public void not(){
+    for(int i = 0; i < vectorSize; i++) {
+      vector[i] = !vector[i];
+    }
+  }//end not()
+
+  @Override
+  public void or(Filter filter){
+    if(filter == null
+        || !(filter instanceof BloomFilter)
+        || filter.vectorSize != this.vectorSize
+        || filter.nbHash != this.nbHash) {
+      throw new IllegalArgumentException("filters cannot be or-ed");
+    }
+
+    BloomFilter bf = (BloomFilter)filter;
+
+    for(int i = 0; i < vectorSize; i++) {
+      this.vector[i] |= bf.vector[i];
+    }
+  }//end or()
+
+  @Override
+  public void xor(Filter filter){
+    if(filter == null
+        || !(filter instanceof BloomFilter)
+        || filter.vectorSize != this.vectorSize
+        || filter.nbHash != this.nbHash) {
+      throw new IllegalArgumentException("filters cannot be xor-ed");
+    }
+
+    BloomFilter bf = (BloomFilter)filter;
+
+    for(int i = 0; i < vectorSize; i++) {
+      this.vector[i] = (this.vector[i] && !bf.vector[i])
+      || (!this.vector[i] && bf.vector[i]);
+    }
+  }//and xor()
+
+  /** Returns a String representation of <i>this</i> Bloom filter. */
+  @Override
+  public String toString(){
+    StringBuilder res = new StringBuilder();
+
+    for(int i = 0; i < vectorSize; i++) {
+      res.append(vector[i] ? "1" : "0");
+    }
+    return res.toString();
+  }//end toString()
+
+  /** Returns a shallow copy of <i>this</i> Bloom filter. */
+  @Override
+  public Object clone(){
+    BloomFilter bf = new BloomFilter(vectorSize, nbHash);
+    bf.or(this);
+    return bf;
+  }//end clone()
+
+  @Override
+  public boolean equals(Object o) {
+    return this.compareTo(o) == 0;
+  }
+  
+  @Override
+  public int hashCode() {
+    int result = super.hashCode();
+    for(int i = 0; i < vector.length; i++) {
+      result ^= Boolean.valueOf(vector[i]).hashCode();
+    }
+    return result;
+  }
+
+  // Writable
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    super.write(out);
+    for(int i = 0; i < vector.length; i++) {
+      out.writeBoolean(vector[i]);
+    }
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    super.readFields(in);
+    vector = new boolean[vectorSize];
+    for(int i = 0; i < vector.length; i++) {
+      vector[i] = in.readBoolean();
+    }
+  }
+
+  // Comparable
+  
+  @Override
+  public int compareTo(Object o) {
+    int result = super.compareTo(o);
+    
+    BloomFilter other = (BloomFilter)o;
+      
+    for(int i = 0; result == 0 && i < vector.length; i++) {
+      result = (vector[i] == other.vector[i] ? 0
+          : (vector[i] ? 1 : -1));
+    }
+    return result;
+  }// end compareTo
+}//end class

Added: 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=auto&rev=547159
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/CountingBloomFilter.java (added)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/CountingBloomFilter.java Thu Jun 14 00:40:49 2007
@@ -0,0 +1,241 @@
+/**
+ *
+ * Copyright (c) 2005, European Commission project OneLab under contract 034819 (http://www.one-lab.org)
+ * All rights reserved.
+ * Redistribution and use in source and binary forms, with or 
+ * without modification, are permitted provided that the following 
+ * conditions are met:
+ *  - Redistributions of source code must retain the above copyright 
+ *    notice, this list of conditions and the following disclaimer.
+ *  - Redistributions in binary form must reproduce the above copyright 
+ *    notice, this list of conditions and the following disclaimer in 
+ *    the documentation and/or other materials provided with the distribution.
+ *  - Neither the name of the University Catholique de Louvain - UCL
+ *    nor the names of its contributors may be used to endorse or 
+ *    promote products derived from this software without specific prior 
+ *    written permission.
+ *    
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS 
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT 
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS 
+ * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE 
+ * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, 
+ * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, 
+ * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; 
+ * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER 
+ * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT 
+ * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN 
+ * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 
+ * POSSIBILITY OF SUCH DAMAGE.
+ */
+package org.onelab.filter;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+/**
+ * Implements a <i>counting Bloom filter</i>, as defined by Fan et al. in a ToN
+ * 2000 paper.
+ * <p>
+ * A counting Bloom filter is an improvement to standard a Bloom filter as it
+ * allows dynamic additions and deletions of set membership information.  This 
+ * is achieved through the use of a counting vector instead of a bit vector.
+ * 
+ * @author <a href="mailto:donnet@ucl.ac.be">Benoit Donnet</a> - Universite Catholique de Louvain - Faculte des Sciences Appliquees - Departement d'Ingenierie Informatique.
+ * contract <a href="http://www.one-lab.org">European Commission One-Lab Project 034819</a>.
+ *
+ * @version 1.0 - 5 Feb. 07
+ * 
+ * @see org.onelab.filter.Filter The general behavior of a filter
+ * 
+ * @see <a href="http://portal.acm.org/citation.cfm?id=343571.343572">Summary cache: a scalable wide-area web cache sharing protocol</a>
+ */
+public final class CountingBloomFilter extends Filter {
+  /** Counter vector. */
+  private byte[] vector;
+
+  /** Default constructor - use with readFields */
+  public CountingBloomFilter() {}
+  
+  /**
+   * Constructor
+   * @param vectorSize The vector size of <i>this</i> filter.
+   * @param nbHash The number of hash function to consider.
+   */
+  public CountingBloomFilter(int vectorSize, int nbHash){
+    super(vectorSize, nbHash);
+    vector = new byte[vectorSize];
+  }//end constructor
+
+  @Override
+  public void add(Key key) {
+    if(key == null) {
+      throw new NullPointerException("key can not be null");
+    }
+
+    int[] h = hash.hash(key);
+
+    for(int i = 0; i < nbHash; i++) {
+      vector[h[i]]++;
+    }
+  }//end add()
+
+  /**
+   * Removes a specified key from <i>this</i> counting Bloom filter.
+   * <p>
+   * <b>Invariant</b>: nothing happens if the specified key does not belong to <i>this</i> counter Bloom filter.
+   * @param key The key to remove.
+   */
+  public void delete(Key key) {
+    if(key == null) {
+      throw new NullPointerException("Key may not be null");
+    }
+    if(!membershipTest(key)) {
+      throw new IllegalArgumentException("Key is not a member");
+    }
+
+    int[] h = hash.hash(key);
+
+    for(int i = 0; i < nbHash; i++) {
+      if(vector[h[i]] >= 1) {
+        vector[h[i]]--;
+      }
+    }
+  }//end delete
+
+  @Override
+  public void and(Filter filter){
+    if(filter == null
+        || !(filter instanceof CountingBloomFilter)
+        || filter.vectorSize != this.vectorSize
+        || filter.nbHash != this.nbHash) {
+      throw new IllegalArgumentException("filters cannot be and-ed");
+    }
+    CountingBloomFilter cbf = (CountingBloomFilter)filter;
+
+    for(int i = 0; i < vectorSize; i++) {
+      this.vector[i] &= cbf.vector[i];
+    }
+  }//end and()
+
+  @Override
+  public boolean membershipTest(Key key){
+    if(key == null) {
+      throw new NullPointerException("Key may not be null");
+    }
+
+    int[] h = hash.hash(key);
+
+    for(int i = 0; i < nbHash; i++) {
+      if(vector[h[i]] == 0) {
+        return false;
+      }
+    }
+
+    return true;
+  }//end membershipTest()
+
+  @Override
+  public void not(){
+    throw new UnsupportedOperationException("not() is undefined for "
+        + this.getClass().getName());
+  }//end not()
+
+  @Override
+  public void or(Filter filter){
+    if(filter == null
+        || !(filter instanceof CountingBloomFilter)
+        || filter.vectorSize != this.vectorSize
+        || filter.nbHash != this.nbHash) {
+      throw new IllegalArgumentException("filters cannot be or-ed");
+    }
+
+    CountingBloomFilter cbf = (CountingBloomFilter)filter;
+
+    for(int i = 0; i < vectorSize; i++) {
+      this.vector[i] |= cbf.vector[i];
+    }
+  }//end or()
+
+  @Override
+  @SuppressWarnings("unused")
+  public void xor(Filter filter){
+    throw new UnsupportedOperationException("xor() is undefined for "
+        + this.getClass().getName());
+  }//end xor()
+
+  @Override
+  public String toString(){
+    StringBuilder res = new StringBuilder();
+
+    for(int i = 0; i < vectorSize; i++) {
+      if(i > 0) {
+        res.append(" ");
+      }
+      res.append(vector[i]&0xff);
+    }
+
+    return res.toString();
+  }//end toString()
+
+  @Override
+  public Object clone(){
+    CountingBloomFilter cbf = new CountingBloomFilter(vectorSize, nbHash);
+    cbf.or(this);
+    return cbf;
+  }//end clone()
+
+  @Override
+  public boolean equals(Object o) {
+    return this.compareTo(o) == 0;
+  }
+  
+  @Override
+  public int hashCode() {
+    int result = super.hashCode();
+    for(int i = 0; i < vector.length; i++) {
+      result ^= Byte.valueOf(vector[i]).hashCode();
+    }
+    return result;
+  }
+
+  // Writable
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    super.write(out);
+    for(int i = 0; i < vector.length; i++) {
+      out.writeByte(vector[i]);
+    }
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    super.readFields(in);
+    vector = new byte[vectorSize];
+    for(int i = 0; i < vector.length; i++) {
+      vector[i] = in.readByte();
+    }
+  }
+
+  // Comparable
+  
+  @Override
+  public int compareTo(Object o) {
+    int result = super.compareTo(o);
+    
+    if(result == 0) {
+      CountingBloomFilter other = (CountingBloomFilter)o;
+      
+      for(int i = 0; i < vector.length; i++) {
+        result = vector[i] - other.vector[i];
+        
+        if(result != 0) {
+          break;
+        }
+      }
+    }
+    return result;
+  }// end compareTo
+}//end class

Added: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/DynamicBloomFilter.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/DynamicBloomFilter.java?view=auto&rev=547159
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/DynamicBloomFilter.java (added)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/DynamicBloomFilter.java Thu Jun 14 00:40:49 2007
@@ -0,0 +1,314 @@
+/**
+ *
+ * Copyright (c) 2005, European Commission project OneLab under contract 034819 (http://www.one-lab.org)
+ * All rights reserved.
+ * Redistribution and use in source and binary forms, with or 
+ * without modification, are permitted provided that the following 
+ * conditions are met:
+ *  - Redistributions of source code must retain the above copyright 
+ *    notice, this list of conditions and the following disclaimer.
+ *  - Redistributions in binary form must reproduce the above copyright 
+ *    notice, this list of conditions and the following disclaimer in 
+ *    the documentation and/or other materials provided with the distribution.
+ *  - Neither the name of the University Catholique de Louvain - UCL
+ *    nor the names of its contributors may be used to endorse or 
+ *    promote products derived from this software without specific prior 
+ *    written permission.
+ *    
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS 
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT 
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS 
+ * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE 
+ * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, 
+ * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, 
+ * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; 
+ * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER 
+ * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT 
+ * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN 
+ * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 
+ * POSSIBILITY OF SUCH DAMAGE.
+ */
+package org.onelab.filter;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+/**
+ * Implements a <i>dynamic Bloom filter</i>, as defined in the INFOCOM 2006 paper.
+ * <p>
+ * A dynamic Bloom filter (DBF) makes use of a <code>s * m</code> bit matrix but
+ * each of the <code>s</code> rows is a standard Bloom filter. The creation 
+ * process of a DBF is iterative. At the start, the DBF is a <code>1 * m</code>
+ * bit matrix, i.e., it is composed of a single standard Bloom filter.
+ * It assumes that <code>n<sub>r</sub></code> elements are recorded in the 
+ * initial bit vector, where <code>n<sub>r</sub> <= n</code> (<code>n</code> is
+ * the cardinality of the set <code>A</code> to record in the filter).  
+ * <p>
+ * As the size of <code>A</code> grows during the execution of the application,
+ * several keys must be inserted in the DBF.  When inserting a key into the DBF,
+ * one must first get an active Bloom filter in the matrix.  A Bloom filter is
+ * active when the number of recorded keys, <code>n<sub>r</sub></code>, is 
+ * strictly less than the current cardinality of <code>A</code>, <code>n</code>.
+ * If an active Bloom filter is found, the key is inserted and 
+ * <code>n<sub>r</sub></code> is incremented by one. On the other hand, if there
+ * is no active Bloom filter, a new one is created (i.e., a new row is added to
+ * the matrix) according to the current size of <code>A</code> and the element
+ * is added in this new Bloom filter and the <code>n<sub>r</sub></code> value of
+ * this new Bloom filter is set to one.  A given key is said to belong to the
+ * DBF if the <code>k</code> positions are set to one in one of the matrix rows.
+ * 
+ * @author <a href="mailto:donnet@ucl.ac.be">Benoit Donnet</a> - Universite Catholique de Louvain - Faculte des Sciences Appliquees - Departement d'Ingenierie Informatique.
+ * contract <a href="http://www.one-lab.org">European Commission One-Lab Project 034819</a>.
+ *
+ * @version 1.0 - 6 Feb. 07
+ * 
+ * @see org.onelab.filter.Filter The general behavior of a filter
+ * @see org.onelab.filter.BloomFilter A Bloom filter
+ * 
+ * @see <a href="http://www.cse.fau.edu/~jie/research/publications/Publication_files/infocom2006.pdf">Theory and Network Applications of Dynamic Bloom Filters</a>
+ */
+public class DynamicBloomFilter extends Filter {
+  /** 
+   * Threshold for the maximum number of key to record in a dynamic Bloom filter row.
+   */
+  int nr;
+
+  /**
+   * The number of keys recorded in the current standard active Bloom filter.
+   */
+  int currentNbRecord;
+
+  /**
+   * The matrix of Bloom filter.
+   */
+  BloomFilter[] matrix;
+
+  /**
+   * Constructor.
+   * <p>
+   * Builds an empty Dynamic Bloom filter.
+   * @param vectorSize The number of bits in the vector.
+   * @param nbHash The number of hash function to consider.
+   * @param nr The threshold for the maximum number of keys to record in a dynamic Bloom filter row.
+   */
+  public DynamicBloomFilter(int vectorSize, int nbHash, int nr) {
+    super(vectorSize, nbHash);
+
+    this.nr = nr;
+    this.currentNbRecord = 0;
+
+    matrix = new BloomFilter[1];
+    matrix[0] = new BloomFilter(this.vectorSize, this.nbHash);
+  }//end constructor
+
+  @Override
+  public void add(Key key){
+    if(key == null) {
+      throw new NullPointerException("Key can not be null");
+    }
+
+    BloomFilter bf = getActiveStandardBF();
+
+    if(bf == null){
+      addRow();
+      bf = matrix[matrix.length - 1];
+      currentNbRecord = 0;
+    }
+
+    bf.add(key);
+
+    currentNbRecord++;
+  }//end add()
+
+  @Override
+  public void and(Filter filter) {
+    if(filter == null
+        || !(filter instanceof DynamicBloomFilter)
+        || filter.vectorSize != this.vectorSize
+        || filter.nbHash != this.nbHash) {
+      throw new IllegalArgumentException("filters cannot be and-ed");
+    }
+
+    DynamicBloomFilter dbf = (DynamicBloomFilter)filter;
+
+    if(dbf.matrix.length != this.matrix.length || dbf.nr != this.nr) {
+      throw new IllegalArgumentException("filters cannot be and-ed");
+    }
+
+    for(int i = 0; i < matrix.length; i++) {
+      matrix[i].and(dbf.matrix[i]);
+    }
+  }//end and()
+
+  @Override
+  public boolean membershipTest(Key key){
+    if(key == null) {
+      return true;
+    }
+
+    for(int i = 0; i < matrix.length; i++) {
+      if(matrix[i].membershipTest(key)) {
+        return true;
+      }
+    }
+
+    return false;
+  }//end membershipTest()
+
+  @Override
+  public void not(){
+    for(int i = 0; i < matrix.length; i++) {
+      matrix[i].not();
+    }
+  }//end not()
+
+  @Override
+  public void or(Filter filter){
+    if(filter == null
+        || !(filter instanceof DynamicBloomFilter)
+        || filter.vectorSize != this.vectorSize
+        || filter.nbHash != this.nbHash) {
+      throw new IllegalArgumentException("filters cannot be or-ed");
+    }
+
+    DynamicBloomFilter dbf = (DynamicBloomFilter)filter;
+
+    if(dbf.matrix.length != this.matrix.length || dbf.nr != this.nr) {
+      throw new IllegalArgumentException("filters cannot be or-ed");
+    }
+    for(int i = 0; i < matrix.length; i++) {
+      matrix[i].or(dbf.matrix[i]);
+    }
+  }//end or()
+
+  @Override
+  public void xor(Filter filter){
+    if(filter == null
+        || !(filter instanceof DynamicBloomFilter)
+        || filter.vectorSize != this.vectorSize
+        || filter.nbHash != this.nbHash) {
+      throw new IllegalArgumentException("filters cannot be xor-ed");
+    }
+    DynamicBloomFilter dbf = (DynamicBloomFilter)filter;
+
+    if(dbf.matrix.length != this.matrix.length || dbf.nr != this.nr) {
+      throw new IllegalArgumentException("filters cannot be xor-ed");
+    }
+
+      for(int i = 0; i<matrix.length; i++) {
+        matrix[i].xor(dbf.matrix[i]);
+    }
+  }//end xor()
+
+  @Override
+  public String toString(){
+    StringBuilder res = new StringBuilder();
+
+    for(int i=0; i<matrix.length; i++) {
+      res.append(matrix[i]);
+      res.append(Character.LINE_SEPARATOR);
+    }
+    return res.toString();
+  }//end toString()
+
+  @Override
+  public Object clone(){
+    DynamicBloomFilter dbf = new DynamicBloomFilter(vectorSize, nbHash, nr);
+    dbf.currentNbRecord = this.currentNbRecord;
+    dbf.matrix = new BloomFilter[this.matrix.length];
+    for(int i = 0; i < this.matrix.length; i++) {
+      dbf.matrix[i] = (BloomFilter)this.matrix[i].clone();
+    }
+    return dbf;
+  }//end clone()
+
+  @Override
+  public boolean equals(Object o) {
+    return this.compareTo(o) == 0;
+  }
+  
+  @Override
+  public int hashCode() {
+    int result = super.hashCode();
+    for(int i = 0; i < matrix.length; i++) {
+      result ^= matrix[i].hashCode();
+    }
+    return result;
+  }
+
+  // Writable
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    super.write(out);
+    for(int i = 0; i < matrix.length; i++) {
+      matrix[i].write(out);
+    }
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    super.readFields(in);
+    matrix = new BloomFilter[vectorSize];
+    for(int i = 0; i < matrix.length; i++) {
+      matrix[i].readFields(in);
+    }
+  }
+
+  // Comparable
+  
+  @Override
+  public int compareTo(Object o) {
+    int result = super.compareTo(o);
+    
+    if(result == 0) {
+      DynamicBloomFilter other = (DynamicBloomFilter)o;
+      
+      result = this.nr - other.nr;
+      
+      if(result == 0) {
+        result = this.currentNbRecord - other.currentNbRecord;
+        
+        if(result == 0) {
+          for(int i = 0; i < matrix.length; i++) {
+            result = matrix[i].compareTo(other.matrix[i]) ;
+            
+            if(result != 0) {
+              break;
+            }
+          }
+        }
+      }
+    }
+    return result;
+  }// end compareTo
+
+  /**
+   * Adds a new row to <i>this</i> dynamic Bloom filter.
+   */
+  private void addRow(){
+    BloomFilter[] tmp = new BloomFilter[matrix.length + 1];
+
+    for(int i = 0; i < matrix.length; i++) {
+      tmp[i] = (BloomFilter)matrix[i].clone();
+    }
+
+    tmp[tmp.length-1] = new BloomFilter(vectorSize, nbHash);
+
+    matrix = tmp;
+  }//end addRow()
+
+  /**
+   * Returns the active standard Bloom filter in <i>this</i> dynamic Bloom filter.
+   * @return BloomFilter The active standard Bloom filter.
+   * 			 <code>Null</code> otherwise.
+   */
+  private BloomFilter getActiveStandardBF() {
+    if(currentNbRecord >= nr) {
+      return null;
+    }
+
+    return matrix[matrix.length - 1];
+  }//end getActiveStandardBF()
+}//end class

Added: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/Filter.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/Filter.java?view=auto&rev=547159
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/Filter.java (added)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/Filter.java Thu Jun 14 00:40:49 2007
@@ -0,0 +1,210 @@
+/**
+ *
+ * Copyright (c) 2005, European Commission project OneLab under contract 034819
+ * (http://www.one-lab.org)
+ * 
+ * All rights reserved.
+ * Redistribution and use in source and binary forms, with or 
+ * without modification, are permitted provided that the following 
+ * conditions are met:
+ *  - Redistributions of source code must retain the above copyright 
+ *    notice, this list of conditions and the following disclaimer.
+ *  - Redistributions in binary form must reproduce the above copyright 
+ *    notice, this list of conditions and the following disclaimer in 
+ *    the documentation and/or other materials provided with the distribution.
+ *  - Neither the name of the University Catholique de Louvain - UCL
+ *    nor the names of its contributors may be used to endorse or 
+ *    promote products derived from this software without specific prior 
+ *    written permission.
+ *    
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS 
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT 
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS 
+ * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE 
+ * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, 
+ * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, 
+ * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; 
+ * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER 
+ * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT 
+ * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN 
+ * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 
+ * POSSIBILITY OF SUCH DAMAGE.
+ */
+package org.onelab.filter;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Defines the general behavior of a filter.
+ * <p>
+ * A filter is a data structure which aims at offering a lossy summary of a set <code>A</code>.  The
+ * key idea is to map entries of <code>A</code> (also called <i>keys</i>) into several positions 
+ * in a vector through the use of several hash functions.
+ * <p>
+ * Typically, a filter will be implemented as a Bloom filter (or a Bloom filter extension).
+ * <p>
+ * It must be extended in order to define the real behavior.
+ * 
+ * @see org.onelab.filter.Filter The general behavior of a filter
+ *
+ * @version 1.0 - 2 Feb. 07
+ * 
+ * @see org.onelab.filter.Key The general behavior of a key
+ * @see org.onelab.filter.HashFunction A hash function
+ */
+public abstract class Filter implements WritableComparable {
+  /** The vector size of <i>this</i> filter. */
+  int vectorSize;
+
+  /** The hash function used to map a key to several positions in the vector. */
+  protected HashFunction hash;
+
+  /** The number of hash function to consider. */
+  int nbHash;
+
+  protected Filter() {}
+  
+  /** 
+   * Constructor.
+   * @param vectorSize The vector size of <i>this</i> filter.
+   * @param nbHash The number of hash functions to consider.
+   */
+  protected Filter(int vectorSize, int nbHash){
+    this.vectorSize = vectorSize;
+    this.nbHash = nbHash;
+    this.hash = new HashFunction(this.vectorSize, this.nbHash);
+  }//end constructor
+
+  /**
+   * Adds a key to <i>this</i> filter.
+   * @param key The key to add.
+   */
+  public abstract void add(Key key);
+
+  /**
+   * Determines wether a specified key belongs to <i>this</i> filter.
+   * @param key The key to test.
+   * @return boolean True if the specified key belongs to <i>this</i> filter.
+   * 		     False otherwise.
+   */
+  public abstract boolean membershipTest(Key key);
+
+  /**
+   * Peforms a logical AND between <i>this</i> filter and a specified filter.
+   * <p>
+   * <b>Invariant</b>: The result is assigned to <i>this</i> filter.
+   * @param filter The filter to AND with.
+   */
+  public abstract void and(Filter filter);
+
+  /**
+   * Peforms a logical OR between <i>this</i> filter and a specified filter.
+   * <p>
+   * <b>Invariant</b>: The result is assigned to <i>this</i> filter.
+   * @param filter The filter to OR with.
+   */
+  public abstract void or(Filter filter);
+
+  /**
+   * Peforms a logical XOR between <i>this</i> filter and a specified filter.
+   * <p>
+   * <b>Invariant</b>: The result is assigned to <i>this</i> filter.
+   * @param filter The filter to XOR with.
+   */
+  public abstract void xor(Filter filter);
+
+  /**
+   * Performs a logical NOT on <i>this</i> filter.
+   * <p>
+   * The result is assigned to <i>this</i> filter.
+   */
+  public abstract void not();
+
+  /**
+   * Adds a list of keys to <i>this</i> filter.
+   * @param keys The list of keys.
+   */
+  public void add(ArrayList<Key> keys){
+    if(keys == null) {
+      throw new IllegalArgumentException("ArrayList<Key> may not be null");
+    }
+
+    for(Key key: keys) {
+      add(key);
+    }
+  }//end add()
+
+  /**
+   * Adds a collection of keys to <i>this</i> filter.
+   * @param keys The collection of keys.
+   */
+  public void add(Collection<Key> keys){
+    if(keys == null) {
+      throw new IllegalArgumentException("Collection<Key> may not be null");
+    }
+    for(Key key: keys) {
+      add(key);
+    }
+  }//end add()
+
+  /**
+   * Adds an array of keys to <i>this</i> filter.
+   * @param keys The array of keys.
+   */
+  public void add(Key[] keys){
+    if(keys == null) {
+      throw new IllegalArgumentException("Key[] may not be null");
+    }
+    for(int i = 0; i < keys.length; i++) {
+      add(keys[i]);
+    }
+  }//end add()
+  
+  @Override
+  public int hashCode() {
+    int result = Integer.valueOf(this.nbHash).hashCode();
+    result ^= Integer.valueOf(this.vectorSize);
+    return result;
+  }
+
+  // Writable interface
+  
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.io.Writable#write(java.io.DataOutput)
+   */
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(this.nbHash);
+    out.writeInt(this.vectorSize);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.io.Writable#readFields(java.io.DataInput)
+   */
+  public void readFields(DataInput in) throws IOException {
+    this.nbHash = in.readInt();
+    this.vectorSize = in.readInt();
+    this.hash = new HashFunction(this.vectorSize, this.nbHash);
+  }
+  
+  // Comparable interface
+  
+  /* (non-Javadoc)
+   * @see java.lang.Comparable#compareTo(java.lang.Object)
+   */
+  public int compareTo(Object o) {
+    Filter other = (Filter)o;
+    int result = this.vectorSize - other.vectorSize;
+    if(result == 0) {
+      result = this.nbHash - other.nbHash;
+    }
+    
+    return result;
+  }
+
+  
+}//end class

Added: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/HashFunction.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/HashFunction.java?view=auto&rev=547159
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/HashFunction.java (added)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/HashFunction.java Thu Jun 14 00:40:49 2007
@@ -0,0 +1,124 @@
+/**
+ *
+ * Copyright (c) 2005, European Commission project OneLab under contract 034819 
+ * (http://www.one-lab.org)
+ * 
+ * All rights reserved.
+ * Redistribution and use in source and binary forms, with or 
+ * without modification, are permitted provided that the following 
+ * conditions are met:
+ *  - Redistributions of source code must retain the above copyright 
+ *    notice, this list of conditions and the following disclaimer.
+ *  - Redistributions in binary form must reproduce the above copyright 
+ *    notice, this list of conditions and the following disclaimer in 
+ *    the documentation and/or other materials provided with the distribution.
+ *  - Neither the name of the University Catholique de Louvain - UCL
+ *    nor the names of its contributors may be used to endorse or 
+ *    promote products derived from this software without specific prior 
+ *    written permission.
+ *    
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS 
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT 
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS 
+ * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE 
+ * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, 
+ * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, 
+ * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; 
+ * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER 
+ * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT 
+ * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN 
+ * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 
+ * POSSIBILITY OF SUCH DAMAGE.
+ */
+package org.onelab.filter;
+
+import java.security.*;
+
+/**
+ * Implements a hash object that returns a certain number of hashed values.
+ * <p>
+ * It is based on the SHA-1 algorithm. 
+ * 
+ * @see org.onelab.filter.Filter The general behavior of a filter
+ *
+ * @version 1.0 - 2 Feb. 07
+ * 
+ * @see org.onelab.filter.Key The general behavior of a key being stored in a filter
+ * @see org.onelab.filter.Filter The general behavior of a filter
+ * 
+ * @see <a href="http://www.itl.nist.gov/fipspubs/fip180-1.htm">SHA-1 algorithm</a>
+ */
+public final class HashFunction{
+  /** The SHA-1 algorithm. */
+  private MessageDigest sha;
+
+  /** The number of hashed values. */
+  private int nbHash;
+
+  /** The maximum highest returned value. */
+  private int maxValue;
+
+  /**
+   * Constructor.
+   * <p>
+   * Builds a hash function that must obey to a given maximum number of returned values and a highest value.
+   * @param maxValue The maximum highest returned value.
+   * @param nbHash The number of resulting hashed values.
+   */
+  public HashFunction(int maxValue, int nbHash) {
+    try {
+      sha = MessageDigest.getInstance("SHA-1");
+      
+    } catch(NoSuchAlgorithmException e) {
+      throw new AssertionError(e);
+    }
+
+    if(maxValue <= 0) {
+      throw new IllegalArgumentException("maxValue must be > 0");
+    }
+    
+    if(nbHash <= 0) {
+      throw new IllegalArgumentException("nbHash must be > 0");
+    }
+
+    this.maxValue = maxValue;
+    this.nbHash = nbHash;
+  }//end constructor
+
+  /** Clears <i>this</i> hash function. */
+  public void clear(){
+    sha.reset();
+  }//end clear()
+
+  /**
+   * Hashes a specified key into several integers.
+   * @param k The specified key.
+   * @return The array of hashed values.
+   */
+  @SuppressWarnings("unchecked")
+  public int[] hash(Key k){
+      byte[] b = k.getBytes();
+      if(b == null) {
+        throw new NullPointerException("buffer reference is null");
+      }
+      if(b.length == 0) {
+        throw new IllegalArgumentException("key length must be > 0");
+      }
+      sha.update(b);
+      byte[] digestBytes = sha.digest();
+      int[] result = new int[nbHash];
+      int nbBytePerInt = digestBytes.length/nbHash;
+      int offset = 0;
+      for(int i = 0; i < nbHash; i++){
+        int val = 0;
+        for(int j = offset; j < offset + nbBytePerInt; j++) {
+          val |=
+            (digestBytes[offset] & 0xff) << ((nbBytePerInt - 1 - (j - offset)) * 8);
+        }
+        result[i] = Math.abs(val) % maxValue;
+        offset += nbBytePerInt;
+      }
+      return result;
+  }//end hash() 
+
+}//end class

Added: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/Key.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/Key.java?view=auto&rev=547159
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/Key.java (added)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/Key.java Thu Jun 14 00:40:49 2007
@@ -0,0 +1,160 @@
+/**
+ *
+ * Copyright (c) 2005, European Commission project OneLab under contract 034819 (http://www.one-lab.org)
+ * All rights reserved.
+ * Redistribution and use in source and binary forms, with or 
+ * without modification, are permitted provided that the following 
+ * conditions are met:
+ *  - Redistributions of source code must retain the above copyright 
+ *    notice, this list of conditions and the following disclaimer.
+ *  - Redistributions in binary form must reproduce the above copyright 
+ *    notice, this list of conditions and the following disclaimer in 
+ *    the documentation and/or other materials provided with the distribution.
+ *  - Neither the name of the University Catholique de Louvain - UCL
+ *    nor the names of its contributors may be used to endorse or 
+ *    promote products derived from this software without specific prior 
+ *    written permission.
+ *    
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS 
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT 
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS 
+ * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE 
+ * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, 
+ * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, 
+ * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; 
+ * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER 
+ * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT 
+ * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN 
+ * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 
+ * POSSIBILITY OF SUCH DAMAGE.
+ */
+package org.onelab.filter;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * The general behavior of a key that must be stored in a filter.
+ * 
+ * @see org.onelab.filter.Filter The general behavior of a filter
+ */
+public class Key implements WritableComparable {
+  /** Byte value of key */
+  byte[] bytes;
+  
+  /**
+   * The weight associated to <i>this</i> key.
+   * <p>
+   * <b>Invariant</b>: if it is not specified, each instance of 
+   * <code>Key</code> will have a default weight of 1.0
+   */
+  double weight;
+
+  /** default constructor - use with readFields */
+  public Key() {}
+
+  /**
+   * Constructor.
+   * <p>
+   * Builds a key with a default weight.
+   * @param value The byte value of <i>this</i> key.
+   */
+  public Key(byte[] value) {
+    this(value, 1.0);
+  }//end constructor
+
+  /**
+   * Constructor.
+   * <p>
+   * Builds a key with a specified weight.
+   * @param value The value of <i>this</i> key.
+   * @param weight The weight associated to <i>this</i> key.
+   */
+  public Key(byte[] value, double weight) {
+    if(value == null) {
+      throw new IllegalArgumentException("value can not be null");
+    }
+    this.bytes = value;
+    this.weight = weight;
+  }//end constructor
+
+  /** @return byte[] The value of <i>this</i> key. */
+  public byte[] getBytes() {
+    return this.bytes;
+  }
+
+  /** @return Returns the weight associated to <i>this</i> key. */
+  public double getWeight(){
+    return weight;
+  }//end getWeight()
+
+  /**
+   * Increments the weight of <i>this</i> key with a specified value. 
+   * @param weight The increment.
+   */
+  public void incrementWeight(double weight){
+    this.weight += weight;
+  }//end incrementWeight()
+
+  /** Increments the weight of <i>this</i> key by one. */
+  public void incrementWeight(){
+    this.weight++;
+  }//end incrementWeight()
+
+  @Override
+  public boolean equals(Object o) {
+    return this.compareTo(o) == 0;
+  }
+  
+  @Override
+  public int hashCode() {
+    int result = 0;
+    for(int i = 0; i < bytes.length; i++) {
+      result ^= Byte.valueOf(bytes[i]).hashCode();
+    }
+    result ^= Double.valueOf(weight).hashCode();
+    return result;
+  }
+
+  // Writable
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.io.Writable#write(java.io.DataOutput)
+   */
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(bytes.length);
+    out.write(bytes);
+    out.writeDouble(weight);
+  }
+  
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.io.Writable#readFields(java.io.DataInput)
+   */
+  public void readFields(DataInput in) throws IOException {
+    this.bytes = new byte[in.readInt()];
+    in.readFully(this.bytes);
+    weight = in.readDouble();
+  }
+  
+  // Comparable
+  
+  /* (non-Javadoc)
+   * @see java.lang.Comparable#compareTo(java.lang.Object)
+   */
+  public int compareTo(Object o) {
+    Key other = (Key)o;
+
+    int result = this.bytes.length - other.getBytes().length;
+    for(int i = 0; result == 0 && i < bytes.length; i++) {
+      result = this.bytes[i] - other.bytes[i];
+    }
+    
+    if(result == 0) {
+      result = Double.valueOf(this.weight - other.weight).intValue();
+    }
+    return result;
+  }
+}//end class

Added: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/RemoveScheme.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/RemoveScheme.java?view=auto&rev=547159
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/RemoveScheme.java (added)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/RemoveScheme.java Thu Jun 14 00:40:49 2007
@@ -0,0 +1,75 @@
+/**
+ *
+ * Copyright (c) 2005, European Commission project OneLab under contract 034819
+ * (http://www.one-lab.org)
+ * 
+ * All rights reserved.
+ * Redistribution and use in source and binary forms, with or 
+ * without modification, are permitted provided that the following 
+ * conditions are met:
+ *  - Redistributions of source code must retain the above copyright 
+ *    notice, this list of conditions and the following disclaimer.
+ *  - Redistributions in binary form must reproduce the above copyright 
+ *    notice, this list of conditions and the following disclaimer in 
+ *    the documentation and/or other materials provided with the distribution.
+ *  - Neither the name of the University Catholique de Louvain - UCL
+ *    nor the names of its contributors may be used to endorse or 
+ *    promote products derived from this software without specific prior 
+ *    written permission.
+ *    
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS 
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT 
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS 
+ * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE 
+ * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, 
+ * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, 
+ * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; 
+ * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER 
+ * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT 
+ * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN 
+ * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 
+ * POSSIBILITY OF SUCH DAMAGE.
+ */
+package org.onelab.filter;
+
+/**
+ * Defines the different remove scheme for retouched Bloom filters.
+ * 
+ * @author <a href="mailto:donnet@ucl.ac.be">Benoit Donnet</a> - Universite Catholique de Louvain - Faculte des Sciences Appliquees - Departement d'Ingenierie Informatique.
+ * contract <a href="http://www.one-lab.org">European Commission One-Lab Project 034819</a>.
+ *
+ * @version 1.0 - 7 Feb. 07
+ */
+public interface RemoveScheme {
+  /**
+   * Random selection.
+   * <p>
+   * The idea is to randomly select a bit to reset.
+   */
+  public final static short RANDOM = 0;
+
+  /**
+   * MinimumFN Selection.
+   * <p>
+   * The idea is to select the bit to reset that will generate the minimum
+   * number of false negative.
+   */
+  public final static short MINIMUM_FN = 1;
+
+  /**
+   * MaximumFP Selection.
+   * <p>
+   * The idea is to select the bit to reset that will remove the maximum number
+   * of false positive.
+   */
+  public final static short MAXIMUM_FP = 2;
+
+  /**
+   * Ratio Selection.
+   * <p>
+   * The idea is to select the bit to reset that will, at the same time, remove
+   * the maximum number of false positve while minimizing the amount of false
+   * negative generated.
+   */
+  public final static short RATIO = 3;
+}//end interface

Added: 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=auto&rev=547159
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/RetouchedBloomFilter.java (added)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/onelab/filter/RetouchedBloomFilter.java Thu Jun 14 00:40:49 2007
@@ -0,0 +1,479 @@
+/**
+ *
+ * Copyright (c) 2005, European Commission project OneLab under contract 034819 (http://www.one-lab.org)
+ * All rights reserved.
+ * Redistribution and use in source and binary forms, with or 
+ * without modification, are permitted provided that the following 
+ * conditions are met:
+ *  - Redistributions of source code must retain the above copyright 
+ *    notice, this list of conditions and the following disclaimer.
+ *  - Redistributions in binary form must reproduce the above copyright 
+ *    notice, this list of conditions and the following disclaimer in 
+ *    the documentation and/or other materials provided with the distribution.
+ *  - Neither the name of the University Catholique de Louvain - UCL
+ *    nor the names of its contributors may be used to endorse or 
+ *    promote products derived from this software without specific prior 
+ *    written permission.
+ *    
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS 
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT 
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS 
+ * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE 
+ * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, 
+ * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, 
+ * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; 
+ * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER 
+ * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT 
+ * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN 
+ * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 
+ * POSSIBILITY OF SUCH DAMAGE.
+ */
+package org.onelab.filter;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Random;
+
+/**
+ * Implements a <i>retouched Bloom filter</i>, as defined in the CoNEXT 2006 paper.
+ * <p>
+ * It allows the removal of selected false positives at the cost of introducing
+ * random false negatives, and with the benefit of eliminating some random false
+ * positives at the same time.
+ * 
+ * @author <a href="mailto:donnet@ucl.ac.be">Benoit Donnet</a> - Universite Catholique de Louvain - Faculte des Sciences Appliquees - Departement d'Ingenierie Informatique.
+ * contract <a href="http://www.one-lab.org">European Commission One-Lab Project 034819</a>.
+ *
+ * @version 1.0 - 7 Feb. 07
+ * 
+ * @see org.onelab.filter.Filter The general behavior of a filter
+ * @see org.onelab.filter.BloomFilter A Bloom filter
+ * @see org.onelab.filter.RemoveScheme The different selective clearing algorithms
+ * 
+ * @see <a href="http://www-rp.lip6.fr/site_npa/site_rp/_publications/740-rbf_cameraready.pdf">Retouched Bloom Filters: Allowing Networked Applications to Trade Off Selected False Positives Against False Negatives</a>
+ */
+public final class RetouchedBloomFilter extends BloomFilter
+implements RemoveScheme {
+  /**
+   * KeyList vector (or ElementList Vector, as defined in the paper) of false positives.
+   */
+  ArrayList<Key>[] fpVector;
+
+  /**
+   * KeyList vector of keys recorded in the filter.
+   */
+  ArrayList<Key>[] keyVector;
+
+  /**
+   * Ratio vector.
+   */
+  double[] ratio;
+  
+  private Random rand;
+
+  /** Default constructor - use with readFields */
+  public RetouchedBloomFilter() {}
+  
+  /**
+   * Constructor
+   * @param vectorSize The vector size of <i>this</i> filter.
+   * @param nbHash The number of hash function to consider.
+   */
+  public RetouchedBloomFilter(int vectorSize, int nbHash) {
+    super(vectorSize, nbHash);
+
+    this.rand = null;
+    createVector();
+  }//end constructor
+
+  @Override
+  public void add(Key key){
+    if(key == null) {
+      throw new NullPointerException("key can not be null");
+    }
+
+    int[] h = hash.hash(key);
+
+    for(int i = 0; i < nbHash; i++) {
+      vector[h[i]] = true;
+      keyVector[h[i]].add(key);
+    }//end for - i
+  }//end add()
+
+  /**
+   * Adds a false positive information to <i>this</i> retouched Bloom filter.
+   * <p>
+   * <b>Invariant</b>: if the false positive is <code>null</code>, nothing happens.
+   * @param key The false positive key to add.
+   */
+  public void addFalsePositive(Key key){
+    if(key == null) {
+      throw new NullPointerException("key can not be null");
+    }
+
+    int[] h = hash.hash(key);
+
+    for(int i = 0; i < nbHash; i++) {
+      fpVector[h[i]].add(key);
+    }
+  }//end addFalsePositive()
+
+  /**
+   * Adds a collection of false positive information to <i>this</i> retouched Bloom filter.
+   * @param coll The collection of false positive.
+   */
+  public void addFalsePositive(Collection<Key> coll) {
+    if(coll == null) {
+      throw new NullPointerException("Collection<Key> can not be null");
+    }
+    
+    for(Key k: coll) {
+      addFalsePositive(k);
+    }
+  }//end addFalsePositive()
+
+  /**
+   * Adds a list of false positive information to <i>this</i> retouched Bloom filter.
+   * @param keys The list of false positive.
+   */
+  public void addFalsePositive(ArrayList<Key> keys){
+    if(keys == null) {
+      throw new NullPointerException("ArrayList<Key> can not be null");
+    }
+
+    for(Key k: keys) {
+      addFalsePositive(k);
+    }
+  }//end addFalsePositive()
+
+  /**
+   * Adds an array of false positive information to <i>this</i> retouched Bloom filter.
+   * @param keys The array of false positive.
+   */
+  public void addFalsePositive(Key[] keys){
+    if(keys == null) {
+      throw new NullPointerException("Key[] can not be null");
+    }
+
+    for(int i = 0; i < keys.length; i++) {
+      addFalsePositive(keys[i]);
+    }
+  }//end addFalsePositive()
+
+  /**
+   * Performs the selective clearing for a given key.
+   * @param k The false positive key to remove from <i>this</i> retouched Bloom filter.
+   * @param scheme The selective clearing scheme to apply.
+   */
+  public void selectiveClearing(Key k, short scheme) {
+    if(k == null) {
+      throw new NullPointerException("Key can not be null");
+    }
+
+    if(!membershipTest(k)) {
+      throw new IllegalArgumentException("Key is not a member");
+    }
+
+    int index = 0;
+    int[] h = hash.hash(k);
+
+    switch(scheme) {
+
+    case RANDOM:
+      index = randomRemove();
+      break;
+    
+    case MINIMUM_FN:
+      index = minimumFnRemove(h);
+      break;
+    
+    case MAXIMUM_FP:
+      index = maximumFpRemove(h);
+      break;
+    
+    case RATIO:
+      index = ratioRemove(h);
+      break;
+    
+    default:
+      throw new AssertionError("Undefined selective clearing scheme");
+
+    }//end switch
+
+    clearBit(index);
+  }//end selectiveClearing()
+
+  private int randomRemove() {
+    if(rand == null) {
+      rand = new Random();
+    }
+
+    return rand.nextInt(nbHash);
+  }//end randomRemove()
+
+  /**
+   * Chooses the bit position that minimizes the number of false negative generated.
+   * @param h The different bit positions.
+   * @return int The position that minimizes the number of false negative generated.
+   */
+  private int minimumFnRemove(int[] h) {
+    int minIndex = Integer.MAX_VALUE;
+    double minValue = Double.MAX_VALUE;
+
+    for(int i = 0; i < nbHash; i++) {
+      double keyWeight = getWeight(keyVector[h[i]]);
+
+      if(keyWeight < minValue) {
+        minIndex = h[i];
+        minValue = keyWeight;
+      }
+
+    }//end for - i
+
+    return minIndex;
+  }//end minimumFnRemove()
+
+  /**
+   * Chooses the bit position that maximizes the number of false positive removed.
+   * @param h The different bit positions.
+   * @return int The position that maximizes the number of false positive removed.
+   */
+  private int maximumFpRemove(int[] h){
+    int maxIndex = Integer.MIN_VALUE;
+    double maxValue = Double.MIN_VALUE;
+
+    for(int i = 0; i < nbHash; i++) {
+      double fpWeight = getWeight(fpVector[h[i]]);
+
+      if(fpWeight > maxValue) {
+        maxValue = fpWeight;
+        maxIndex = h[i];
+      }
+    }
+
+    return maxIndex;
+  }//end maximumFpRemove()
+
+  /**
+   * Chooses the bit position that minimizes the number of false negative generated while maximizing.
+   * the number of false positive removed.
+   * @param h The different bit positions.
+   * @return int The position that minimizes the number of false negative generated while maximizing.
+   */
+  private int ratioRemove(int[] h){
+    computeRatio();
+    int minIndex = Integer.MAX_VALUE;
+    double minValue = Double.MAX_VALUE;
+
+    for(int i = 0; i < nbHash; i++) {
+      if(ratio[h[i]] < minValue) {
+        minValue = ratio[h[i]];
+        minIndex = h[i];
+      }
+    }//end for - i
+
+    return minIndex;
+  }//end ratioRemove()
+
+  /**
+   * Clears a specified bit in the bit vector and keeps up-to-date the KeyList vectors.
+   * @param index The position of the bit to clear.
+   */
+  private void clearBit(int index){
+    if(index < 0 || index >= vectorSize) {
+      throw new ArrayIndexOutOfBoundsException(index);
+    }
+
+    ArrayList<Key> kl = keyVector[index];
+    ArrayList<Key> fpl = fpVector[index];
+
+    // update key list
+    int listSize = kl.size();
+    for(int i = 0; i < listSize && !kl.isEmpty(); i++) {
+      removeKey(kl.get(0), keyVector);
+    }
+
+    kl.clear();
+    keyVector[index].clear();
+
+    //update false positive list
+    listSize = fpl.size();
+    for(int i = 0; i < listSize && !fpl.isEmpty(); i++) {
+      removeKey(fpl.get(0), fpVector);
+    }
+
+    fpl.clear();
+    fpVector[index].clear();
+
+    //update ratio
+    ratio[index] = 0.0;
+
+    //update bit vector
+    vector[index] = false;
+  }//end clearBit()
+
+  /**
+   * Removes a given key from <i>this</i> filer.
+   * @param k The key to remove.
+   * @param vector The counting vector associated to the key.
+   */
+  private void removeKey(Key k, ArrayList<Key>[] vector) {
+    if(k == null) {
+      throw new NullPointerException("Key can not be null");
+    }
+    if(vector == null) {
+      throw new NullPointerException("ArrayList<Key>[] can not be null");
+    }
+
+    int[] h = hash.hash(k);
+
+    for(int i = 0; i < nbHash; i++) {
+      vector[h[i]].remove(k);
+    }
+  }//end removeKey()
+
+  /**
+   * Computes the ratio A/FP.
+   */
+  private void computeRatio() {
+    for(int i = 0; i < vectorSize; i++) {
+      double keyWeight = getWeight(keyVector[i]);
+      double fpWeight = getWeight(fpVector[i]);
+
+      if(keyWeight > 0 && fpWeight > 0) {
+        ratio[i] = keyWeight/fpWeight;
+      }
+    }//end for - i
+  }//end computeRatio()
+
+  private double getWeight(ArrayList<Key> keyList) {
+    double weight = 0.0;
+    for(Key k: keyList) {
+      weight += k.getWeight();
+    }
+    return weight;
+  }
+  
+  /**
+   * Creates and initialises the various vectors.
+   */
+  @SuppressWarnings("unchecked")
+  private void createVector() {
+    fpVector = new ArrayList[vectorSize];
+    keyVector = new ArrayList[vectorSize];
+    ratio = new double[vectorSize];
+
+    for(int i = 0; i < vectorSize; i++) {
+      fpVector[i] = new ArrayList<Key>();
+      keyVector[i] = new ArrayList<Key>();
+      ratio[i] = 0.0;
+    }//end for -i
+  }//end createVector()
+  
+  @Override
+  public boolean equals(Object o) {
+    return this.compareTo(o) == 0;
+  }
+  
+  @Override
+  public int hashCode() {
+    int result = super.hashCode();
+    for(int i = 0; i < fpVector.length; i++) {
+      result ^= fpVector[i].hashCode();
+    }
+    for(int i = 0; i < keyVector.length; i++) {
+      result ^= keyVector[i].hashCode();
+    }
+    for(int i = 0; i < ratio.length; i++) {
+      result ^= Double.valueOf(ratio[i]).hashCode();
+    }
+    return result;
+  }
+
+  // Writable
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    super.write(out);
+    for(int i = 0; i < fpVector.length; i++) {
+      ArrayList<Key> list = fpVector[i];
+      out.writeInt(list.size());
+      for(Key k: list) {
+        k.write(out);
+      }
+    }
+    for(int i = 0; i < keyVector.length; i++) {
+      ArrayList<Key> list = keyVector[i];
+      out.writeInt(list.size());
+      for(Key k: list) {
+        k.write(out);
+      }
+    }
+    for(int i = 0; i < ratio.length; i++) {
+      out.writeDouble(ratio[i]);
+    }
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    super.readFields(in);
+    createVector();
+    for(int i = 0; i < fpVector.length; i++) {
+      ArrayList<Key> list = fpVector[i];
+      int size = in.readInt();
+      for(int j = 0; j < size; j++) {
+        Key k = new Key();
+        k.readFields(in);
+        list.add(k);
+      }
+    }
+    for(int i = 0; i < keyVector.length; i++) {
+      ArrayList<Key> list = keyVector[i];
+      int size = in.readInt();
+      for(int j = 0; j < size; j++) {
+        Key k = new Key();
+        k.readFields(in);
+        list.add(k);
+      }
+    }
+    for(int i = 0; i < ratio.length; i++) {
+      ratio[i] = in.readDouble();
+    }
+  }
+
+  // Comparable
+  
+  @Override
+  public int compareTo(Object o) {
+    int result = super.compareTo(o);
+    
+    RetouchedBloomFilter other = (RetouchedBloomFilter)o;
+      
+    for(int i = 0; result == 0 && i < fpVector.length; i++) {
+      ArrayList<Key> mylist = fpVector[i];
+      ArrayList<Key> otherlist = other.fpVector[i];
+        
+      for(int j = 0; result == 0 && j < mylist.size(); j++) {
+        result = mylist.get(j).compareTo(otherlist.get(j));
+      }
+    }
+
+    for(int i = 0; result == 0 && i < keyVector.length; i++) {
+      ArrayList<Key> mylist = keyVector[i];
+      ArrayList<Key> otherlist = other.keyVector[i];
+        
+      for(int j = 0; result == 0 && j < mylist.size(); j++) {
+        result = mylist.get(j).compareTo(otherlist.get(j));
+      }
+    }
+
+    for(int i = 0; result == 0 && i < ratio.length; i++) {
+      result = Double.valueOf(this.ratio[i] - other.ratio[i]).intValue();
+    }
+    
+    return result;
+  }// end compareTo
+}//end class

Added: lucene/hadoop/trunk/src/contrib/hbase/src/test/org/onelab/test/StringKey.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/onelab/test/StringKey.java?view=auto&rev=547159
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/test/org/onelab/test/StringKey.java (added)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/test/org/onelab/test/StringKey.java Thu Jun 14 00:40:49 2007
@@ -0,0 +1,72 @@
+/**
+ * Copyright (c) 2005, European Commission project OneLab under contract 034819 
+ * (http://www.one-lab.org)
+ * 
+ * All rights reserved.
+ * Redistribution and use in source and binary forms, with or 
+ * without modification, are permitted provided that the following 
+ * conditions are met:
+ *  - Redistributions of source code must retain the above copyright 
+ *    notice, this list of conditions and the following disclaimer.
+ *  - Redistributions in binary form must reproduce the above copyright 
+ *    notice, this list of conditions and the following disclaimer in 
+ *    the documentation and/or other materials provided with the distribution.
+ *  - Neither the name of the University Catholique de Louvain - UCL
+ *    nor the names of its contributors may be used to endorse or 
+ *    promote products derived from this software without specific prior 
+ *    written permission.
+ *    
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS 
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT 
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS 
+ * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE 
+ * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, 
+ * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, 
+ * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; 
+ * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER 
+ * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT 
+ * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN 
+ * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 
+ * POSSIBILITY OF SUCH DAMAGE.
+ */
+package org.onelab.test;
+
+import org.onelab.filter.Key;
+
+/**
+ * Test class for keys.
+ * <p>
+ * It gives an example on how to extend Key.
+ * 
+ * @author <a href="mailto:donnet@ucl.ac.be">Benoit Donnet</a> - Universite Catholique de Louvain - Faculte des Sciences Appliquees - Departement d'Ingenierie Informatique.
+ * contract <a href="http://www.one-lab.org">European Commission One-Lab Project 034819</a>.
+ *
+ * @version 1.0 - 5 Feb. 07
+ * 
+ * @see org.onelab.filter.Key A key stored in a filter
+ */
+public class StringKey extends Key {
+  
+  /** Default constructor - use with readFields */
+  public StringKey() {}
+
+  /**
+   * Construct a Key using the specified String and default weight
+   * 
+   * @param key String key value
+   */
+  public StringKey(String key){
+    super(key.getBytes());
+  }
+
+  /**
+   * Construct a Key using the specified string and weight
+   * 
+   * @param key - String key value
+   * @param weight key weight
+   */
+  public StringKey(String key, double weight){
+    super(key.getBytes(), weight);
+  }
+
+}

Added: lucene/hadoop/trunk/src/contrib/hbase/src/test/org/onelab/test/TestFilter.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/onelab/test/TestFilter.java?view=auto&rev=547159
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/test/org/onelab/test/TestFilter.java (added)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/test/org/onelab/test/TestFilter.java Thu Jun 14 00:40:49 2007
@@ -0,0 +1,92 @@
+/**
+ * Copyright (c) 2005, European Commission project OneLab under contract 034819
+ * (http://www.one-lab.org)
+ * 
+ * All rights reserved.
+ * Redistribution and use in source and binary forms, with or 
+ * without modification, are permitted provided that the following 
+ * conditions are met:
+ *  - Redistributions of source code must retain the above copyright 
+ *    notice, this list of conditions and the following disclaimer.
+ *  - Redistributions in binary form must reproduce the above copyright 
+ *    notice, this list of conditions and the following disclaimer in 
+ *    the documentation and/or other materials provided with the distribution.
+ *  - Neither the name of the University Catholique de Louvain - UCL
+ *    nor the names of its contributors may be used to endorse or 
+ *    promote products derived from this software without specific prior 
+ *    written permission.
+ *    
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS 
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT 
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS 
+ * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE 
+ * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, 
+ * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, 
+ * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; 
+ * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER 
+ * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT 
+ * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN 
+ * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 
+ * POSSIBILITY OF SUCH DAMAGE.
+ */
+package org.onelab.test;
+
+import junit.framework.TestCase;
+
+import org.onelab.filter.*;
+
+/**
+ * Test class.
+ * 
+ * @author <a href="mailto:donnet@ucl.ac.be">Benoit Donnet</a> - Universite Catholique de Louvain - Faculte des Sciences Appliquees - Departement d'Ingenierie Informatique.
+ * contract <a href="http://www.one-lab.org">European Commission One-Lab Project 034819</a>.
+ *
+ * @version 1.0 - 8 Feb. 07
+ */
+public class TestFilter extends TestCase {
+  
+  /** Test a BloomFilter */
+  public void testBloomFilter() {
+    Filter bf = new BloomFilter(8, 2);
+    Key key = new StringKey("toto");
+    Key k2 = new StringKey("lulu");
+    Key k3 = new StringKey("mama");
+    bf.add(key);
+    bf.add(k2);
+    bf.add(k3);
+    assertTrue(bf.membershipTest(key));
+    assertFalse(bf.membershipTest(new StringKey("graknyl")));
+    assertTrue(bf.membershipTest(new StringKey("xyzzy")));      // False positive
+    assertTrue(bf.membershipTest(new StringKey("abcd")));       // False positive
+  }
+  
+  /** Test a CountingBloomFilter */
+  public void testCountingBloomFilter() {
+    Filter bf = new CountingBloomFilter(8, 2);
+    Key key = new StringKey("toto");
+    Key k2 = new StringKey("lulu");
+    Key k3 = new StringKey("mama");
+    bf.add(key);
+    bf.add(k2);
+    bf.add(k3);
+    assertTrue(bf.membershipTest(key));
+    assertFalse(bf.membershipTest(new StringKey("graknyl")));
+    assertTrue(bf.membershipTest(new StringKey("xyzzy")));      // False positive
+    assertTrue(bf.membershipTest(new StringKey("abcd")));       // False positive
+  }
+  
+  /** Test a DynamicBloomFilter */
+  public void testDynamicBloomFilter() {
+    Filter bf = new DynamicBloomFilter(8, 2, 2);
+    Key key = new StringKey("toto");
+    Key k2 = new StringKey("lulu");
+    Key k3 = new StringKey("mama");
+    bf.add(key);
+    bf.add(k2);
+    bf.add(k3);
+    assertTrue(bf.membershipTest(key));
+    assertFalse(bf.membershipTest(new StringKey("graknyl")));
+    assertFalse(bf.membershipTest(new StringKey("xyzzy")));
+    assertTrue(bf.membershipTest(new StringKey("abcd")));       // False positive
+  }
+}//end class