Index: src/contrib/hbase/NOTICE.txt =================================================================== --- src/contrib/hbase/NOTICE.txt (revision 0) +++ src/contrib/hbase/NOTICE.txt (revision 0) @@ -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) Index: src/contrib/hbase/src/java/org/onelab/filter/BloomFilter.java =================================================================== --- src/contrib/hbase/src/java/org/onelab/filter/BloomFilter.java (revision 0) +++ src/contrib/hbase/src/java/org/onelab/filter/BloomFilter.java (revision 0) @@ -0,0 +1,227 @@ +/** + * Created on 2 f�vr. 07 + * + * 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 Universit� 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 Bloom filter, as defined by Bloom in 1970. + *
+ * 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 Benoit Donnet - Universite Catholique de Louvain - Faculte des Sciences Appliquees - Departement d'Ingenierie Informatique. + * contract European Commission One-Lab Project 034819. + * + * @version 1.0 - 2 Feb. 07 + * + * @see org.onelab.filter.Filter The general behavior of a filter + * + * @see Space/Time Trade-Offs in Hash Coding with Allowable Errors + */ +public class BloomFilter extends Filter { + /** The bit vector. */ + boolean[] vector; + + /** Default constructor - use with readFields */ + public BloomFilter() {} + + /** + * Constructor + * @param vectorSize The vector size of this 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 this 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 this 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 Index: src/contrib/hbase/src/java/org/onelab/filter/CountingBloomFilter.java =================================================================== --- src/contrib/hbase/src/java/org/onelab/filter/CountingBloomFilter.java (revision 0) +++ src/contrib/hbase/src/java/org/onelab/filter/CountingBloomFilter.java (revision 0) @@ -0,0 +1,242 @@ +/** + * Created on 5 f�vr. 07 + * + * 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 Universit� 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 counting Bloom filter, as defined by Fan et al. in a ToN + * 2000 paper. + *
+ * 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 Benoit Donnet - Universite Catholique de Louvain - Faculte des Sciences Appliquees - Departement d'Ingenierie Informatique. + * contract European Commission One-Lab Project 034819. + * + * @version 1.0 - 5 Feb. 07 + * + * @see org.onelab.filter.Filter The general behavior of a filter + * + * @see Summary cache: a scalable wide-area web cache sharing protocol + */ +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 this 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 this counting Bloom filter. + *
+ * Invariant: nothing happens if the specified key does not belong to this 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 Index: src/contrib/hbase/src/java/org/onelab/filter/DynamicBloomFilter.java =================================================================== --- src/contrib/hbase/src/java/org/onelab/filter/DynamicBloomFilter.java (revision 0) +++ src/contrib/hbase/src/java/org/onelab/filter/DynamicBloomFilter.java (revision 0) @@ -0,0 +1,315 @@ +/** + * Created on 6 f�vr. 07 + * + * 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 Universit� 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 dynamic Bloom filter, as defined in the INFOCOM 2006 paper. + *
+ * A dynamic Bloom filter (DBF) makes use of a s * m bit matrix but
+ * each of the s rows is a standard Bloom filter. The creation
+ * process of a DBF is iterative. At the start, the DBF is a 1 * m
+ * bit matrix, i.e., it is composed of a single standard Bloom filter.
+ * It assumes that nr elements are recorded in the
+ * initial bit vector, where nr <= n (n is
+ * the cardinality of the set A to record in the filter).
+ *
+ * As the size of A 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, nr, is
+ * strictly less than the current cardinality of A, n.
+ * If an active Bloom filter is found, the key is inserted and
+ * nr 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 A and the element
+ * is added in this new Bloom filter and the nr value of
+ * this new Bloom filter is set to one. A given key is said to belong to the
+ * DBF if the k positions are set to one in one of the matrix rows.
+ *
+ * @author Benoit Donnet - Universite Catholique de Louvain - Faculte des Sciences Appliquees - Departement d'Ingenierie Informatique.
+ * contract European Commission One-Lab Project 034819.
+ *
+ * @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 Theory and Network Applications of Dynamic Bloom Filters
+ */
+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.
+ *
+ * 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
+ * A filter is a data structure which aims at offering a lossy summary of a set
+ * Typically, a filter will be implemented as a Bloom filter (or a Bloom filter extension).
+ *
+ * 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 this 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 this 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 this filter.
+ * @param key The key to add.
+ */
+ public abstract void add(Key key);
+
+ /**
+ * Determines wether a specified key belongs to this filter.
+ * @param key The key to test.
+ * @return boolean True if the specified key belongs to this filter.
+ * False otherwise.
+ */
+ public abstract boolean membershipTest(Key key);
+
+ /**
+ * Peforms a logical AND between this filter and a specified filter.
+ *
+ * Invariant: The result is assigned to this filter.
+ * @param filter The filter to AND with.
+ */
+ public abstract void and(Filter filter);
+
+ /**
+ * Peforms a logical OR between this filter and a specified filter.
+ *
+ * Invariant: The result is assigned to this filter.
+ * @param filter The filter to OR with.
+ */
+ public abstract void or(Filter filter);
+
+ /**
+ * Peforms a logical XOR between this filter and a specified filter.
+ *
+ * Invariant: The result is assigned to this filter.
+ * @param filter The filter to XOR with.
+ */
+ public abstract void xor(Filter filter);
+
+ /**
+ * Performs a logical NOT on this filter.
+ *
+ * The result is assigned to this filter.
+ */
+ public abstract void not();
+
+ /**
+ * Adds a list of keys to this filter.
+ * @param keys The list of keys.
+ */
+ public void add(ArrayList
+ * 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 SHA-1 algorithm
+ */
+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.
+ *
+ * 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 this 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
Index: src/contrib/hbase/src/java/org/onelab/filter/Key.java
===================================================================
--- src/contrib/hbase/src/java/org/onelab/filter/Key.java (revision 0)
+++ src/contrib/hbase/src/java/org/onelab/filter/Key.java (revision 0)
@@ -0,0 +1,161 @@
+/**
+ * Created on 2 f�vr. 07
+ *
+ * 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 Universit� 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 this key.
+ *
+ * Invariant: if it is not specified, each instance of
+ *
+ * Builds a key with a default weight.
+ * @param value The byte value of this key.
+ */
+ public Key(byte[] value) {
+ this(value, 1.0);
+ }//end constructor
+
+ /**
+ * Constructor.
+ *
+ * Builds a key with a specified weight.
+ * @param value The value of this key.
+ * @param weight The weight associated to this 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 this key. */
+ public byte[] getBytes() {
+ return this.bytes;
+ }
+
+ /** @return Returns the weight associated to this key. */
+ public double getWeight(){
+ return weight;
+ }//end getWeight()
+
+ /**
+ * Increments the weight of this key with a specified value.
+ * @param weight The increment.
+ */
+ public void incrementWeight(double weight){
+ this.weight += weight;
+ }//end incrementWeight()
+
+ /** Increments the weight of this 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
Index: src/contrib/hbase/src/java/org/onelab/filter/RemoveScheme.java
===================================================================
--- src/contrib/hbase/src/java/org/onelab/filter/RemoveScheme.java (revision 0)
+++ src/contrib/hbase/src/java/org/onelab/filter/RemoveScheme.java (revision 0)
@@ -0,0 +1,76 @@
+/**
+ * Created on 7 f�vr. 07
+ *
+ * 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 Universit� 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 Benoit Donnet - Universite Catholique de Louvain - Faculte des Sciences Appliquees - Departement d'Ingenierie Informatique.
+ * contract European Commission One-Lab Project 034819.
+ *
+ * @version 1.0 - 7 Feb. 07
+ */
+public interface RemoveScheme {
+ /**
+ * Random selection.
+ *
+ * The idea is to randomly select a bit to reset.
+ */
+ public final static short RANDOM = 0;
+
+ /**
+ * MinimumFN Selection.
+ *
+ * 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.
+ *
+ * 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.
+ *
+ * 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
Index: src/contrib/hbase/src/java/org/onelab/filter/RetouchedBloomFilter.java
===================================================================
--- src/contrib/hbase/src/java/org/onelab/filter/RetouchedBloomFilter.java (revision 0)
+++ src/contrib/hbase/src/java/org/onelab/filter/RetouchedBloomFilter.java (revision 0)
@@ -0,0 +1,480 @@
+/**
+ * Created on 7 f�vr. 07
+ *
+ * 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 Universit� 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 retouched Bloom filter, as defined in the CoNEXT 2006 paper.
+ *
+ * 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 Benoit Donnet - Universite Catholique de Louvain - Faculte des Sciences Appliquees - Departement d'Ingenierie Informatique.
+ * contract European Commission One-Lab Project 034819.
+ *
+ * @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 Retouched Bloom Filters: Allowing Networked Applications to Trade Off Selected False Positives Against False Negatives
+ */
+public final class RetouchedBloomFilter extends BloomFilter
+implements RemoveScheme {
+ /**
+ * KeyList vector (or ElementList Vector, as defined in the paper) of false positives.
+ */
+ ArrayList
+ * Invariant: if the false positive is
+ * It gives an example on how to extend Key.
+ *
+ * @author Benoit Donnet - Universite Catholique de Louvain - Faculte des Sciences Appliquees - Departement d'Ingenierie Informatique.
+ * @contract European Commission One-Lab Project 034819.
+ *
+ * @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);
+ }
+
+}
Index: src/contrib/hbase/src/test/org/onelab/test/TestFilter.java
===================================================================
--- src/contrib/hbase/src/test/org/onelab/test/TestFilter.java (revision 0)
+++ src/contrib/hbase/src/test/org/onelab/test/TestFilter.java (revision 0)
@@ -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 Universit� 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 Benoit Donnet - Universite Catholique de Louvain - Faculte des Sciences Appliquees - Departement d'Ingenierie Informatique.
+ * @contract European Commission One-Lab Project 034819.
+ *
+ * @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
Null otherwise.
+ */
+ private BloomFilter getActiveStandardBF() {
+ if(currentNbRecord >= nr) {
+ return null;
+ }
+
+ return matrix[matrix.length - 1];
+ }//end getActiveStandardBF()
+}//end class
Index: src/contrib/hbase/src/java/org/onelab/filter/Filter.java
===================================================================
--- src/contrib/hbase/src/java/org/onelab/filter/Filter.java (revision 0)
+++ src/contrib/hbase/src/java/org/onelab/filter/Filter.java (revision 0)
@@ -0,0 +1,211 @@
+/**
+ * Created on 2 f�vr. 07
+ *
+ * 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 Universit� 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.
+ * A. The
+ * key idea is to map entries of A (also called keys) into several positions
+ * in a vector through the use of several hash functions.
+ * Key will have a default weight of 1.0
+ */
+ double weight;
+
+ /** default constructor - use with readFields */
+ public Key() {}
+
+ /**
+ * Constructor.
+ * null, 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 this retouched Bloom filter.
+ * @param coll The collection of false positive.
+ */
+ public void addFalsePositive(Collection