001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.util;
019
020import java.io.DataInput;
021import java.io.IOException;
022
023import org.apache.hadoop.conf.Configuration;
024import org.apache.hadoop.hbase.CellComparatorImpl;
025import org.apache.hadoop.hbase.io.hfile.CacheConfig;
026import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilter;
027import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilterBase;
028import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilterWriter;
029import org.apache.hadoop.hbase.io.hfile.HFile;
030import org.apache.hadoop.hbase.regionserver.BloomType;
031import org.apache.yetus.audience.InterfaceAudience;
032import org.slf4j.Logger;
033import org.slf4j.LoggerFactory;
034
035/**
036 * Handles Bloom filter initialization based on configuration and serialized metadata in the reader
037 * and writer of {@link org.apache.hadoop.hbase.regionserver.HStoreFile}.
038 */
039@InterfaceAudience.Private
040public final class BloomFilterFactory {
041
042  private static final Logger LOG =
043      LoggerFactory.getLogger(BloomFilterFactory.class.getName());
044
045  /** This class should not be instantiated. */
046  private BloomFilterFactory() {}
047
048  /**
049   * Specifies the target error rate to use when selecting the number of keys
050   * per Bloom filter.
051   */
052  public static final String IO_STOREFILE_BLOOM_ERROR_RATE =
053      "io.storefile.bloom.error.rate";
054
055  /**
056   * Maximum folding factor allowed. The Bloom filter will be shrunk by
057   * the factor of up to 2 ** this times if we oversize it initially.
058   */
059  public static final String IO_STOREFILE_BLOOM_MAX_FOLD =
060      "io.storefile.bloom.max.fold";
061
062  /**
063   * For default (single-block) Bloom filters this specifies the maximum number
064   * of keys.
065   */
066  public static final String IO_STOREFILE_BLOOM_MAX_KEYS =
067      "io.storefile.bloom.max.keys";
068
069  /** Master switch to enable Bloom filters */
070  public static final String IO_STOREFILE_BLOOM_ENABLED =
071      "io.storefile.bloom.enabled";
072
073  /** Master switch to enable Delete Family Bloom filters */
074  public static final String IO_STOREFILE_DELETEFAMILY_BLOOM_ENABLED =
075      "io.storefile.delete.family.bloom.enabled";
076
077  /**
078   * Target Bloom block size. Bloom filter blocks of approximately this size
079   * are interleaved with data blocks.
080   */
081  public static final String IO_STOREFILE_BLOOM_BLOCK_SIZE =
082      "io.storefile.bloom.block.size";
083
084  /** Maximum number of times a Bloom filter can be "folded" if oversized */
085  private static final int MAX_ALLOWED_FOLD_FACTOR = 7;
086
087  /**
088   * Instantiates the correct Bloom filter class based on the version provided
089   * in the meta block data.
090   *
091   * @param meta the byte array holding the Bloom filter's metadata, including
092   *          version information
093   * @param reader the {@link HFile} reader to use to lazily load Bloom filter
094   *          blocks
095   * @return an instance of the correct type of Bloom filter
096   * @throws IllegalArgumentException
097   */
098  public static BloomFilter
099      createFromMeta(DataInput meta, HFile.Reader reader)
100      throws IllegalArgumentException, IOException {
101    int version = meta.readInt();
102    switch (version) {
103      case CompoundBloomFilterBase.VERSION:
104        return new CompoundBloomFilter(meta, reader);
105
106      default:
107        throw new IllegalArgumentException(
108          "Bad bloom filter format version " + version
109        );
110    }
111  }
112
113  /**
114   * @return true if general Bloom (Row or RowCol) filters are enabled in the
115   * given configuration
116   */
117  public static boolean isGeneralBloomEnabled(Configuration conf) {
118    return conf.getBoolean(IO_STOREFILE_BLOOM_ENABLED, true);
119  }
120
121  /**
122   * @return true if Delete Family Bloom filters are enabled in the given configuration
123   */
124  public static boolean isDeleteFamilyBloomEnabled(Configuration conf) {
125    return conf.getBoolean(IO_STOREFILE_DELETEFAMILY_BLOOM_ENABLED, true);
126  }
127
128  /**
129   * @return the Bloom filter error rate in the given configuration
130   */
131  public static float getErrorRate(Configuration conf) {
132    return conf.getFloat(IO_STOREFILE_BLOOM_ERROR_RATE, (float) 0.01);
133  }
134
135  /**
136   * @return the value for Bloom filter max fold in the given configuration
137   */
138  public static int getMaxFold(Configuration conf) {
139    return conf.getInt(IO_STOREFILE_BLOOM_MAX_FOLD, MAX_ALLOWED_FOLD_FACTOR);
140  }
141
142  /** @return the compound Bloom filter block size from the configuration */
143  public static int getBloomBlockSize(Configuration conf) {
144    return conf.getInt(IO_STOREFILE_BLOOM_BLOCK_SIZE, 128 * 1024);
145  }
146
147  /**
148  * @return max key for the Bloom filter from the configuration
149  */
150  public static int getMaxKeys(Configuration conf) {
151    return conf.getInt(IO_STOREFILE_BLOOM_MAX_KEYS, 128 * 1000 * 1000);
152  }
153
154  /**
155   * Creates a new general (Row or RowCol) Bloom filter at the time of
156   * {@link org.apache.hadoop.hbase.regionserver.HStoreFile} writing.
157   *
158   * @param conf
159   * @param cacheConf
160   * @param bloomType
161   * @param maxKeys an estimate of the number of keys we expect to insert.
162   *        Irrelevant if compound Bloom filters are enabled.
163   * @param writer the HFile writer
164   * @return the new Bloom filter, or null in case Bloom filters are disabled
165   *         or when failed to create one.
166   */
167  public static BloomFilterWriter createGeneralBloomAtWrite(Configuration conf,
168      CacheConfig cacheConf, BloomType bloomType, int maxKeys,
169      HFile.Writer writer) {
170    if (!isGeneralBloomEnabled(conf)) {
171      LOG.trace("Bloom filters are disabled by configuration for "
172          + writer.getPath()
173          + (conf == null ? " (configuration is null)" : ""));
174      return null;
175    } else if (bloomType == BloomType.NONE) {
176      LOG.trace("Bloom filter is turned off for the column family");
177      return null;
178    }
179
180    float err = getErrorRate(conf);
181
182    // In case of row/column Bloom filter lookups, each lookup is an OR if two
183    // separate lookups. Therefore, if each lookup's false positive rate is p,
184    // the resulting false positive rate is err = 1 - (1 - p)^2, and
185    // p = 1 - sqrt(1 - err).
186    if (bloomType == BloomType.ROWCOL) {
187      err = (float) (1 - Math.sqrt(1 - err));
188    }
189
190    int maxFold = conf.getInt(IO_STOREFILE_BLOOM_MAX_FOLD,
191        MAX_ALLOWED_FOLD_FACTOR);
192
193    // Do we support compound bloom filters?
194    // In case of compound Bloom filters we ignore the maxKeys hint.
195    CompoundBloomFilterWriter bloomWriter = new CompoundBloomFilterWriter(getBloomBlockSize(conf),
196        err, Hash.getHashType(conf), maxFold, cacheConf.shouldCacheBloomsOnWrite(),
197        bloomType == BloomType.ROWCOL ? CellComparatorImpl.COMPARATOR : null, bloomType);
198    writer.addInlineBlockWriter(bloomWriter);
199    return bloomWriter;
200  }
201
202  /**
203   * Creates a new Delete Family Bloom filter at the time of
204   * {@link org.apache.hadoop.hbase.regionserver.HStoreFile} writing.
205   * @param conf
206   * @param cacheConf
207   * @param maxKeys an estimate of the number of keys we expect to insert.
208   *        Irrelevant if compound Bloom filters are enabled.
209   * @param writer the HFile writer
210   * @return the new Bloom filter, or null in case Bloom filters are disabled
211   *         or when failed to create one.
212   */
213  public static BloomFilterWriter createDeleteBloomAtWrite(Configuration conf,
214      CacheConfig cacheConf, int maxKeys, HFile.Writer writer) {
215    if (!isDeleteFamilyBloomEnabled(conf)) {
216      LOG.info("Delete Bloom filters are disabled by configuration for "
217          + writer.getPath()
218          + (conf == null ? " (configuration is null)" : ""));
219      return null;
220    }
221
222    float err = getErrorRate(conf);
223
224    int maxFold = getMaxFold(conf);
225    // In case of compound Bloom filters we ignore the maxKeys hint.
226    CompoundBloomFilterWriter bloomWriter = new CompoundBloomFilterWriter(getBloomBlockSize(conf),
227        err, Hash.getHashType(conf), maxFold, cacheConf.shouldCacheBloomsOnWrite(),
228        null, BloomType.ROW);
229    writer.addInlineBlockWriter(bloomWriter);
230    return bloomWriter;
231  }
232};