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:23:03 UTC

svn commit: r1181588 - in /hbase/branches/0.89/src/main: java/org/apache/hadoop/hbase/ java/org/apache/hadoop/hbase/mapreduce/ java/org/apache/hadoop/hbase/regionserver/ java/org/apache/hadoop/hbase/util/ ruby/ ruby/hbase/

Author: nspiegelberg
Date: Tue Oct 11 02:23:02 2011
New Revision: 1181588

URL: http://svn.apache.org/viewvc?rev=1181588&view=rev
Log:
Per-CF bloom error rate

Summary:
Move BloomFilter error rate to the ColumnFamily config so we can play with this
number in the new schema

Test Plan:
Tested using hbase shell for create table, alter table, put and scan.
Also tested to check if the existing functionality does not break.

Reviewed By: nspiegelberg
Reviewers: nspiegelberg, kannan
Commenters: kannan, pkhemani
CC: hbase@lists, nspiegelberg, nileema, kannan, pkhemani
Differential Revision: 274513
Task ID: 571268

Modified:
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
    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/BloomFilterFactory.java
    hbase/branches/0.89/src/main/ruby/hbase.rb
    hbase/branches/0.89/src/main/ruby/hbase/admin.rb

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java?rev=1181588&r1=1181587&r2=1181588&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java Tue Oct 11 02:23:02 2011
@@ -77,6 +77,7 @@ public class HColumnDescriptor implement
   public static final String LENGTH = "LENGTH";
   public static final String TTL = "TTL";
   public static final String BLOOMFILTER = "BLOOMFILTER";
+  public static final String BLOOMFILTER_ERRORRATE = "BLOOMFILTER_ERRORRATE";
   public static final String FOREVER = "FOREVER";
   public static final String REPLICATION_SCOPE = "REPLICATION_SCOPE";
 
@@ -119,6 +120,12 @@ public class HColumnDescriptor implement
   public static final String DEFAULT_BLOOMFILTER = StoreFile.BloomType.NONE.toString();
 
   /**
+   * Default value for bloom filter error rate.
+   */
+  public static final float DEFAULT_BLOOMFILTER_ERROR_RATE = 0.01f;
+
+
+  /**
    * Default time to live of cell contents.
    */
   public static final int DEFAULT_TTL = HConstants.FOREVER;
@@ -205,6 +212,7 @@ public class HColumnDescriptor implement
    * a <code>:</code>
    * @throws IllegalArgumentException if the number of versions is &lt;= 0
    */
+
   public HColumnDescriptor(final byte [] familyName, final int maxVersions,
       final String compression, final boolean inMemory,
       final boolean blockCacheEnabled,
@@ -213,6 +221,14 @@ public class HColumnDescriptor implement
       DEFAULT_BLOCKSIZE, timeToLive, bloomFilter, DEFAULT_REPLICATION_SCOPE);
   }
 
+  public HColumnDescriptor(final byte [] familyName, final int maxVersions,
+	      final String compression, final boolean inMemory,
+	      final boolean blockCacheEnabled, final int blocksize,
+	      final int timeToLive, final String bloomFilter, final int scope) {
+
+	  this(familyName, maxVersions, compression, inMemory, blockCacheEnabled,
+			  blocksize, timeToLive, bloomFilter, scope, DEFAULT_BLOOMFILTER_ERROR_RATE);
+   }
   /**
    * Constructor
    * @param familyName Column family name. Must be 'printable' -- digit or
@@ -227,16 +243,17 @@ public class HColumnDescriptor implement
    * (use HConstants.FOREVER for unlimited TTL)
    * @param bloomFilter Bloom filter type for this column
    * @param scope The scope tag for this column
-   *
+   * @param bloomErrorRate Bloom filter error rate for this column
    * @throws IllegalArgumentException if passed a family name that is made of
    * other than 'word' characters: i.e. <code>[a-zA-Z_0-9]</code> or contains
    * a <code>:</code>
    * @throws IllegalArgumentException if the number of versions is &lt;= 0
    */
+
   public HColumnDescriptor(final byte [] familyName, final int maxVersions,
       final String compression, final boolean inMemory,
       final boolean blockCacheEnabled, final int blocksize,
-      final int timeToLive, final String bloomFilter, final int scope) {
+      final int timeToLive, final String bloomFilter, final int scope, float bloomErrorRate) {
     isLegalFamilyName(familyName);
     this.name = familyName;
 
@@ -253,6 +270,7 @@ public class HColumnDescriptor implement
       valueOf(compression.toUpperCase()));
     setBloomFilterType(StoreFile.BloomType.
       valueOf(bloomFilter.toUpperCase()));
+    setBloomFilterErrorRate(bloomErrorRate);
     setBlocksize(blocksize);
     setScope(scope);
   }
@@ -485,6 +503,14 @@ public class HColumnDescriptor implement
     setValue(BLOOMFILTER, bt.toString());
   }
 
+   public void setBloomFilterErrorRate(float bloomErrorRate) {
+	   setValue(BLOOMFILTER_ERRORRATE, Float.toString(bloomErrorRate));
+   }
+   public float getBloomFilterErrorRate() {
+	    String value = getValue(BLOOMFILTER_ERRORRATE);
+	    return (value != null)? Float.valueOf(value).floatValue() : DEFAULT_BLOOMFILTER_ERROR_RATE;
+	  }
+
    /**
     * @return the scope tag
     */

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java?rev=1181588&r1=1181587&r2=1181588&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java Tue Oct 11 02:23:02 2011
@@ -538,7 +538,7 @@ public class HTableDescriptor implements
   // Writable
 
   public void readFields(DataInput in) throws IOException {
-    int version = in.readInt();
+	int version = in.readInt();
     if (version < 3)
       throw new IOException("versions < 3 are not supported (and never existed!?)");
     // version 3+

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java?rev=1181588&r1=1181587&r2=1181588&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java Tue Oct 11 02:23:02 2011
@@ -274,9 +274,10 @@ public class LoadIncrementalHFiles exten
       Algorithm compression = familyDescriptor.getCompression();
       BloomType bloomFilterType = familyDescriptor.getBloomFilterType();
 
+      float err = familyDescriptor.getBloomFilterErrorRate();
       halfWriter = new StoreFile.Writer(
-          fs, outFile, blocksize, compression, conf, KeyValue.COMPARATOR,
-          bloomFilterType, 0);
+          fs, outFile, blocksize, compression,  conf, KeyValue.COMPARATOR,
+          bloomFilterType, err, 0);
       HFileScanner scanner = halfReader.getScanner(false, false, false);
       scanner.seekTo();
       do {

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java?rev=1181588&r1=1181587&r2=1181588&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java Tue Oct 11 02:23:02 2011
@@ -523,7 +523,7 @@ public class Store implements HeapSize {
   throws IOException {
     return StoreFile.createWriter(this.fs, region.getTmpDir(), this.blocksize,
         this.compression, this.comparator, this.conf,
-        this.family.getBloomFilterType(), maxKeyCount);
+        this.family.getBloomFilterType(), this.family.getBloomFilterErrorRate(), maxKeyCount);
   }
 
   /*

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=1181588&r1=1181587&r2=1181588&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:23:02 2011
@@ -49,10 +49,12 @@ import org.apache.hadoop.hbase.io.HalfSt
 import org.apache.hadoop.hbase.io.Reference;
 import org.apache.hadoop.hbase.io.hfile.BlockCache;
 import org.apache.hadoop.hbase.io.hfile.Compression;
+import org.apache.hadoop.hbase.io.hfile.Compression.Algorithm;
 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.regionserver.StoreFile.BloomType;
 import org.apache.hadoop.hbase.util.BloomFilter;
 import org.apache.hadoop.hbase.util.BloomFilterFactory;
 import org.apache.hadoop.hbase.util.BloomFilterWriter;
@@ -553,6 +555,19 @@ public class StoreFile {
     return createWriter(fs, dir, blocksize, null, null, conf, BloomType.NONE,
         0);
   }
+  public static StoreFile.Writer createWriter(final FileSystem fs,
+          final Path dir,
+          final int blocksize,
+          final Compression.Algorithm algorithm,
+          final KeyValue.KVComparator c,
+          final Configuration conf,
+          BloomType bloomType,
+          long maxKeyCount)
+  throws IOException {
+	  return createWriter(fs, dir, blocksize, null, null, conf, BloomType.NONE,
+			  BloomFilterFactory.getErrorRate(conf), 0);
+
+  }
 
   /**
    * Create a store file writer. Client is responsible for closing file when done.
@@ -565,6 +580,7 @@ public class StoreFile {
    * @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 bloomErrorRate column family setting for bloom filter error rate
    * @param maxKeyCount estimated maximum number of keys we expect to add
    * @return HFile.Writer
    * @throws IOException
@@ -576,6 +592,7 @@ public class StoreFile {
                                               final KeyValue.KVComparator c,
                                               final Configuration conf,
                                               BloomType bloomType,
+                                              float bloomErrorRate,
                                               long maxKeyCount)
       throws IOException {
 
@@ -589,7 +606,7 @@ public class StoreFile {
 
     return new Writer(fs, path, blocksize,
         algorithm == null? HFile.DEFAULT_COMPRESSION_ALGORITHM: algorithm,
-        conf, c == null ? KeyValue.COMPARATOR: c, bloomType, maxKeyCount);
+        conf, c == null ? KeyValue.COMPARATOR: c, bloomType, bloomErrorRate, maxKeyCount);
   }
 
   /**
@@ -690,6 +707,15 @@ public class StoreFile {
     boolean isTimeRangeTrackerSet = false;
 
     protected HFile.Writer writer;
+
+    public Writer(FileSystem fs, Path path, int blocksize,
+            Compression.Algorithm compress, final Configuration conf,
+            final KVComparator comparator, BloomType bloomType,  long maxKeys)
+            throws IOException {
+	this(fs, path, blocksize, compress, conf, comparator, bloomType,
+			BloomFilterFactory.getErrorRate(conf), maxKeys);
+    }
+
     /**
      * Creates an HFile.Writer that also write helpful meta data.
      * @param fs file system to write to
@@ -699,13 +725,14 @@ public class StoreFile {
      * @param conf user configuration
      * @param comparator key comparator
      * @param bloomType bloom filter setting
+     * @param bloomErrorRate error rate for bloom filter
      * @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)
+        final KVComparator comparator, BloomType bloomType, float bloomErrorRate, long maxKeys)
         throws IOException {
 
       writer = HFile.getWriterFactory(conf).createWriter(
@@ -715,7 +742,7 @@ public class StoreFile {
       this.kvComparator = comparator;
 
       bloomFilterWriter = BloomFilterFactory.createBloomAtWrite(conf,
-          bloomType, (int) Math.min(maxKeys, Integer.MAX_VALUE), writer);
+          bloomType, (int) Math.min(maxKeys, Integer.MAX_VALUE), writer, bloomErrorRate);
       if (bloomFilterWriter != null) {
         this.bloomType = bloomType;
         LOG.info("Bloom filter type for " + path + ": " + this.bloomType +
@@ -726,7 +753,8 @@ public class StoreFile {
       }
     }
 
-    /**
+
+	/**
      * Writes meta data.
      * Call before {@link #close()} since its written as meta data to this file.
      * @param maxSequenceId Maximum sequence id.

Modified: 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=1181588&r1=1181587&r2=1181588&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java Tue Oct 11 02:23:02 2011
@@ -181,7 +181,7 @@ public final class BloomFilterFactory {
    *         or when failed to create one.
    */
   public static BloomFilterWriter createBloomAtWrite(Configuration conf,
-      BloomType bloomType, int maxKeys, HFile.Writer writer) {
+      BloomType bloomType, int maxKeys, HFile.Writer writer, float bloomErrorRate) {
     if (!isBloomEnabled(conf)) {
       LOG.info("Bloom filters are disabled by configuration for "
           + writer.getPath()
@@ -192,14 +192,12 @@ public final class BloomFilterFactory {
       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));
+      bloomErrorRate = (float) (1 - Math.sqrt(1 - bloomErrorRate));
     }
 
     int maxFold = getMaxFold(conf);
@@ -207,7 +205,7 @@ public final class BloomFilterFactory {
     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,
+          getBloomBlockSize(conf), bloomErrorRate, Hash.getHashType(conf), maxFold,
           cacheChunksOnWrite(conf), bloomType == BloomType.ROWCOL
               ? KeyValue.KEY_COMPARATOR : Bytes.BYTES_RAWCOMPARATOR);
       writer.addInlineBlockWriter(bloomWriter);
@@ -222,7 +220,7 @@ public final class BloomFilterFactory {
             + ", not using Bloom filter");
         return null;
       } else if (maxKeys < tooBig) {
-        BloomFilterWriter bloom = new ByteBloomFilter((int) maxKeys, err,
+        BloomFilterWriter bloom = new ByteBloomFilter((int) maxKeys, bloomErrorRate,
             Hash.getHashType(conf), maxFold);
         bloom.allocBloom();
         return bloom;

Modified: hbase/branches/0.89/src/main/ruby/hbase.rb
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/ruby/hbase.rb?rev=1181588&r1=1181587&r2=1181588&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/ruby/hbase.rb (original)
+++ hbase/branches/0.89/src/main/ruby/hbase.rb Tue Oct 11 02:23:02 2011
@@ -35,6 +35,7 @@ java_import org.apache.hadoop.hbase.HTab
 include_class('java.lang.Integer') {|package,name| "J#{name}" }
 include_class('java.lang.Long') {|package,name| "J#{name}" }
 include_class('java.lang.Boolean') {|package,name| "J#{name}" }
+include_class('java.lang.Float') {|package,name| "J#{name}" }
 
 module HBaseConstants
   COLUMN = "COLUMN"

Modified: hbase/branches/0.89/src/main/ruby/hbase/admin.rb
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/ruby/hbase/admin.rb?rev=1181588&r1=1181587&r2=1181588&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/ruby/hbase/admin.rb (original)
+++ hbase/branches/0.89/src/main/ruby/hbase/admin.rb Tue Oct 11 02:23:02 2011
@@ -344,6 +344,7 @@ module Hbase
 
       family.setBlockCacheEnabled(JBoolean.valueOf(arg[HColumnDescriptor::BLOCKCACHE])) if arg.include?(HColumnDescriptor::BLOCKCACHE)
       family.setBloomFilterType(StoreFile::BloomType.valueOf(arg[HColumnDescriptor::BLOOMFILTER])) if arg.include?(HColumnDescriptor::BLOOMFILTER)
+      family.setBloomFilterErrorRate(JFloat.valueOf(arg[HColumnDescriptor::BLOOMFILTER_ERRORRATE])) if arg.include?(HColumnDescriptor::BLOOMFILTER_ERRORRATE)
       family.setScope(JInteger.valueOf(arg[REPLICATION_SCOPE])) if arg.include?(HColumnDescriptor::REPLICATION_SCOPE)
       family.setInMemory(JBoolean.valueOf(arg[IN_MEMORY])) if arg.include?(HColumnDescriptor::IN_MEMORY)
       family.setTimeToLive(JInteger.valueOf(arg[HColumnDescriptor::TTL])) if arg.include?(HColumnDescriptor::TTL)