You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ns...@apache.org on 2011/10/11 04:19:55 UTC

svn commit: r1181557 [1/2] - in /hbase/branches/0.89/src: main/java/org/apache/hadoop/hbase/regionserver/ main/java/org/apache/hadoop/hbase/util/ main/resources/ test/java/org/apache/hadoop/hbase/regionserver/ test/java/org/apache/hadoop/hbase/util/

Author: nspiegelberg
Date: Tue Oct 11 02:19:54 2011
New Revision: 1181557

URL: http://svn.apache.org/viewvc?rev=1181557&view=rev
Log:
Compound Bloom filters

Summary:
This is a port of D237675 to be consistent with other HFile format version 2
changes. Instead of breaking an existing ByteBloomFilter's bit array into
blocks, we accumulate keys into a Bloom filter chunk as an HFile is being
written, and flush this Bloom filter to disk as soon as a fixed number of keys
is reached. At read time, the appropriate Bloom filter block is determined using
binary search on the key, loaded, cached, and queried.

New configuration options:
io.storefile.bloom.compound - a boolean option specifying whether compound
Bloom filters are turned on
io.storefile.bloom.block.size - the target size of a single Bloom filter chunk
(only takes effect when io.storefile.bloom.compound=true)
io.storefile.bloom.cacheonwrite - allows to enable cache-on-write for Bloom
chunks

The TestCompoundBloomFilter unit test validates the false positive rate to
ensure it closely matches what is expected. Compound Bloom filters do not rely
on an estimate of how many keys will be added to the Bloom filter, so they can
hit the target false positive rate much more precisely. There are still reasons
for the discrepancy between the theoretical and observed false positive rate,
and they are controlled for in the unit test:
- Previously, the number of bits in the Bloom was computed as if the number of
hash functions could be an arbitrary number, while it must be an integer. In
this implementation, the number of bits is adjusted to ensure the given target
error rate as soon as the precise (rounded) number of hash functions is known.
- The last chunk of a compound Bloom filter may be undersized, even if it
undergoes folding, so the observed error rate may be lower than expected.
Therefore, the last chunk of a multi-chunk compound Bloom filter is excluded
when checking the lower bound of the false positive rate.
- We use combinatorial hashing, which gives a slightly higher false positive
rate than a set of truly independent hash functions (thanks to Nicolas for
pointing this out). However, we can test the false positive rate as though we
had independent hash functions, by looking up random bits instead of the bits
the hash functions point to. This technique is referred to as "fake lookups" in
the code and is obviously only valid for the unit test. That said, it does yield
the false positive rate that is closer to the expectation than that of our
production Bloom filters.

While most of the Bloom filter logic is kept out of HFile, there is only one
version of StoreFile (i.e. there is no hierarchy of StoreFile implementations
similar to that introduced for StoreFile in D251875). Therefore, HFile needs to
know where to store Bloom filter metadata, even though it does not need to know
anything about how that metadata is structured. In version 2, we store Bloom
filter metadata as part of the "load-on-open" section of the HFile.

Finally, Bloom filter code has been split into a writer interface and Bloom
filter itself (which one might call reader) to avoid keeping around fields and
code that is only used when writing at read time. Note
that CompoundBloomFilterWriter implements InlineBlockWriter, an interface
introduced in D250060 for writing "inline" blocks into the data section of a
version 2 HFile.

Test Plan:
Unit tests. Load test (HBaseTest).
Still need to improve HBaseTest to allow testing ROWCOL Bloom filters.

Reviewed By: nspiegelberg
Reviewers: nspiegelberg, kannan, kranganathan, gqchen, jgray, aaiyer
Commenters: jgray
CC: hbase-hdfs@lists, , mbautin, jgray, nspiegelberg, kenny
Revert Plan:
HFile format version 2 introduces backwards-compatible but
non-reverse-compatible changes.

Differential Revision: 251932

Added:
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/BloomFilterBase.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/BloomFilterWriter.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilter.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterBase.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterWriter.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java
Modified:
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/BloomFilter.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/ByteBloomFilter.java
    hbase/branches/0.89/src/main/resources/hbase-default.xml
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/util/TestByteBloomFilter.java

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java?rev=1181557&r1=1181556&r2=1181557&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java Tue Oct 11 02:19:54 2011
@@ -19,12 +19,12 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import java.io.DataInput;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.lang.management.ManagementFactory;
 import java.lang.management.MemoryUsage;
 import java.nio.ByteBuffer;
-import java.text.NumberFormat;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
@@ -51,11 +51,12 @@ import org.apache.hadoop.hbase.io.hfile.
 import org.apache.hadoop.hbase.io.hfile.Compression;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.io.hfile.HFileWriterV1;
 import org.apache.hadoop.hbase.io.hfile.LruBlockCache;
 import org.apache.hadoop.hbase.util.BloomFilter;
-import org.apache.hadoop.hbase.util.ByteBloomFilter;
+import org.apache.hadoop.hbase.util.BloomFilterFactory;
+import org.apache.hadoop.hbase.util.BloomFilterWriter;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Hash;
 import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.io.RawComparator;
 import org.apache.hadoop.io.WritableUtils;
@@ -68,10 +69,10 @@ import com.google.common.collect.Orderin
 /**
  * A Store data file.  Stores usually have one or more of these files.  They
  * are produced by flushing the memstore to disk.  To
- * create, call {@link #createWriter(FileSystem, Path, int)} and append data.  Be
- * sure to add any metadata before calling close on the Writer
- * (Use the appendMetadata convenience methods). On close, a StoreFile is
- * sitting in the Filesystem.  To refer to it, create a StoreFile instance
+ * create, call {@link #createWriter(FileSystem, Path, int, Configuration)}
+ * and append data. Be sure to add any metadata before calling close on the
+ * Writer (Use the appendMetadata convenience methods). On close, a StoreFile
+ * is sitting in the Filesystem.  To refer to it, create a StoreFile instance
  * passing filesystem and path.  To read, call {@link #createReader()}.
  * <p>StoreFiles may also reference store files in another Store.
  *
@@ -81,11 +82,6 @@ import com.google.common.collect.Orderin
 public class StoreFile {
   static final Log LOG = LogFactory.getLog(StoreFile.class.getName());
 
-  // Config keys.
-  static final String IO_STOREFILE_BLOOM_ERROR_RATE = "io.storefile.bloom.error.rate";
-  static final String IO_STOREFILE_BLOOM_MAX_FOLD = "io.storefile.bloom.max.fold";
-  static final String IO_STOREFILE_BLOOM_MAX_KEYS = "io.storefile.bloom.max.keys";
-  static final String IO_STOREFILE_BLOOM_ENABLED = "io.storefile.bloom.enabled";
   static final String HFILE_BLOCK_CACHE_SIZE_KEY = "hfile.block.cache.size";
 
   public static enum BloomType {
@@ -102,21 +98,26 @@ public class StoreFile {
      */
     ROWCOL
   }
+
   // Keys for fileinfo values in HFile
+
   /** Max Sequence ID in FileInfo */
   public static final byte [] MAX_SEQ_ID_KEY = Bytes.toBytes("MAX_SEQ_ID_KEY");
+
   /** Major compaction flag in FileInfo */
-  public static final byte [] MAJOR_COMPACTION_KEY = Bytes.toBytes("MAJOR_COMPACTION_KEY");
+  public static final byte[] MAJOR_COMPACTION_KEY =
+      Bytes.toBytes("MAJOR_COMPACTION_KEY");
+
   /** Bloom filter Type in FileInfo */
-  static final byte[] BLOOM_FILTER_TYPE_KEY = Bytes.toBytes("BLOOM_FILTER_TYPE");
+  static final byte[] BLOOM_FILTER_TYPE_KEY =
+      Bytes.toBytes("BLOOM_FILTER_TYPE");
+
+  /** Last Bloom filter key in FileInfo */
+  private static final byte[] LAST_BLOOM_KEY = Bytes.toBytes("LAST_BLOOM_KEY");
+
   /** Key for Timerange information in metadata*/
   static final byte[] TIMERANGE_KEY = Bytes.toBytes("TIMERANGE");
 
-  /** Meta data block name for bloom filter meta-info (ie: bloom params/specs) */
-  static final String BLOOM_FILTER_META_KEY = "BLOOM_FILTER_META";
-  /** Meta data block name for bloom filter data (ie: bloom bits) */
-  static final String BLOOM_FILTER_DATA_KEY = "BLOOM_FILTER_DATA";
-
   // Make default block size for StoreFiles 8k while testing.  TODO: FIX!
   // Need to make it 8k for testing.
   public static final int DEFAULT_BLOCKSIZE_SMALL = 8 * 1024;
@@ -125,14 +126,19 @@ public class StoreFile {
   private static BlockCache hfileBlockCache = null;
 
   private final FileSystem fs;
+
   // This file's path.
   private final Path path;
+
   // If this storefile references another, this is the reference instance.
   private Reference reference;
+
   // If this StoreFile references another, this is the other files path.
   private Path referencePath;
+
   // Should the block cache be used or not.
   private boolean blockcache;
+
   // Is this from an in-memory store
   private boolean inMemory;
 
@@ -201,11 +207,12 @@ public class StoreFile {
     }
     // ignore if the column family config says "no bloom filter"
     // even if there is one in the hfile.
-    if (conf.getBoolean(IO_STOREFILE_BLOOM_ENABLED, true)) {
+    if (BloomFilterFactory.isBloomEnabled(conf)) {
       this.bloomType = bt;
     } else {
+      LOG.info("Ignoring bloom filter check for file " + path + ": " +
+          "bloomType=" + bt + " (disabled in config)");
       this.bloomType = BloomType.NONE;
-      LOG.info("Ignoring bloom filter check for file (disabled in config)");
     }
 
     // cache the modification time stamp of this store file
@@ -396,7 +403,7 @@ public class StoreFile {
     } else {
       this.reader = new Reader(this.fs, this.path, getBlockCache(),
           this.inMemory,
-          this.conf.getBoolean("hbase.rs.evictblocksonclose", true));
+          this.conf.getBoolean(HFile.EVICT_BLOCKS_ON_CLOSE_KEY, true));
     }
 
     // Load up indices and fileinfo.
@@ -541,13 +548,10 @@ public class StoreFile {
    * @return StoreFile.Writer
    * @throws IOException
    */
-  public static Writer createWriter(final FileSystem fs,
-                                              final Path dir,
-                                              final int blocksize)
-      throws IOException {
-
-    return createWriter(fs, dir, blocksize, null, null, null, BloomType.NONE, 0,
-        false);
+  public static Writer createWriter(final FileSystem fs, final Path dir,
+      final int blocksize, Configuration conf) throws IOException {
+    return createWriter(fs, dir, blocksize, null, null, conf, BloomType.NONE,
+        0);
   }
 
   /**
@@ -558,10 +562,10 @@ public class StoreFile {
    * Creates a file with a unique name in this directory.
    * @param blocksize
    * @param algorithm Pass null to get default.
+   * @param c Pass null to get default.
    * @param conf HBase system configuration. used with bloom filters
    * @param bloomType column family setting for bloom filters
-   * @param c Pass null to get default.
-   * @param maxKeySize peak theoretical entry size (maintains error rate)
+   * @param maxKeyCount estimated maximum number of keys we expect to add
    * @return HFile.Writer
    * @throws IOException
    */
@@ -572,22 +576,20 @@ public class StoreFile {
                                               final KeyValue.KVComparator c,
                                               final Configuration conf,
                                               BloomType bloomType,
-                                              long maxKeySize,
-                                              final boolean cacheOnWrite)
+                                              long maxKeyCount)
       throws IOException {
 
     if (!fs.exists(dir)) {
       fs.mkdirs(dir);
     }
     Path path = getUniqueFile(fs, dir);
-    if(conf == null || !conf.getBoolean(IO_STOREFILE_BLOOM_ENABLED, true)) {
+    if (!BloomFilterFactory.isBloomEnabled(conf)) {
       bloomType = BloomType.NONE;
     }
 
     return new Writer(fs, path, blocksize,
         algorithm == null? HFile.DEFAULT_COMPRESSION_ALGORITHM: algorithm,
-        conf, c == null? KeyValue.COMPARATOR: c, bloomType, maxKeySize,
-            cacheOnWrite);
+        conf, c == null ? KeyValue.COMPARATOR: c, bloomType, maxKeyCount);
   }
 
   /**
@@ -671,11 +673,13 @@ public class StoreFile {
    * local because it is an implementation detail of the HBase regionserver.
    */
   public static class Writer {
-    private final BloomFilter bloomFilter;
+    private final BloomFilterWriter bloomFilterWriter;
     private final BloomType bloomType;
+    private byte[] lastBloomKey;
+    private int lastBloomKeyOffset, lastBloomKeyLen;
     private KVComparator kvComparator;
     private KeyValue lastKv = null;
-    private byte[] lastByteArray = null;
+
     TimeRangeTracker timeRangeTracker = new TimeRangeTracker();
     /* isTimeRangeTrackerSet keeps track if the timeRange has already been set
      * When flushing a memstore, we set TimeRange and use this variable to
@@ -695,61 +699,31 @@ public class StoreFile {
      * @param conf user configuration
      * @param comparator key comparator
      * @param bloomType bloom filter setting
-     * @param maxKeys maximum amount of keys to add (for blooms)
-     * @param cacheOnWrite whether to cache blocks as we write file
+     * @param maxKeys the expected maximum number of keys to be added. Was used
+     *        for Bloom filter size in {@link HFile} format version 1.
      * @throws IOException problem writing to FS
      */
     public Writer(FileSystem fs, Path path, int blocksize,
         Compression.Algorithm compress, final Configuration conf,
-        final KVComparator comparator, BloomType bloomType, long maxKeys,
-        boolean cacheOnWrite)
+        final KVComparator comparator, BloomType bloomType, long maxKeys)
         throws IOException {
 
-      writer = new HFile.Writer(
+      writer = HFile.getWriterFactory(conf).createWriter(
 		fs, path, blocksize, HFile.getBytesPerChecksum(conf, fs.getConf()),
-		compress, comparator.getRawComparator(),
-		cacheOnWrite ? StoreFile.getBlockCache(conf) : null);
+		compress, comparator.getRawComparator());
 
       this.kvComparator = comparator;
 
-      BloomFilter bloom = null;
-      BloomType bt = BloomType.NONE;
-
-      if (bloomType != BloomType.NONE && conf != null) {
-        float err = conf.getFloat(IO_STOREFILE_BLOOM_ERROR_RATE, (float)0.01);
-        // Since in row+col blooms we have 2 calls to shouldSeek() instead of 1
-        // and the false positives are adding up, we should keep the error rate
-        // twice as low in order to maintain the number of false positives as
-        // desired by the user
-        if (bloomType == BloomType.ROWCOL) {
-          err /= 2;
-        }
-        int maxFold = conf.getInt(IO_STOREFILE_BLOOM_MAX_FOLD, 7);
-        int tooBig = conf.getInt(IO_STOREFILE_BLOOM_MAX_KEYS, 128*1000*1000);
-
-        if (maxKeys < tooBig) {
-          try {
-            bloom = new ByteBloomFilter((int)maxKeys, err,
-                Hash.getHashType(conf), maxFold);
-            bloom.allocBloom();
-            bt = bloomType;
-          } catch (IllegalArgumentException iae) {
-            LOG.warn(String.format(
-              "Parse error while creating bloom for %s (%d, %f)",
-              path, maxKeys, err), iae);
-            bloom = null;
-            bt = BloomType.NONE;
-          }
-        } else {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Skipping bloom filter because max keysize too large: "
-                + maxKeys);
-          }
-        }
+      bloomFilterWriter = BloomFilterFactory.createBloomAtWrite(conf,
+          bloomType, (int) Math.min(maxKeys, Integer.MAX_VALUE), writer);
+      if (bloomFilterWriter != null) {
+        this.bloomType = bloomType;
+        LOG.info("Bloom filter type for " + path + ": " + this.bloomType +
+            ", "+ bloomFilterWriter.getClass().getSimpleName());
+      } else {
+        // Not using Bloom filters.
+        this.bloomType = BloomType.NONE;
       }
-
-      this.bloomFilter = bloom;
-      this.bloomType = bt;
     }
 
     /**
@@ -808,7 +782,7 @@ public class StoreFile {
     }
 
     public void append(final KeyValue kv) throws IOException {
-      if (this.bloomFilter != null) {
+      if (this.bloomFilterWriter != null) {
         // only add to the bloom filter on a new, unique key
         boolean newKey = true;
         if (this.lastKv != null) {
@@ -832,24 +806,42 @@ public class StoreFile {
            *  1. Row = Row
            *  2. RowCol = Row + Qualifier
            */
+          byte[] bloomKey;
+          int bloomKeyOffset, bloomKeyLen;
+
           switch (bloomType) {
           case ROW:
-            this.bloomFilter.add(kv.getBuffer(), kv.getRowOffset(),
-                kv.getRowLength());
+            bloomKey = kv.getBuffer();
+            bloomKeyOffset = kv.getRowOffset();
+            bloomKeyLen = kv.getRowLength();
             break;
           case ROWCOL:
             // merge(row, qualifier)
-            int ro = kv.getRowOffset();
-            int rl = kv.getRowLength();
-            int qo = kv.getQualifierOffset();
-            int ql = kv.getQualifierLength();
-            byte [] result = new byte[rl + ql];
-            System.arraycopy(kv.getBuffer(), ro, result, 0,  rl);
-            System.arraycopy(kv.getBuffer(), qo, result, rl, ql);
-            this.bloomFilter.add(result);
+            // TODO: could save one buffer copy in case of compound Bloom
+            // filters when this involves creating a KeyValue
+            bloomKey = bloomFilterWriter.createBloomKey(kv.getBuffer(),
+                kv.getRowOffset(), kv.getRowLength(), kv.getBuffer(),
+                kv.getQualifierOffset(), kv.getQualifierLength());
+            bloomKeyOffset = 0;
+            bloomKeyLen = bloomKey.length;
             break;
           default:
+            throw new IOException("Invalid Bloom filter type: " + bloomType);
           }
+          bloomFilterWriter.add(bloomKey, bloomKeyOffset, bloomKeyLen);
+          if (lastBloomKey != null
+              && bloomFilterWriter.getComparator().compare(bloomKey,
+                  bloomKeyOffset, bloomKeyLen, lastBloomKey,
+                  lastBloomKeyOffset, lastBloomKeyLen) <= 0) {
+            throw new IOException("Non-increasing Bloom keys: "
+                + Bytes.toStringBinary(bloomKey, bloomKeyOffset, bloomKeyLen)
+                + " after "
+                + Bytes.toStringBinary(lastBloomKey, lastBloomKeyOffset,
+                    lastBloomKeyLen));
+          }
+          lastBloomKey = bloomKey;
+          lastBloomKeyOffset = bloomKeyOffset;
+          lastBloomKeyLen = bloomKeyLen;
           this.lastKv = kv;
         }
       }
@@ -862,39 +854,41 @@ public class StoreFile {
     }
 
     boolean hasBloom() {
-      return this.bloomFilter != null;
+      return this.bloomFilterWriter != null;
     }
 
-    public void append(final byte [] key, final byte [] value) throws IOException {
-      if (this.bloomFilter != null) {
-        // only add to the bloom filter on a new row
-        if (this.lastByteArray == null || !Arrays.equals(key, lastByteArray)) {
-          this.bloomFilter.add(key);
-          this.lastByteArray = key;
-        }
-      }
-      writer.append(key, value);
-      includeInTimeRangeTracker(key);
+    /**
+     * For unit testing only.
+     * @return the Bloom filter used by this writer.
+     */
+    BloomFilterWriter getBloomWriter() {
+      return bloomFilterWriter;
     }
 
     public void close() throws IOException {
-      // make sure we wrote something to the bloom before adding it
-      if (this.bloomFilter != null && this.bloomFilter.getKeyCount() > 0) {
-        bloomFilter.compactBloom();
-        if (this.bloomFilter.getMaxKeys() > 0) {
-          int b = this.bloomFilter.getByteSize();
-          int k = this.bloomFilter.getKeyCount();
-          int m = this.bloomFilter.getMaxKeys();
-          StoreFile.LOG.info("Bloom added to HFile ("
-              + getPath() + "): " + StringUtils.humanReadableInt(b) + ", " +
-              k + "/" + m + " (" + NumberFormat.getPercentInstance().format(
-                ((double)k) / ((double)m)) + ")");
+      // Make sure we wrote something to the Bloom filter before adding it.
+      boolean haveBloom = bloomFilterWriter != null &&
+          bloomFilterWriter.getKeyCount() > 0;
+      if (haveBloom) {
+        bloomFilterWriter.compactBloom();
+        writer.addBloomFilter(bloomFilterWriter);
+        writer.appendFileInfo(BLOOM_FILTER_TYPE_KEY,
+            Bytes.toBytes(bloomType.toString()));
+        if (lastBloomKey != null) {
+          writer.appendFileInfo(LAST_BLOOM_KEY, Arrays.copyOfRange(
+              lastBloomKey, lastBloomKeyOffset, lastBloomKeyOffset
+                  + lastBloomKeyLen));
         }
-        writer.appendMetaBlock(BLOOM_FILTER_META_KEY, bloomFilter.getMetaWriter());
-        writer.appendMetaBlock(BLOOM_FILTER_DATA_KEY, bloomFilter.getDataWriter());
-        writer.appendFileInfo(BLOOM_FILTER_TYPE_KEY, Bytes.toBytes(bloomType.toString()));
       }
       writer.close();
+
+      // Log final Bloom filter statistics. This needs to be done after close()
+      // because compound Bloom filters might be finalized as part of closing.
+      if (haveBloom && bloomFilterWriter.getMaxKeys() > 0) {
+        StoreFile.LOG.info("Bloom added to HFile ("
+            + getPath() + "): " +
+            bloomFilterWriter.toString().replace("\n", "; "));
+      }
     }
 
     public void appendFileInfo(byte[] key, byte[] value) throws IOException {
@@ -920,15 +914,17 @@ public class StoreFile {
     protected long sequenceID = -1;
     private final String bloomAccessedMetric;
     private final String bloomSkippedMetric;
+    private byte[] lastBloomKey;
 
     public Reader(FileSystem fs, Path path, BlockCache blockCache,
         boolean inMemory, boolean evictOnClose)
         throws IOException {
-      reader = new HFile.Reader(fs, path, blockCache, inMemory, evictOnClose);
+      reader = HFile.createReader(fs, path, blockCache, inMemory, evictOnClose);
 
       // prepare the text (key) for the metrics
-      bloomAccessedMetric = reader.cfName + ".keyMaybeInBloomCnt";
-      bloomSkippedMetric = reader.cfName + ".keyNotInBloomCnt";
+      bloomAccessedMetric = reader.getColumnFamilyName()
+          + ".keyMaybeInBloomCnt";
+      bloomSkippedMetric = reader.getColumnFamilyName() + ".keyNotInBloomCnt";
       bloomFilterType = BloomType.NONE;
     }
 
@@ -1009,7 +1005,7 @@ public class StoreFile {
     }
 
     public boolean shouldSeek(Scan scan, final SortedSet<byte[]> columns) {
-        return (passesTimerangeFilter(scan) && passesBloomFilter(scan,columns));
+      return (passesTimerangeFilter(scan) && passesBloomFilter(scan, columns));
     }
 
     /**
@@ -1025,40 +1021,78 @@ public class StoreFile {
       }
     }
 
-    private boolean passesBloomFilter(Scan scan, final SortedSet<byte[]> columns) {
+    private boolean passesBloomFilter(Scan scan,
+        final SortedSet<byte[]> columns) {
       if (this.bloomFilter == null || !scan.isGetScan()) {
         return true;
       }
+
+      // Empty file?
+      if (reader.getTrailer().getEntryCount() == 0)
+        return false;
+
       byte[] row = scan.getStartRow();
       byte[] key;
       switch (this.bloomFilterType) {
         case ROW:
           key = row;
           break;
+
         case ROWCOL:
           if (columns != null && columns.size() == 1) {
-            byte[] col = columns.first();
-            key = Bytes.add(row, col);
+            byte[] column = columns.first();
+            key = bloomFilter.createBloomKey(row, 0, row.length,
+                column, 0, column.length);
             break;
           }
-          //$FALL-THROUGH$
+          return true;
+
         default:
           return true;
       }
 
       try {
-        ByteBuffer bloom = reader.getMetaBlock(BLOOM_FILTER_DATA_KEY, true);
-        if (bloom != null) {
+        boolean shouldCheckBloom;
+        ByteBuffer bloom;
+        if (bloomFilter.supportsAutoLoading()) {
+          bloom = null;
+          shouldCheckBloom = true;
+        } else {
+          bloom = reader.getMetaBlock(HFileWriterV1.BLOOM_FILTER_DATA_KEY,
+              true);
+          shouldCheckBloom = bloom != null;
+        }
+
+        if (shouldCheckBloom) {
           boolean exists;
-          if (this.bloomFilterType == BloomType.ROWCOL) {
+
+          // Whether the primary Bloom key is greater than the last Bloom key
+          // from the file info. For row-column Bloom filters this is not yet
+          // a sufficient condition to return false.
+          boolean keyIsAfterLast = lastBloomKey != null
+              && bloomFilter.getComparator().compare(key, lastBloomKey) > 0;
+
+          if (bloomFilterType == BloomType.ROWCOL) {
             // Since a Row Delete is essentially a DeleteFamily applied to all
             // columns, a file might be skipped if using row+col Bloom filter.
             // In order to ensure this file is included an additional check is
             // required looking only for a row bloom.
-            exists = this.bloomFilter.contains(key, bloom) ||
-                     this.bloomFilter.contains(row, bloom);
+            byte[] rowBloomKey = bloomFilter.createBloomKey(row, 0, row.length,
+                null, 0, 0);
+
+            if (keyIsAfterLast
+                && bloomFilter.getComparator().compare(rowBloomKey,
+                    lastBloomKey) > 0) {
+              exists = false;
+            } else {
+              exists =
+                  this.bloomFilter.contains(key, 0, key.length, bloom) ||
+                  this.bloomFilter.contains(rowBloomKey, 0, rowBloomKey.length,
+                      bloom);
+            }
           } else {
-            exists = this.bloomFilter.contains(key, bloom);
+            exists = !keyIsAfterLast
+                && this.bloomFilter.contains(key, 0, key.length, bloom);
           }
 
           if (exists)
@@ -1087,6 +1121,8 @@ public class StoreFile {
         bloomFilterType = BloomType.valueOf(Bytes.toString(b));
       }
 
+      lastBloomKey = fi.get(LAST_BLOOM_KEY);
+
       return fi;
     }
 
@@ -1096,16 +1132,17 @@ public class StoreFile {
       }
 
       try {
-        ByteBuffer b = reader.getMetaBlock(BLOOM_FILTER_META_KEY, false);
-        if (b != null) {
+        DataInput bloomMeta = reader.getBloomFilterMetadata();
+        if (bloomMeta != null) {
           if (bloomFilterType == BloomType.NONE) {
-            throw new IOException("valid bloom filter type not found in FileInfo");
+            throw new IOException(
+                "valid bloom filter type not found in FileInfo");
           }
 
-
-          this.bloomFilter = new ByteBloomFilter(b);
-          LOG.info("Loaded " + (bloomFilterType== BloomType.ROW? "row":"col")
-                 + " bloom filter metadata for " + reader.getName());
+          bloomFilter = BloomFilterFactory.createFromMeta(bloomMeta, reader);
+          LOG.info("Loaded " + bloomFilterType + " " +
+              bloomFilter.getClass().getSimpleName() + " metadata for " +
+              reader.getName());
         }
       } catch (IOException e) {
         LOG.error("Error reading bloom filter meta -- proceeding without", e);
@@ -1116,13 +1153,16 @@ public class StoreFile {
       }
     }
 
+    /**
+     * The number of Bloom filter entries in this store file, or an estimate
+     * thereof, if the Bloom filter is not loaded. This always returns an upper
+     * bound of the number of Bloom filter entries.
+     *
+     * @return an estimate of the number of Bloom filter entries in this file
+     */
     public long getFilterEntries() {
-      return (this.bloomFilter != null) ? this.bloomFilter.getKeyCount()
-          : reader.getFilterEntries();
-    }
-
-    public ByteBuffer getMetaBlock(String bloomFilterDataKey, boolean cacheBlock) throws IOException {
-      return reader.getMetaBlock(bloomFilterDataKey, cacheBlock);
+      return bloomFilter != null ? bloomFilter.getKeyCount()
+          : reader.getEntries();
     }
 
     public void setBloomFilterFaulty() {
@@ -1164,6 +1204,28 @@ public class StoreFile {
     public void setSequenceID(long sequenceID) {
       this.sequenceID = sequenceID;
     }
+
+    BloomFilter getBloomFilter() {
+      return bloomFilter;
+    }
+
+    long getUncompressedDataIndexSize() {
+      return reader.getTrailer().getUncompressedDataIndexSize();
+    }
+
+    public long getTotalBloomSize() {
+      if (bloomFilter == null)
+        return 0;
+      return bloomFilter.getByteSize();
+    }
+
+    public int getHFileVersion() {
+      return reader.getTrailer().getVersion();
+    }
+
+    HFile.Reader getHFileReader() {
+      return reader;
+    }
   }
 
   /**
@@ -1215,4 +1277,5 @@ public class StoreFile {
         }
       });
   }
+
 }

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/BloomFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/BloomFilter.java?rev=1181557&r1=1181556&r2=1181557&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/BloomFilter.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/BloomFilter.java Tue Oct 11 02:19:54 2011
@@ -19,61 +19,28 @@
  */
 package org.apache.hadoop.hbase.util;
 
-import org.apache.hadoop.io.Writable;
-
 import java.nio.ByteBuffer;
 
 /**
  * Defines the general behavior of a bloom filter.
- * <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.
  *
  * <p>
- * Originally created by
- * <a href="http://www.one-lab.org">European Commission One-Lab Project 034819</a>.
+ * 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.
  *
- * <p>
- * It must be extended in order to define the real behavior.
+ * @see {@link BloomFilterWriter} for the ability to add elements to a Bloom
+ *      filter
  */
-public interface BloomFilter {
-  /**
-   * Allocate memory for the bloom filter data.  Note that bloom data isn't
-   * allocated by default because it can grow large & reads would be better
-   * managed by the LRU cache.
-   */
-  void allocBloom();
-
-  /**
-   * Add the specified binary to the bloom filter.
-   *
-   * @param buf data to be added to the bloom
-   */
-  void add(byte []buf);
-
-  /**
-   * Add the specified binary to the bloom filter.
-   *
-   * @param buf data to be added to the bloom
-   * @param offset offset into the data to be added
-   * @param len length of the data to be added
-   */
-  void add(byte []buf, int offset, int len);
-
-  /**
-   * Check if the specified key is contained in the bloom filter.
-   *
-   * @param buf data to check for existence of
-   * @param bloom bloom filter data to search
-   * @return true if matched by bloom, false if not
-   */
-  boolean contains(byte [] buf, ByteBuffer bloom);
+public interface BloomFilter extends BloomFilterBase {
 
   /**
    * Check if the specified key is contained in the bloom filter.
@@ -81,41 +48,16 @@ public interface BloomFilter {
    * @param buf data to check for existence of
    * @param offset offset into the data
    * @param length length of the data
-   * @param bloom bloom filter data to search
+   * @param bloom bloom filter data to search. This can be null if auto-loading
+   *        is supported.
    * @return true if matched by bloom, false if not
    */
   boolean contains(byte [] buf, int offset, int length, ByteBuffer bloom);
 
   /**
-   * @return The number of keys added to the bloom
-   */
-  int getKeyCount();
-
-  /**
-   * @return The max number of keys that can be inserted
-   *         to maintain the desired error rate
-   */
-  public int getMaxKeys();
-
-  /**
-   * @return Size of the bloom, in bytes
+   * @return true if this Bloom filter can automatically load its data
+   *         and thus allows a null byte buffer to be passed to contains()
    */
-  public int getByteSize();
+  boolean supportsAutoLoading();
 
-  /**
-   * Compact the bloom before writing metadata & data to disk
-   */
-  void compactBloom();
-
-  /**
-   * Get a writable interface into bloom filter meta data.
-   * @return writable class
-   */
-  Writable getMetaWriter();
-
-  /**
-   * Get a writable interface into bloom filter data (actual bloom).
-   * @return writable class
-   */
-  Writable getDataWriter();
 }

Added: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/BloomFilterBase.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/BloomFilterBase.java?rev=1181557&view=auto
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/BloomFilterBase.java (added)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/BloomFilterBase.java Tue Oct 11 02:19:54 2011
@@ -0,0 +1,56 @@
+/*
+ * Copyright 2011 The Apache Software Foundation
+ *
+ * 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.apache.hadoop.hbase.util;
+
+import org.apache.hadoop.io.RawComparator;
+
+/**
+ * Common methods Bloom filter methods required at read and write time.
+ */
+public interface BloomFilterBase {
+
+  /**
+   * @return The number of keys added to the bloom
+   */
+  long getKeyCount();
+
+  /**
+   * @return The max number of keys that can be inserted
+   *         to maintain the desired error rate
+   */
+  long getMaxKeys();
+
+  /**
+   * @return Size of the bloom, in bytes
+   */
+  long getByteSize();
+
+  /**
+   * Create a key for a row-column Bloom filter.
+   */
+  byte[] createBloomKey(byte[] rowBuf, int rowOffset, int rowLen,
+      byte[] qualBuf, int qualOffset, int qualLen);
+
+  /**
+   * @return Bloom key comparator
+   */
+  RawComparator<byte[]> getComparator();
+
+}

Added: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java?rev=1181557&view=auto
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java (added)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java Tue Oct 11 02:19:54 2011
@@ -0,0 +1,208 @@
+/*
+ * Copyright 2011 The Apache Software Foundation
+ *
+ * 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.apache.hadoop.hbase.util;
+
+import java.io.DataInput;
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.regionserver.StoreFile.BloomType;
+
+/**
+ * Handles Bloom filter initialization based on configuration and serialized
+ * metadata in the reader and writer of {@link StoreFile}.
+ */
+public final class BloomFilterFactory {
+
+  private static final Log LOG =
+      LogFactory.getLog(BloomFilterFactory.class.getName());
+
+  /** This class should not be instantiated. */
+  private BloomFilterFactory() {}
+
+  /**
+   * Specifies the target error rate to use when selecting the number of keys
+   * per Bloom filter.
+   */
+  public static final String IO_STOREFILE_BLOOM_ERROR_RATE =
+      "io.storefile.bloom.error.rate";
+
+  /**
+   * Maximum folding factor allowed. The Bloom filter will be shrunk by
+   * the factor of up to 2 ** this times if we oversize it initially.
+   */
+  public static final String IO_STOREFILE_BLOOM_MAX_FOLD =
+      "io.storefile.bloom.max.fold";
+
+  /**
+   * For default (single-block) Bloom filters this specifies the maximum number
+   * of keys.
+   */
+  public static final String IO_STOREFILE_BLOOM_MAX_KEYS =
+      "io.storefile.bloom.max.keys";
+
+  /** Master switch to enable Bloom filters */
+  public static final String IO_STOREFILE_BLOOM_ENABLED =
+      "io.storefile.bloom.enabled";
+
+  /**
+   * Target Bloom block size. Bloom filter blocks of approximately this size
+   * are interleaved with data blocks.
+   */
+  public static final String IO_STOREFILE_BLOOM_BLOCK_SIZE =
+      "io.storefile.bloom.block.size";
+
+  /** Whether to cache compound Bloom filter blocks on write */
+  public static final String IO_STOREFILE_BLOOM_CACHE_ON_WRITE =
+      "io.storefile.bloom.cacheonwrite";
+
+  /** Maximum number of times a Bloom filter can be "folded" if oversized */
+  private static final int MAX_ALLOWED_FOLD_FACTOR = 7;
+
+  /**
+   * Instantiates the correct Bloom filter class based on the version provided
+   * in the meta block data.
+   *
+   * @param meta the byte array holding the Bloom filter's metadata, including
+   *          version information
+   * @param reader the {@link HFile} reader to use to lazily load Bloom filter
+   *          blocks
+   * @return an instance of the correct type of Bloom filter
+   * @throws IllegalArgumentException
+   */
+  public static BloomFilter
+      createFromMeta(DataInput meta, HFile.Reader reader)
+      throws IllegalArgumentException, IOException {
+    int version = meta.readInt();
+    switch (version) {
+      case ByteBloomFilter.VERSION:
+        // This is only possible in a version 1 HFile. We are ignoring the
+        // passed comparator because raw byte comparators are always used
+        // in version 1 Bloom filters.
+        return new ByteBloomFilter(meta);
+
+      case CompoundBloomFilterBase.VERSION:
+        return new CompoundBloomFilter(meta, reader);
+
+      default:
+        throw new IllegalArgumentException(
+          "Bad bloom filter format version " + version
+        );
+    }
+  }
+
+  /**
+   * @return true if Bloom filters are enabled in the given configuration
+   */
+  public static boolean isBloomEnabled(Configuration conf) {
+    return conf.getBoolean(IO_STOREFILE_BLOOM_ENABLED, true);
+  }
+
+  public static float getErrorRate(Configuration conf) {
+    return conf.getFloat(IO_STOREFILE_BLOOM_ERROR_RATE, (float) 0.01);
+  }
+
+  /**
+   * Creates a new Bloom filter at the time of
+   * {@link org.apache.hadoop.hbase.regionserver.StoreFile} writing.
+   *
+   * @param conf
+   * @param bloomType
+   * @param maxKeys an estimate of the number of keys we expect to insert.
+   *        Irrelevant if compound Bloom filters are enabled.
+   * @param writer the HFile writer
+   * @param comparator the comparator to use for compound Bloom filters. This
+   *        has no effect if creating single-chunk version 1 Bloom filters.
+   * @return the new Bloom filter, or null in case Bloom filters are disabled
+   *         or when failed to create one.
+   */
+  public static BloomFilterWriter createBloomAtWrite(Configuration conf,
+      BloomType bloomType, int maxKeys, HFile.Writer writer) {
+    if (!isBloomEnabled(conf)) {
+      LOG.info("Bloom filters are disabled by configuration for "
+          + writer.getPath()
+          + (conf == null ? " (configuration is null)" : ""));
+      return null;
+    } else if (bloomType == BloomType.NONE) {
+      LOG.info("Bloom filter is turned off for the column family");
+      return null;
+    }
+
+    float err = getErrorRate(conf);
+
+    // In case of row/column Bloom filter lookups, each lookup is an OR if two
+    // separate lookups. Therefore, if each lookup's false positive rate is p,
+    // the resulting false positive rate is err = 1 - (1 - p)^2, and
+    // p = 1 - sqrt(1 - err).
+    if (bloomType == BloomType.ROWCOL) {
+      err = (float) (1 - Math.sqrt(1 - err));
+    }
+
+    int maxFold = conf.getInt(IO_STOREFILE_BLOOM_MAX_FOLD,
+        MAX_ALLOWED_FOLD_FACTOR);
+
+    if (HFile.getFormatVersion(conf) > HFile.MIN_FORMAT_VERSION) {
+      // In case of compound Bloom filters we ignore the maxKeys hint.
+      CompoundBloomFilterWriter bloomWriter = new CompoundBloomFilterWriter(
+          getBloomBlockSize(conf), err, Hash.getHashType(conf), maxFold,
+          cacheChunksOnWrite(conf), bloomType == BloomType.ROWCOL
+              ? KeyValue.KEY_COMPARATOR : Bytes.BYTES_RAWCOMPARATOR);
+      writer.addInlineBlockWriter(bloomWriter);
+      return bloomWriter;
+    } else {
+      // A single-block Bloom filter. Only used when testing HFile format
+      // version 1.
+      int tooBig = conf.getInt(IO_STOREFILE_BLOOM_MAX_KEYS,
+          128 * 1000 * 1000);
+
+      if (maxKeys <= 0) {
+        LOG.warn("Invalid maximum number of keys specified: " + maxKeys
+            + ", not using Bloom filter");
+        return null;
+      } else if (maxKeys < tooBig) {
+        BloomFilterWriter bloom = new ByteBloomFilter((int) maxKeys, err,
+            Hash.getHashType(conf), maxFold);
+        bloom.allocBloom();
+        return bloom;
+      } else {
+        LOG.debug("Skipping bloom filter because max keysize too large: "
+            + maxKeys);
+      }
+    }
+    return null;
+  }
+
+  /** @return the compound Bloom filter block size from the configuration */
+  public static int getBloomBlockSize(Configuration conf) {
+    return conf.getInt(IO_STOREFILE_BLOOM_BLOCK_SIZE, 128 * 1024);
+  }
+
+  /** @return whether to cache compound Bloom filter chunks on write */
+  public static boolean cacheChunksOnWrite(Configuration conf) {
+    return conf.getBoolean(IO_STOREFILE_BLOOM_CACHE_ON_WRITE, false);
+  }
+
+};

Added: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/BloomFilterWriter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/BloomFilterWriter.java?rev=1181557&view=auto
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/BloomFilterWriter.java (added)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/BloomFilterWriter.java Tue Oct 11 02:19:54 2011
@@ -0,0 +1,61 @@
+/*
+ * Copyright 2011 The Apache Software Foundation
+ *
+ * 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.apache.hadoop.hbase.util;
+
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Specifies methods needed to add elements to a Bloom filter and serialize the
+ * resulting Bloom filter as a sequence of bytes.
+ */
+public interface BloomFilterWriter extends BloomFilterBase {
+
+  /** Allocate memory for the bloom filter data. */
+  void allocBloom();
+
+  /** Compact the Bloom filter before writing metadata & data to disk. */
+  void compactBloom();
+
+  /**
+   * Get a writable interface into bloom filter meta data.
+   *
+   * @return a writable instance that can be later written to a stream
+   */
+  Writable getMetaWriter();
+
+  /**
+   * Get a writable interface into bloom filter data (the actual Bloom bits).
+   * Not used for compound Bloom filters.
+   *
+   * @return a writable instance that can be later written to a stream
+   */
+  Writable getDataWriter();
+
+  /**
+   * Add the specified binary to the bloom filter.
+   *
+   * @param buf data to be added to the bloom
+   * @param offset offset into the data to be added
+   * @param len length of the data to be added
+   */
+  void add(byte[] buf, int offset, int len);
+
+}

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/ByteBloomFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/ByteBloomFilter.java?rev=1181557&r1=1181556&r2=1181557&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/ByteBloomFilter.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/ByteBloomFilter.java Tue Oct 11 02:19:54 2011
@@ -20,41 +20,71 @@
 
 package org.apache.hadoop.hbase.util;
 
+import org.apache.hadoop.io.RawComparator;
 import org.apache.hadoop.io.Writable;
 
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.text.NumberFormat;
+import java.util.Random;
 
 /**
  * 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.
+ * 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.
  *
  * <p>
- * Originally inspired by
- * <a href="http://www.one-lab.org">European Commission One-Lab Project 034819</a>.
+ * Originally inspired by <a href="http://www.one-lab.org">European Commission
+ * One-Lab Project 034819</a>.
+ *
+ * Bloom filters are very sensitive to the number of elements inserted into
+ * them. For HBase, the number of entries depends on the size of the data stored
+ * in the column. Currently the default region size is 256MB, so entry count ~=
+ * 256MB / (average value size for column). Despite this rule of thumb, there is
+ * no efficient way to calculate the entry count after compactions. Therefore,
+ * it is often easier to use a dynamic bloom filter that will add extra space
+ * instead of allowing the error rate to grow.
+ *
+ * ( http://www.eecs.harvard.edu/~michaelm/NEWWORK/postscripts/BloomFilterSurvey
+ * .pdf )
+ *
+ * m denotes the number of bits in the Bloom filter (bitSize) n denotes the
+ * number of elements inserted into the Bloom filter (maxKeys) k represents the
+ * number of hash functions used (nbHash) e represents the desired false
+ * positive rate for the bloom (err)
+ *
+ * If we fix the error rate (e) and know the number of entries, then the optimal
+ * bloom size m = -(n * ln(err) / (ln(2)^2) ~= n * ln(err) / ln(0.6185)
+ *
+ * The probability of false positives is minimized when k = m/n ln(2).
  *
  * @see BloomFilter 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>
+ * @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 ByteBloomFilter implements BloomFilter {
+public class ByteBloomFilter implements BloomFilter, BloomFilterWriter {
+
   /** Current file format version */
   public static final int VERSION = 1;
 
-  /** Bytes (B) in the array */
+  /** Bytes (B) in the array. This actually has to fit into an int. */
   protected long byteSize;
   /** Number of hash functions */
-  protected final int hashCount;
+  protected int hashCount;
   /** Hash type */
   protected final int hashType;
   /** Hash Function */
@@ -66,6 +96,21 @@ public class ByteBloomFilter implements 
   /** Bloom bits */
   protected ByteBuffer bloom;
 
+  /** Record separator for the Bloom filter statistics human-readable string */
+  public static final String STATS_RECORD_SEP = "; ";
+
+  /**
+   * Used in computing the optimal Bloom filter size. This approximately equals
+   * 0.480453.
+   */
+  public static final double LOG2_SQUARED = Math.log(2) * Math.log(2);
+
+  /**
+   * A random number generator to use for "fake lookups" when testing to
+   * estimate the ideal false positive rate.
+   */
+  private static Random randomGeneratorForTest;
+
   /** Bit-value lookup array to prevent doing the same work over and over */
   private static final byte [] bitvals = {
     (byte) 0x01,
@@ -76,86 +121,209 @@ public class ByteBloomFilter implements 
     (byte) 0x20,
     (byte) 0x40,
     (byte) 0x80
-    };
+  };
 
   /**
    * Loads bloom filter meta data from file input.
    * @param meta stored bloom meta data
    * @throws IllegalArgumentException meta data is invalid
    */
-  public ByteBloomFilter(ByteBuffer meta)
-  throws IllegalArgumentException {
-    int version = meta.getInt();
-    if (version != VERSION) throw new IllegalArgumentException("Bad version");
-
-    this.byteSize = meta.getInt();
-    this.hashCount = meta.getInt();
-    this.hashType = meta.getInt();
-    this.keyCount = meta.getInt();
+  public ByteBloomFilter(DataInput meta)
+      throws IOException, IllegalArgumentException {
+    this.byteSize = meta.readInt();
+    this.hashCount = meta.readInt();
+    this.hashType = meta.readInt();
+    this.keyCount = meta.readInt();
     this.maxKeys = this.keyCount;
 
     this.hash = Hash.getInstance(this.hashType);
+    if (hash == null) {
+      throw new IllegalArgumentException("Invalid hash type: " + hashType);
+    }
     sanityCheck();
   }
 
   /**
-   * Determines & initializes bloom filter meta data from user config.  Call
-   * {@link #allocBloom()} to allocate bloom filter data.
-   * @param maxKeys Maximum expected number of keys that will be stored in this bloom
-   * @param errorRate Desired false positive error rate.  Lower rate = more storage required
-   * @param hashType Type of hash function to use
-   * @param foldFactor When finished adding entries, you may be able to 'fold'
-   * this bloom to save space.  Tradeoff potentially excess bytes in bloom for
-   * ability to fold if keyCount is exponentially greater than maxKeys.
-   * @throws IllegalArgumentException
+   * @param maxKeys
+   * @param errorRate
+   * @return the number of bits for a Bloom filter than can hold the given
+   *         number of keys and provide the given error rate, assuming that the
+   *         optimal number of hash functions is used and it does not have to
+   *         be an integer.
    */
-  public ByteBloomFilter(int maxKeys, float errorRate, int hashType, int foldFactor)
-      throws IllegalArgumentException {
-    /*
-     * Bloom filters are very sensitive to the number of elements inserted
-     * into them. For HBase, the number of entries depends on the size of the
-     * data stored in the column. Currently the default region size is 256MB,
-     * so entry count ~= 256MB / (average value size for column).  Despite
-     * this rule of thumb, there is no efficient way to calculate the entry
-     * count after compactions.  Therefore, it is often easier to use a
-     * dynamic bloom filter that will add extra space instead of allowing the
-     * error rate to grow.
-     *
-     * ( http://www.eecs.harvard.edu/~michaelm/NEWWORK/postscripts/BloomFilterSurvey.pdf )
-     *
-     * m denotes the number of bits in the Bloom filter (bitSize)
-     * n denotes the number of elements inserted into the Bloom filter (maxKeys)
-     * k represents the number of hash functions used (nbHash)
-     * e represents the desired false positive rate for the bloom (err)
-     *
-     * If we fix the error rate (e) and know the number of entries, then
-     * the optimal bloom size m = -(n * ln(err) / (ln(2)^2)
-     *                         ~= n * ln(err) / ln(0.6185)
-     *
-     * The probability of false positives is minimized when k = m/n ln(2).
-     */
-    long bitSize = (long)Math.ceil(maxKeys * (Math.log(errorRate) / Math.log(0.6185)));
-    int functionCount = (int)Math.ceil(Math.log(2) * (bitSize / maxKeys));
+  public static long computeBitSize(long maxKeys, double errorRate) {
+    return (long) Math.ceil(maxKeys * (-Math.log(errorRate) / LOG2_SQUARED));
+  }
 
-    // increase byteSize so folding is possible
-    long byteSize = (bitSize + 7) / 8;
+  /**
+   * The maximum number of keys we can put into a Bloom filter of a certain
+   * size to maintain the given error rate, assuming the number of hash
+   * functions is chosen optimally and does not even have to be an integer
+   * (hence the "ideal" in the function name).
+   *
+   * @param bitSize
+   * @param errorRate
+   * @return maximum number of keys that can be inserted into the Bloom filter
+   * @see {@link #computeMaxKeys(long, double, int)} for a more precise
+   *      estimate
+   */
+  public static long idealMaxKeys(long bitSize, double errorRate) {
+    // The reason we need to use floor here is that otherwise we might put
+    // more keys in a Bloom filter than is allowed by the target error rate.
+    return (long) (bitSize * (LOG2_SQUARED / -Math.log(errorRate)));
+  }
+
+  /**
+   * The maximum number of keys we can put into a Bloom filter of a certain
+   * size to get the given error rate, with the given number of hash functions.
+   *
+   * @param bitSize
+   * @param errorRate
+   * @param hashCount
+   * @return the maximum number of keys that can be inserted in a Bloom filter
+   *         to maintain the target error rate, if the number of hash functions
+   *         is provided.
+   */
+  public static long computeMaxKeys(long bitSize, double errorRate,
+      int hashCount) {
+    return (long) (-bitSize * 1.0 / hashCount *
+        Math.log(1 - Math.exp(Math.log(errorRate) / hashCount)));
+  }
+
+  /**
+   * Computes the error rate for this Bloom filter, taking into account the
+   * actual number of hash functions and keys inserted. The return value of
+   * this function changes as a Bloom filter is being populated. Used for
+   * reporting the actual error rate of compound Bloom filters when writing
+   * them out.
+   *
+   * @return error rate for this particular Bloom filter
+   */
+  public double actualErrorRate() {
+    return actualErrorRate(keyCount, byteSize * 8, hashCount);
+  }
+
+  /**
+   * Computes the actual error rate for the given number of elements, number
+   * of bits, and number of hash functions. Taken directly from the
+   * <a href=
+   * "http://en.wikipedia.org/wiki/Bloom_filter#Probability_of_false_positives"
+   * > Wikipedia Bloom filter article</a>.
+   *
+   * @param maxKeys
+   * @param bitSize
+   * @param functionCount
+   * @return the actual error rate
+   */
+  public static double actualErrorRate(long maxKeys, long bitSize,
+      int functionCount) {
+    return Math.exp(Math.log(1 - Math.exp(-functionCount * maxKeys * 1.0
+        / bitSize)) * functionCount);
+  }
+
+  /**
+   * Increases the given byte size of a Bloom filter until it can be folded by
+   * the given factor.
+   *
+   * @param bitSize
+   * @param foldFactor
+   * @return
+   */
+  public static int computeFoldableByteSize(long bitSize, int foldFactor) {
+    long byteSizeLong = (bitSize + 7) / 8;
     int mask = (1 << foldFactor) - 1;
-    if ( (mask & byteSize) != 0) {
-      byteSize >>= foldFactor;
-      ++byteSize;
-      byteSize <<= foldFactor;
+    if ((mask & byteSizeLong) != 0) {
+      byteSizeLong >>= foldFactor;
+      ++byteSizeLong;
+      byteSizeLong <<= foldFactor;
+    }
+    if (byteSizeLong > Integer.MAX_VALUE) {
+      throw new IllegalArgumentException("byteSize=" + byteSizeLong + " too "
+          + "large for bitSize=" + bitSize + ", foldFactor=" + foldFactor);
     }
+    return (int) byteSizeLong;
+  }
 
-    this.byteSize = byteSize;
-    this.hashCount = functionCount;
+  private static int optimalFunctionCount(int maxKeys, long bitSize) {
+    return (int) Math.ceil(Math.log(2) * (bitSize / maxKeys));
+  }
+
+  /** Private constructor used by other constructors. */
+  private ByteBloomFilter(int hashType) {
     this.hashType = hashType;
-    this.keyCount = 0;
+    this.hash = Hash.getInstance(hashType);
+  }
+
+  /**
+   * Determines & initializes bloom filter meta data from user config. Call
+   * {@link #allocBloom()} to allocate bloom filter data.
+   *
+   * @param maxKeys Maximum expected number of keys that will be stored in this
+   *          bloom
+   * @param errorRate Desired false positive error rate. Lower rate = more
+   *          storage required
+   * @param hashType Type of hash function to use
+   * @param foldFactor When finished adding entries, you may be able to 'fold'
+   *          this bloom to save space. Tradeoff potentially excess bytes in
+   *          bloom for ability to fold if keyCount is exponentially greater
+   *          than maxKeys.
+   * @throws IllegalArgumentException
+   */
+  public ByteBloomFilter(int maxKeys, double errorRate, int hashType,
+      int foldFactor) throws IllegalArgumentException {
+    this(hashType);
+
+    long bitSize = computeBitSize(maxKeys, errorRate);
+    hashCount = optimalFunctionCount(maxKeys, bitSize);
     this.maxKeys = maxKeys;
 
-    this.hash = Hash.getInstance(hashType);
+    // increase byteSize so folding is possible
+    byteSize = computeFoldableByteSize(bitSize, foldFactor);
+
     sanityCheck();
   }
 
+  /**
+   * Creates a Bloom filter of the given size.
+   *
+   * @param byteSizeHint the desired number of bytes for the Bloom filter bit
+   *          array. Will be increased so that folding is possible.
+   * @param errorRate target false positive rate of the Bloom filter
+   * @param hashType Bloom filter hash function type
+   * @param foldFactor
+   * @return the new Bloom filter of the desired size
+   */
+  public static ByteBloomFilter createBySize(int byteSizeHint,
+      double errorRate, int hashType, int foldFactor) {
+    ByteBloomFilter bbf = new ByteBloomFilter(hashType);
+
+    bbf.byteSize = computeFoldableByteSize(byteSizeHint * 8, foldFactor);
+    long bitSize = bbf.byteSize * 8;
+    bbf.maxKeys = (int) idealMaxKeys(bitSize, errorRate);
+    bbf.hashCount = optimalFunctionCount(bbf.maxKeys, bitSize);
+
+    // Adjust max keys to bring error rate closer to what was requested,
+    // because byteSize was adjusted to allow for folding, and hashCount was
+    // rounded.
+    bbf.maxKeys = (int) computeMaxKeys(bitSize, errorRate, bbf.hashCount);
+
+    return bbf;
+  }
+
+  /**
+   * Creates another similar Bloom filter. Does not copy the actual bits, and
+   * sets the new filter's key count to zero.
+   *
+   * @return a Bloom filter with the same configuration as this
+   */
+  public ByteBloomFilter createAnother() {
+    ByteBloomFilter bbf = new ByteBloomFilter(hashType);
+    bbf.byteSize = byteSize;
+    bbf.hashCount = hashCount;
+    bbf.maxKeys = maxKeys;
+    return bbf;
+  }
+
   @Override
   public void allocBloom() {
     if (this.bloom != null) {
@@ -190,7 +358,6 @@ public class ByteBloomFilter implements 
     }
   }
 
-  @Override
   public void add(byte [] buf) {
     add(buf, 0, buf.length);
   }
@@ -212,42 +379,64 @@ public class ByteBloomFilter implements 
     ++this.keyCount;
   }
 
-  /**
-   * Should only be used in tests when writing a bloom filter.
-   */
+  /** Should only be used in tests */
   boolean contains(byte [] buf) {
     return contains(buf, 0, buf.length, this.bloom);
   }
 
-  /**
-   * Should only be used in tests when writing a bloom filter.
-   */
+  /** Should only be used in tests */
   boolean contains(byte [] buf, int offset, int length) {
-    return contains(buf, offset, length, this.bloom);
+    return contains(buf, offset, length, bloom);
   }
 
-  @Override
-  public boolean contains(byte [] buf, ByteBuffer theBloom) {
-    return contains(buf, 0, buf.length, theBloom);
+  /** Should only be used in tests */
+  boolean contains(byte[] buf, ByteBuffer bloom) {
+    return contains(buf, 0, buf.length, bloom);
   }
 
   @Override
-  public boolean contains(byte [] buf, int offset, int length,
+  public boolean contains(byte[] buf, int offset, int length,
       ByteBuffer theBloom) {
+    if (theBloom == null) {
+      // In a version 1 HFile Bloom filter data is stored in a separate meta
+      // block which is loaded on demand, but in version 2 it is pre-loaded.
+      // We want to use the same API in both cases.
+      theBloom = bloom;
+    }
 
-    if(theBloom.limit() != this.byteSize) {
-      throw new IllegalArgumentException("Bloom does not match expected size");
+    if (theBloom.limit() != byteSize) {
+      throw new IllegalArgumentException("Bloom does not match expected size:"
+          + " theBloom.limit()=" + theBloom.limit() + ", byteSize=" + byteSize);
     }
 
-    int hash1 = this.hash.hash(buf, offset, length, 0);
-    int hash2 = this.hash.hash(buf, offset, length, hash1);
+    return contains(buf, offset, length, theBloom.array(),
+        theBloom.arrayOffset(), (int) byteSize, hash, hashCount);
+  }
 
-    for (int i = 0; i < this.hashCount; i++) {
-      long hashLoc = Math.abs((hash1 + i * hash2) % (this.byteSize * 8));
-      if (!get(hashLoc, theBloom) ) {
-        return false;
+  public static boolean contains(byte[] buf, int offset, int length,
+      byte[] bloomArray, int bloomOffset, int bloomSize, Hash hash,
+      int hashCount) {
+
+    int hash1 = hash.hash(buf, offset, length, 0);
+    int hash2 = hash.hash(buf, offset, length, hash1);
+    int bloomBitSize = bloomSize * 8;
+
+    if (randomGeneratorForTest == null) {
+      // Production mode.
+      for (int i = 0; i < hashCount; i++) {
+        long hashLoc = Math.abs((hash1 + i * hash2) % bloomBitSize);
+        if (!get(hashLoc, bloomArray, bloomOffset))
+          return false;
+      }
+    } else {
+      // Test mode with "fake lookups" to estimate "ideal false positive rate".
+      for (int i = 0; i < hashCount; i++) {
+        long hashLoc = randomGeneratorForTest.nextInt(bloomBitSize);
+        if (!get(hashLoc, bloomArray, bloomOffset))
+          return false;
       }
     }
+
     return true;
   }
 
@@ -273,27 +462,31 @@ public class ByteBloomFilter implements 
    * @param pos index of bit
    * @return true if bit at specified index is 1, false if 0.
    */
-  static boolean get(long pos, ByteBuffer theBloom) {
+  static boolean get(long pos, byte[] bloomArray, int bloomOffset) {
     int bytePos = (int)(pos / 8);
     int bitPos = (int)(pos % 8);
-    byte curByte = theBloom.get(bytePos);
+    byte curByte = bloomArray[bloomOffset + bytePos];
     curByte &= bitvals[bitPos];
     return (curByte != 0);
   }
 
   @Override
-  public int getKeyCount() {
-    return this.keyCount;
+  public long getKeyCount() {
+    return keyCount;
   }
 
   @Override
-  public int getMaxKeys() {
-    return this.maxKeys;
+  public long getMaxKeys() {
+    return maxKeys;
   }
 
   @Override
-  public int getByteSize() {
-    return (int)this.byteSize;
+  public long getByteSize() {
+    return byteSize;
+  }
+
+  public int getHashType() {
+    return hashType;
   }
 
   @Override
@@ -367,7 +560,7 @@ public class ByteBloomFilter implements 
     @Override
     public void write(DataOutput out) throws IOException {
       out.writeInt(VERSION);
-      out.writeInt((int)byteSize);
+      out.writeInt((int) byteSize);
       out.writeInt(hashCount);
       out.writeInt(hashType);
       out.writeInt(keyCount);
@@ -387,4 +580,73 @@ public class ByteBloomFilter implements 
     }
   }
 
+  public int getHashCount() {
+    return hashCount;
+  }
+
+  @Override
+  public boolean supportsAutoLoading() {
+    return bloom != null;
+  }
+
+  public static void setFakeLookupMode(boolean enabled) {
+    if (enabled) {
+      randomGeneratorForTest = new Random(283742987L);
+    } else {
+      randomGeneratorForTest = null;
+    }
+  }
+
+  /**
+   * {@inheritDoc}
+   * Just concatenate row and column by default. May return the original row
+   * buffer if the column qualifier is empty.
+   */
+  @Override
+  public byte[] createBloomKey(byte[] rowBuf, int rowOffset, int rowLen,
+      byte[] qualBuf, int qualOffset, int qualLen) {
+    // Optimize the frequent case when only the row is provided.
+    if (qualLen <= 0 && rowOffset == 0 && rowLen == rowBuf.length)
+      return rowBuf;
+
+    byte [] result = new byte[rowLen + qualLen];
+    System.arraycopy(rowBuf, rowOffset, result, 0,  rowLen);
+    if (qualLen > 0)
+      System.arraycopy(qualBuf, qualOffset, result, rowLen, qualLen);
+    return result;
+  }
+
+  @Override
+  public RawComparator<byte[]> getComparator() {
+    return Bytes.BYTES_RAWCOMPARATOR;
+  }
+
+  /**
+   * A human-readable string with statistics for the given Bloom filter.
+   *
+   * @param bloomFilter the Bloom filter to output statistics for;
+   * @return a string consisting of "&lt;key&gt;: &lt;value&gt;" parts
+   *         separated by {@link #STATS_RECORD_SEP}.
+   */
+  public static String formatStats(BloomFilterBase bloomFilter) {
+    StringBuilder sb = new StringBuilder();
+    long k = bloomFilter.getKeyCount();
+    long m = bloomFilter.getMaxKeys();
+
+    sb.append("BloomSize: " + bloomFilter.getByteSize() + STATS_RECORD_SEP);
+    sb.append("No of Keys in bloom: " + k + STATS_RECORD_SEP);
+    sb.append("Max Keys for bloom: " + m);
+    if (m > 0) {
+      sb.append(STATS_RECORD_SEP + "Percentage filled: "
+          + NumberFormat.getPercentInstance().format(k * 1.0 / m));
+    }
+    return sb.toString();
+  }
+
+  @Override
+  public String toString() {
+    return formatStats(this) + STATS_RECORD_SEP + "Actual error rate: "
+        + String.format("%.8f", actualErrorRate());
+  }
+
 }

Added: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilter.java?rev=1181557&view=auto
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilter.java (added)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilter.java Tue Oct 11 02:19:54 2011
@@ -0,0 +1,175 @@
+/*
+ * Copyright 2011 The Apache Software Foundation
+ *
+ * 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.apache.hadoop.hbase.util;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.io.hfile.FixedFileTrailer;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileBlock;
+import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex;
+import org.apache.hadoop.io.RawComparator;
+
+/**
+ * A Bloom filter implementation built on top of {@link ByteBloomFilter},
+ * encapsulating a set of fixed-size Bloom filters written out at the time of
+ * {@link org.apache.hadoop.hbase.io.hfile.HFile} generation into the data
+ * block stream, and loaded on demand at query time. This class only provides
+ * reading capabilities.
+ */
+public class CompoundBloomFilter extends CompoundBloomFilterBase
+    implements BloomFilter {
+
+  /** Used to load chunks on demand */
+  private HFile.Reader reader;
+
+  private HFileBlockIndex.BlockIndexReader index;
+
+  private int hashCount;
+  private Hash hash;
+
+  private long[] numQueriesPerChunk;
+  private long[] numPositivesPerChunk;
+
+  /**
+   * De-serialization for compound Bloom filter metadata. Must be consistent
+   * with what {@link CompoundBloomFilterWriter} does.
+   *
+   * @param meta serialized Bloom filter metadata without any magic blocks
+   * @throws IOException
+   */
+  public CompoundBloomFilter(DataInput meta, HFile.Reader reader)
+      throws IOException {
+    this.reader = reader;
+
+    totalByteSize = meta.readLong();
+    hashCount = meta.readInt();
+    hashType = meta.readInt();
+    totalKeyCount = meta.readLong();
+    totalMaxKeys = meta.readLong();
+    numChunks = meta.readInt();
+    comparator = FixedFileTrailer.createComparator(
+        Bytes.toString(Bytes.readByteArray(meta)));
+
+    hash = Hash.getInstance(hashType);
+    if (hash == null) {
+      throw new IllegalArgumentException("Invalid hash type: " + hashType);
+    }
+
+    index = new HFileBlockIndex.BlockIndexReader(comparator, 1);
+    index.readRootIndex(meta, numChunks);
+  }
+
+  @Override
+  public boolean contains(byte[] key, int keyOffset, int keyLength,
+      ByteBuffer bloom) {
+    // We try to store the result in this variable so we can update stats for
+    // testing, but when an error happens, we log a message and return.
+    boolean result;
+
+    int block = index.rootBlockContainingKey(key, keyOffset, keyLength);
+    if (block < 0) {
+      result = false; // This key is not in the file.
+    } else {
+      HFileBlock bloomBlock;
+      try {
+        // We cache the block and use a positional read.
+        bloomBlock = reader.readBlock(index.getRootBlockOffset(block),
+            index.getRootBlockDataSize(block), true, true, false);
+      } catch (IOException ex) {
+        // The Bloom filter is broken, turn it off.
+        throw new IllegalArgumentException(
+            "Failed to load Bloom block for key "
+                + Bytes.toStringBinary(key, keyOffset, keyLength), ex);
+      }
+
+      ByteBuffer bloomBuf = bloomBlock.getBufferReadOnly();
+      result = ByteBloomFilter.contains(key, keyOffset, keyLength,
+          bloomBuf.array(), bloomBuf.arrayOffset() + HFileBlock.HEADER_SIZE,
+          bloomBlock.getUncompressedSizeWithoutHeader(), hash, hashCount);
+    }
+
+    if (numQueriesPerChunk != null && block >= 0) {
+      // Update statistics. Only used in unit tests.
+      ++numQueriesPerChunk[block];
+      if (result)
+        ++numPositivesPerChunk[block];
+    }
+
+    return result;
+  }
+
+  public boolean supportsAutoLoading() {
+    return true;
+  }
+
+  public int getNumChunks() {
+    return numChunks;
+  }
+
+  @Override
+  public RawComparator<byte[]> getComparator() {
+    return comparator;
+  }
+
+  public void enableTestingStats() {
+    numQueriesPerChunk = new long[numChunks];
+    numPositivesPerChunk = new long[numChunks];
+  }
+
+  public String formatTestingStats() {
+    StringBuilder sb = new StringBuilder();
+    for (int i = 0; i < numChunks; ++i) {
+      sb.append("chunk #");
+      sb.append(i);
+      sb.append(": queries=");
+      sb.append(numQueriesPerChunk[i]);
+      sb.append(", positives=");
+      sb.append(numPositivesPerChunk[i]);
+      sb.append(", positiveRatio=");
+      sb.append(numPositivesPerChunk[i] * 1.0 / numQueriesPerChunk[i]);
+      sb.append(";\n");
+    }
+    return sb.toString();
+  }
+
+  public long getNumQueriesForTesting(int chunk) {
+    return numQueriesPerChunk[chunk];
+  }
+
+  public long getNumPositivesForTesting(int chunk) {
+    return numPositivesPerChunk[chunk];
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append(ByteBloomFilter.formatStats(this));
+    sb.append(ByteBloomFilter.STATS_RECORD_SEP +
+        "Number of chunks: " + numChunks);
+    sb.append(ByteBloomFilter.STATS_RECORD_SEP +
+        "Comparator: " + comparator.getClass().getSimpleName());
+    return sb.toString();
+  }
+
+}

Added: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterBase.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterBase.java?rev=1181557&view=auto
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterBase.java (added)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterBase.java Tue Oct 11 02:19:54 2011
@@ -0,0 +1,75 @@
+package org.apache.hadoop.hbase.util;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.io.RawComparator;
+
+public class CompoundBloomFilterBase implements BloomFilterBase {
+
+  /**
+   * At read time, the total number of chunks. At write time, the number of
+   * chunks created so far. The first chunk has an ID of 0, and the current
+   * chunk has the ID of numChunks - 1.
+   */
+  protected int numChunks;
+
+  /**
+   * The Bloom filter version. There used to be a DynamicByteBloomFilter which
+   * had version 2.
+   */
+  public static final int VERSION = 3;
+
+  /** Target error rate for configuring the filter and for information */
+  protected float errorRate;
+
+  /** The total number of keys in all chunks */
+  protected long totalKeyCount;
+  protected long totalByteSize;
+  protected long totalMaxKeys;
+
+  /** Hash function type to use, as defined in {@link Hash} */
+  protected int hashType;
+
+  /** Comparator used to compare Bloom filter keys */
+  protected RawComparator<byte[]> comparator;
+
+  @Override
+  public long getMaxKeys() {
+    return totalMaxKeys;
+  }
+
+  @Override
+  public long getKeyCount() {
+    return totalKeyCount;
+  }
+
+  @Override
+  public long getByteSize() {
+    return totalByteSize;
+  }
+
+  private static final byte[] DUMMY = new byte[0];
+
+  /**
+   * Prepare an ordered pair of row and qualifier to be compared using
+   * {@link KeyValue.KeyComparator}. This is only used for row-column Bloom
+   * filters.
+   */
+  @Override
+  public byte[] createBloomKey(byte[] row, int roffset, int rlength,
+      byte[] qualifier, int qoffset, int qlength) {
+    if (qualifier == null)
+      qualifier = DUMMY;
+
+    // Make sure this does not specify a timestamp so that the default maximum
+    // (most recent) timestamp is used.
+    KeyValue kv = KeyValue.createFirstOnRow(row, roffset, rlength, DUMMY, 0, 0,
+        qualifier, qoffset, qlength);
+    return kv.getKey();
+  }
+
+  @Override
+  public RawComparator<byte[]> getComparator() {
+    return comparator;
+  }
+
+}