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