Author: jimk
Date: Tue Apr  1 10:56:58 2008
New Revision: 643487

URL: http://svn.apache.org/viewvc?rev=643487&view=rev
Log:
HBASE-552   Fix bloom filter bugs (Andrzej Bialecki via Jim Kellerman)

Modified:
    hadoop/hbase/branches/0.1/CHANGES.txt
    hadoop/hbase/branches/0.1/src/java/org/onelab/filter/BloomFilter.java
    hadoop/hbase/branches/0.1/src/java/org/onelab/filter/DynamicBloomFilter.java
    hadoop/hbase/branches/0.1/src/java/org/onelab/filter/Key.java

Modified: hadoop/hbase/branches/0.1/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/hadoop/hbase/branches/0.1/CHANGES.txt?rev=643487&r1=643486&r2=643487&view=diff
==============================================================================
--- hadoop/hbase/branches/0.1/CHANGES.txt (original)
+++ hadoop/hbase/branches/0.1/CHANGES.txt Tue Apr  1 10:56:58 2008
@@ -6,6 +6,7 @@
                iteration, edits are aggregated up into the millions
    HBASE-505   Region assignments should never time out so long as the region
                server reports that it is processing the open request
+   HBASE-552   Fix bloom filter bugs (Andrzej Bialecki via Jim Kellerman)
 
   NEW FEATURES
    HBASE-548   Tool to online single region

Modified: hadoop/hbase/branches/0.1/src/java/org/onelab/filter/BloomFilter.java
URL: 
http://svn.apache.org/viewvc/hadoop/hbase/branches/0.1/src/java/org/onelab/filter/BloomFilter.java?rev=643487&r1=643486&r2=643487&view=diff
==============================================================================
--- hadoop/hbase/branches/0.1/src/java/org/onelab/filter/BloomFilter.java 
(original)
+++ hadoop/hbase/branches/0.1/src/java/org/onelab/filter/BloomFilter.java Tue 
Apr  1 10:56:58 2008
@@ -1,242 +1,240 @@
-/**
- *
- * 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.
- */
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.onelab.filter;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import java.util.BitSet;
-
-/**
- * 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. 
- * 
- * 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 {
-  private static final byte[] bitvalues = new byte[] {
-    (byte)0x01,
-    (byte)0x02,
-    (byte)0x04,
-    (byte)0x08,
-    (byte)0x10,
-    (byte)0x20,
-    (byte)0x40,
-    (byte)0x80
-  };
-  
-  /** The bit vector. */
-  BitSet bits;
-
-  /** Default constructor - use with readFields */
-  public BloomFilter() {
-    super();
-  }
-  
-  /**
-   * 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);
-
-    bits = new BitSet(this.vectorSize);
-  }//end constructor
-
-  /** [EMAIL PROTECTED] */
-  @Override
-  public void add(Key key) {
-    if(key == null) {
-      throw new NullPointerException("key cannot be null");
-    }
-
-    int[] h = hash.hash(key);
-    hash.clear();
-
-    for(int i = 0; i < nbHash; i++) {
-      bits.set(h[i]);
-    }
-  }//end add()
-
-  /** [EMAIL PROTECTED] */
-  @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");
-    }
-
-    this.bits.and(((BloomFilter) filter).bits);
-  }//end and()
-
-  /** [EMAIL PROTECTED] */
-  @Override
-  public boolean membershipTest(Key key){
-    if(key == null) {
-      throw new NullPointerException("key cannot be null");
-    }
-
-    int[] h = hash.hash(key);
-    hash.clear();
-    for(int i = 0; i < nbHash; i++) {
-      if(!bits.get(h[i])) {
-        return false;
-      }
-    }
-    return true;
-  }//end memberhsipTest()
-
-  /** [EMAIL PROTECTED] */
-  @Override
-  public void not(){
-    bits.flip(0, vectorSize - 1);
-  }//end not()
-
-  /** [EMAIL PROTECTED] */
-  @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");
-    }
-    bits.or(((BloomFilter) filter).bits);
-  }//end or()
-
-  /** [EMAIL PROTECTED] */
-  @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");
-    }
-    bits.xor(((BloomFilter) filter).bits);
-  }//and xor()
-
-  /** [EMAIL PROTECTED] */
-  @Override
-  public String toString(){
-    return bits.toString();
-  }//end toString()
-
-  /** [EMAIL PROTECTED] */
-  @Override
-  public Object clone(){
-    BloomFilter bf = new BloomFilter(vectorSize, nbHash);
-    bf.or(this);
-    return bf;
-  }//end clone()
-
-  // Writable
-
-  /** [EMAIL PROTECTED] */
-  @Override
-  public void write(DataOutput out) throws IOException {
-    super.write(out);
-    byte[] bytes = new byte[getNBytes()];
-    for(int i = 0, byteIndex = 0, bitIndex = 0; i < vectorSize; i++, 
bitIndex++) {
-      if (bitIndex == 8) {
-        bitIndex = 0;
-        byteIndex++;
-      }
-      if (bitIndex == 0) {
-        bytes[byteIndex] = 0;
-      }
-      if (bits.get(i)) {
-        bytes[byteIndex] |= bitvalues[bitIndex];
-      }
-    }
-    out.write(bytes);
-  }
-
-  /** [EMAIL PROTECTED] */
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    super.readFields(in);
-    byte[] bytes = new byte[getNBytes()];
-    in.readFully(bytes);
-    for(int i = 0, byteIndex = 0, bitIndex = 0; i < vectorSize; i++, 
bitIndex++) {
-      if (bitIndex == 8) {
-        bitIndex = 0;
-        byteIndex++;
-      }
-      if (bitIndex == 0) {
-        bytes[byteIndex] = 0;
-      }
-      if ((bytes[byteIndex] & bitvalues[bitIndex]) != 0) {
-        bits.set(i);
-      }
-    }
-  }
-  
-  /* @return number of bytes needed to hold bit vector */
-  private int getNBytes() {
-    return (vectorSize + 7) / 8;
-  }
-}//end class
+/**
+ *
+ * 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.
+ */
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.onelab.filter;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import java.util.BitSet;
+
+/**
+ * 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. 
+ * 
+ * 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 {
+  private static final byte[] bitvalues = new byte[] {
+    (byte)0x01,
+    (byte)0x02,
+    (byte)0x04,
+    (byte)0x08,
+    (byte)0x10,
+    (byte)0x20,
+    (byte)0x40,
+    (byte)0x80
+  };
+  
+  /** The bit vector. */
+  BitSet bits;
+
+  /** Default constructor - use with readFields */
+  public BloomFilter() {
+    super();
+  }
+  
+  /**
+   * 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);
+
+    bits = new BitSet(this.vectorSize);
+  }//end constructor
+
+  /** [EMAIL PROTECTED] */
+  @Override
+  public void add(Key key) {
+    if(key == null) {
+      throw new NullPointerException("key cannot be null");
+    }
+
+    int[] h = hash.hash(key);
+    hash.clear();
+
+    for(int i = 0; i < nbHash; i++) {
+      bits.set(h[i]);
+    }
+  }//end add()
+
+  /** [EMAIL PROTECTED] */
+  @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");
+    }
+
+    this.bits.and(((BloomFilter) filter).bits);
+  }//end and()
+
+  /** [EMAIL PROTECTED] */
+  @Override
+  public boolean membershipTest(Key key){
+    if(key == null) {
+      throw new NullPointerException("key cannot be null");
+    }
+
+    int[] h = hash.hash(key);
+    hash.clear();
+    for(int i = 0; i < nbHash; i++) {
+      if(!bits.get(h[i])) {
+        return false;
+      }
+    }
+    return true;
+  }//end memberhsipTest()
+
+  /** [EMAIL PROTECTED] */
+  @Override
+  public void not(){
+    bits.flip(0, vectorSize - 1);
+  }//end not()
+
+  /** [EMAIL PROTECTED] */
+  @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");
+    }
+    bits.or(((BloomFilter) filter).bits);
+  }//end or()
+
+  /** [EMAIL PROTECTED] */
+  @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");
+    }
+    bits.xor(((BloomFilter) filter).bits);
+  }//and xor()
+
+  /** [EMAIL PROTECTED] */
+  @Override
+  public String toString(){
+    return bits.toString();
+  }//end toString()
+
+  /** [EMAIL PROTECTED] */
+  @Override
+  public Object clone(){
+    BloomFilter bf = new BloomFilter(vectorSize, nbHash);
+    bf.or(this);
+    return bf;
+  }//end clone()
+
+  // Writable
+
+  /** [EMAIL PROTECTED] */
+  @Override
+  public void write(DataOutput out) throws IOException {
+    super.write(out);
+    byte[] bytes = new byte[getNBytes()];
+    for(int i = 0, byteIndex = 0, bitIndex = 0; i < vectorSize; i++, 
bitIndex++) {
+      if (bitIndex == 8) {
+        bitIndex = 0;
+        byteIndex++;
+      }
+      if (bitIndex == 0) {
+        bytes[byteIndex] = 0;
+      }
+      if (bits.get(i)) {
+        bytes[byteIndex] |= bitvalues[bitIndex];
+      }
+    }
+    out.write(bytes);
+  }
+
+  /** [EMAIL PROTECTED] */
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    super.readFields(in);
+    bits = new BitSet(vectorSize);
+    byte[] bytes = new byte[getNBytes()];
+    in.readFully(bytes);
+    for(int i = 0, byteIndex = 0, bitIndex = 0; i < vectorSize; i++, 
bitIndex++) {
+      if (bitIndex == 8) {
+        bitIndex = 0;
+        byteIndex++;
+      }
+      if ((bytes[byteIndex] & bitvalues[bitIndex]) != 0) {
+        bits.set(i);
+      }
+    }
+  }
+  
+  /* @return number of bytes needed to hold bit vector */
+  private int getNBytes() {
+    return (vectorSize + 7) / 8;
+  }
+}//end class

Modified: 
hadoop/hbase/branches/0.1/src/java/org/onelab/filter/DynamicBloomFilter.java
URL: 
http://svn.apache.org/viewvc/hadoop/hbase/branches/0.1/src/java/org/onelab/filter/DynamicBloomFilter.java?rev=643487&r1=643486&r2=643487&view=diff
==============================================================================
--- 
hadoop/hbase/branches/0.1/src/java/org/onelab/filter/DynamicBloomFilter.java 
(original)
+++ 
hadoop/hbase/branches/0.1/src/java/org/onelab/filter/DynamicBloomFilter.java 
Tue Apr  1 10:56:58 2008
@@ -1,298 +1,310 @@
-/**
- *
- * 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.
- */
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.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.
- * 
- * 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
-
-  /** [EMAIL PROTECTED] */
-  @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()
-
-  /** [EMAIL PROTECTED] */
-  @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()
-
-  /** [EMAIL PROTECTED] */
-  @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()
-
-  /** [EMAIL PROTECTED] */
-  @Override
-  public void not(){
-    for(int i = 0; i < matrix.length; i++) {
-      matrix[i].not();
-    }
-  }//end not()
-
-  /** [EMAIL PROTECTED] */
-  @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()
-
-  /** [EMAIL PROTECTED] */
-  @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()
-
-  /** [EMAIL PROTECTED] */
-  @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()
-
-  /** [EMAIL PROTECTED] */
-  @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()
-
-  // Writable
-
-  /** [EMAIL PROTECTED] */
-  @Override
-  public void write(DataOutput out) throws IOException {
-    super.write(out);
-    for(int i = 0; i < matrix.length; i++) {
-      matrix[i].write(out);
-    }
-  }
-
-  /** [EMAIL PROTECTED] */
-  @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);
-    }
-  }
-
-  /**
-   * 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
+/**
+ *
+ * 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.
+ */
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.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.
+ * 
+ * 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.
+   */
+  private int nr;
+
+  /**
+   * The number of keys recorded in the current standard active Bloom filter.
+   */
+  private int currentNbRecord;
+
+  /**
+   * The matrix of Bloom filter.
+   */
+  private BloomFilter[] matrix;
+
+  /**
+   * Zero-args constructor for the serialization.
+   */
+  public DynamicBloomFilter() { }
+
+  /**
+   * 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
+
+  /** [EMAIL PROTECTED] */
+  @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()
+
+  /** [EMAIL PROTECTED] */
+  @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()
+
+  /** [EMAIL PROTECTED] */
+  @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()
+
+  /** [EMAIL PROTECTED] */
+  @Override
+  public void not(){
+    for(int i = 0; i < matrix.length; i++) {
+      matrix[i].not();
+    }
+  }//end not()
+
+  /** [EMAIL PROTECTED] */
+  @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()
+
+  /** [EMAIL PROTECTED] */
+  @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()
+
+  /** [EMAIL PROTECTED] */
+  @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()
+
+  /** [EMAIL PROTECTED] */
+  @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()
+
+  // Writable
+
+  /** [EMAIL PROTECTED] */
+  @Override
+  public void write(DataOutput out) throws IOException {
+    super.write(out);
+    out.writeInt(nr);
+    out.writeInt(currentNbRecord);
+    out.writeInt(matrix.length);
+    for (int i = 0; i < matrix.length; i++) {
+      matrix[i].write(out);
+    }
+  }
+
+  /** [EMAIL PROTECTED] */
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    super.readFields(in);
+    nr = in.readInt();
+    currentNbRecord = in.readInt();
+    int len = in.readInt();
+    matrix = new BloomFilter[len];
+    for (int i = 0; i < matrix.length; i++) {
+      matrix[i] = new BloomFilter();
+      matrix[i].readFields(in);
+    }
+  }
+
+  /**
+   * 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

Modified: hadoop/hbase/branches/0.1/src/java/org/onelab/filter/Key.java
URL: 
http://svn.apache.org/viewvc/hadoop/hbase/branches/0.1/src/java/org/onelab/filter/Key.java?rev=643487&r1=643486&r2=643487&view=diff
==============================================================================
--- hadoop/hbase/branches/0.1/src/java/org/onelab/filter/Key.java (original)
+++ hadoop/hbase/branches/0.1/src/java/org/onelab/filter/Key.java Tue Apr  1 
10:56:58 2008
@@ -1,173 +1,177 @@
-/**
- *
- * 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.
- */
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.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()
-
-  /** [EMAIL PROTECTED] */
-  @Override
-  public boolean equals(Object o) {
-    return this.compareTo(o) == 0;
-  }
-  
-  /** [EMAIL PROTECTED] */
-  @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
-
-  /** [EMAIL PROTECTED] */
-  public void write(DataOutput out) throws IOException {
-    out.writeInt(bytes.length);
-    out.write(bytes);
-    out.writeDouble(weight);
-  }
-  
-  /** [EMAIL PROTECTED] */
-  public void readFields(DataInput in) throws IOException {
-    this.bytes = new byte[in.readInt()];
-    in.readFully(this.bytes);
-    weight = in.readDouble();
-  }
-  
-  // Comparable
-  
-  /** [EMAIL PROTECTED] */
-  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
+/**
+ *
+ * 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.
+ */
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.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) {
+    set(value, weight);
+  }//end constructor
+
+  public void set(byte[] value, double weight) {
+    if(value == null) {
+      throw new IllegalArgumentException("value can not be null");
+    }
+    this.bytes = value;
+    this.weight = weight;
+  }
+  
+  /** @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()
+
+  /** [EMAIL PROTECTED] */
+  @Override
+  public boolean equals(Object o) {
+    return this.compareTo(o) == 0;
+  }
+  
+  /** [EMAIL PROTECTED] */
+  @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
+
+  /** [EMAIL PROTECTED] */
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(bytes.length);
+    out.write(bytes);
+    out.writeDouble(weight);
+  }
+  
+  /** [EMAIL PROTECTED] */
+  public void readFields(DataInput in) throws IOException {
+    this.bytes = new byte[in.readInt()];
+    in.readFully(this.bytes);
+    weight = in.readDouble();
+  }
+  
+  // Comparable
+  
+  /** [EMAIL PROTECTED] */
+  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


Reply via email to