001/*
002 *
003 * Licensed to the Apache Software Foundation (ASF) under one
004 * or more contributor license agreements.  See the NOTICE file
005 * distributed with this work for additional information
006 * regarding copyright ownership.  The ASF licenses this file
007 * to you under the Apache License, Version 2.0 (the
008 * "License"); you may not use this file except in compliance
009 * with the License.  You may obtain a copy of the License at
010 *
011 *     http://www.apache.org/licenses/LICENSE-2.0
012 *
013 * Unless required by applicable law or agreed to in writing, software
014 * distributed under the License is distributed on an "AS IS" BASIS,
015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
016 * See the License for the specific language governing permissions and
017 * limitations under the License.
018 */
019
020package org.apache.hadoop.hbase.util;
021
022import java.io.DataInput;
023import java.io.DataOutput;
024import java.io.IOException;
025import java.nio.ByteBuffer;
026
027import org.apache.hadoop.hbase.Cell;
028import org.apache.yetus.audience.InterfaceAudience;
029import org.apache.hadoop.hbase.regionserver.BloomType;
030
031/**
032 * The basic building block for the {@link org.apache.hadoop.hbase.io.hfile.CompoundBloomFilter}
033 */
034@InterfaceAudience.Private
035public class BloomFilterChunk implements BloomFilterBase {
036
037  /** Bytes (B) in the array. This actually has to fit into an int. */
038  protected long byteSize;
039  /** Number of hash functions */
040  protected int hashCount;
041  /** Hash type */
042  protected final int hashType;
043  /** Hash Function */
044  protected final Hash hash;
045  /** Keys currently in the bloom */
046  protected int keyCount;
047  /** Max Keys expected for the bloom */
048  protected int maxKeys;
049  /** Bloom bits */
050  protected ByteBuffer bloom;
051  /** The type of bloom */
052  protected BloomType bloomType;
053
054  /**
055   * Loads bloom filter meta data from file input.
056   * @param meta stored bloom meta data
057   * @throws IllegalArgumentException meta data is invalid
058   */
059  public BloomFilterChunk(DataInput meta)
060      throws IOException, IllegalArgumentException {
061    this.byteSize = meta.readInt();
062    this.hashCount = meta.readInt();
063    this.hashType = meta.readInt();
064    this.keyCount = meta.readInt();
065    this.maxKeys = this.keyCount;
066
067    this.hash = Hash.getInstance(this.hashType);
068    if (hash == null) {
069      throw new IllegalArgumentException("Invalid hash type: " + hashType);
070    }
071    sanityCheck();
072  }
073
074  /**
075   * Computes the error rate for this Bloom filter, taking into account the
076   * actual number of hash functions and keys inserted. The return value of
077   * this function changes as a Bloom filter is being populated. Used for
078   * reporting the actual error rate of compound Bloom filters when writing
079   * them out.
080   *
081   * @return error rate for this particular Bloom filter
082   */
083  public double actualErrorRate() {
084    return BloomFilterUtil.actualErrorRate(keyCount, byteSize * 8, hashCount);
085  }
086
087  public BloomFilterChunk(int hashType, BloomType bloomType) {
088    this.hashType = hashType;
089    this.hash = Hash.getInstance(hashType);
090    this.bloomType = bloomType;
091  }
092
093  /**
094   * Determines & initializes bloom filter meta data from user config. Call
095   * {@link #allocBloom()} to allocate bloom filter data.
096   *
097   * @param maxKeys Maximum expected number of keys that will be stored in this
098   *          bloom
099   * @param errorRate Desired false positive error rate. Lower rate = more
100   *          storage required
101   * @param hashType Type of hash function to use
102   * @param foldFactor When finished adding entries, you may be able to 'fold'
103   *          this bloom to save space. Tradeoff potentially excess bytes in
104   *          bloom for ability to fold if keyCount is exponentially greater
105   *          than maxKeys.
106   * @throws IllegalArgumentException
107   */
108  // Used only in testcases
109  public BloomFilterChunk(int maxKeys, double errorRate, int hashType,
110      int foldFactor) throws IllegalArgumentException {
111    this(hashType, BloomType.ROW);
112
113    long bitSize = BloomFilterUtil.computeBitSize(maxKeys, errorRate);
114    hashCount = BloomFilterUtil.optimalFunctionCount(maxKeys, bitSize);
115    this.maxKeys = maxKeys;
116
117    // increase byteSize so folding is possible
118    byteSize = BloomFilterUtil.computeFoldableByteSize(bitSize, foldFactor);
119
120    sanityCheck();
121  }
122
123  /**
124   * Creates another similar Bloom filter. Does not copy the actual bits, and
125   * sets the new filter's key count to zero.
126   *
127   * @return a Bloom filter with the same configuration as this
128   */
129  public BloomFilterChunk createAnother() {
130    BloomFilterChunk bbf = new BloomFilterChunk(hashType, this.bloomType);
131    bbf.byteSize = byteSize;
132    bbf.hashCount = hashCount;
133    bbf.maxKeys = maxKeys;
134    return bbf;
135  }
136
137  public void allocBloom() {
138    if (this.bloom != null) {
139      throw new IllegalArgumentException("can only create bloom once.");
140    }
141    this.bloom = ByteBuffer.allocate((int)this.byteSize);
142    assert this.bloom.hasArray();
143  }
144
145  void sanityCheck() throws IllegalArgumentException {
146    if(0 >= this.byteSize || this.byteSize > Integer.MAX_VALUE) {
147      throw new IllegalArgumentException("Invalid byteSize: " + this.byteSize);
148    }
149
150    if(this.hashCount <= 0) {
151      throw new IllegalArgumentException("Hash function count must be > 0");
152    }
153
154    if (this.hash == null) {
155      throw new IllegalArgumentException("hashType must be known");
156    }
157
158    if (this.keyCount < 0) {
159      throw new IllegalArgumentException("must have positive keyCount");
160    }
161  }
162
163  void bloomCheck(ByteBuffer bloom)  throws IllegalArgumentException {
164    if (this.byteSize != bloom.limit()) {
165      throw new IllegalArgumentException(
166          "Configured bloom length should match actual length");
167    }
168  }
169
170  // Used only by tests
171  void add(byte [] buf, int offset, int len) {
172    /*
173     * For faster hashing, use combinatorial generation
174     * http://www.eecs.harvard.edu/~kirsch/pubs/bbbf/esa06.pdf
175     */
176    HashKey<byte[]> hashKey = new ByteArrayHashKey(buf, offset, len);
177    int hash1 = this.hash.hash(hashKey, 0);
178    int hash2 = this.hash.hash(hashKey, hash1);
179    setHashLoc(hash1, hash2);
180  }
181
182  public void add(Cell cell) {
183    /*
184     * For faster hashing, use combinatorial generation
185     * http://www.eecs.harvard.edu/~kirsch/pubs/bbbf/esa06.pdf
186     */
187    int hash1;
188    int hash2;
189    HashKey<Cell> hashKey;
190    if (this.bloomType == BloomType.ROWCOL) {
191      hashKey = new RowColBloomHashKey(cell);
192      hash1 = this.hash.hash(hashKey, 0);
193      hash2 = this.hash.hash(hashKey, hash1);
194    } else {
195      hashKey = new RowBloomHashKey(cell);
196      hash1 = this.hash.hash(hashKey, 0);
197      hash2 = this.hash.hash(hashKey, hash1);
198    }
199    setHashLoc(hash1, hash2);
200  }
201
202  private void setHashLoc(int hash1, int hash2) {
203    for (int i = 0; i < this.hashCount; i++) {
204      long hashLoc = Math.abs((hash1 + i * hash2) % (this.byteSize * 8));
205      set(hashLoc);
206    }
207
208    ++this.keyCount;
209  }
210
211  //---------------------------------------------------------------------------
212  /** Private helpers */
213
214  /**
215   * Set the bit at the specified index to 1.
216   *
217   * @param pos index of bit
218   */
219  void set(long pos) {
220    int bytePos = (int)(pos / 8);
221    int bitPos = (int)(pos % 8);
222    byte curByte = bloom.get(bytePos);
223    curByte |= BloomFilterUtil.bitvals[bitPos];
224    bloom.put(bytePos, curByte);
225  }
226
227  /**
228   * Check if bit at specified index is 1.
229   *
230   * @param pos index of bit
231   * @return true if bit at specified index is 1, false if 0.
232   */
233  static boolean get(int pos, ByteBuffer bloomBuf, int bloomOffset) {
234    int bytePos = pos >> 3; //pos / 8
235    int bitPos = pos & 0x7; //pos % 8
236    // TODO access this via Util API which can do Unsafe access if possible(?)
237    byte curByte = bloomBuf.get(bloomOffset + bytePos);
238    curByte &= BloomFilterUtil.bitvals[bitPos];
239    return (curByte != 0);
240  }
241
242  @Override
243  public long getKeyCount() {
244    return keyCount;
245  }
246
247  @Override
248  public long getMaxKeys() {
249    return maxKeys;
250  }
251
252  @Override
253  public long getByteSize() {
254    return byteSize;
255  }
256
257  public int getHashType() {
258    return hashType;
259  }
260
261  public void compactBloom() {
262    // see if the actual size is exponentially smaller than expected.
263    if (this.keyCount > 0 && this.bloom.hasArray()) {
264      int pieces = 1;
265      int newByteSize = (int)this.byteSize;
266      int newMaxKeys = this.maxKeys;
267
268      // while exponentially smaller & folding is lossless
269      while ((newByteSize & 1) == 0 && newMaxKeys > (this.keyCount<<1)) {
270        pieces <<= 1;
271        newByteSize >>= 1;
272        newMaxKeys >>= 1;
273      }
274
275      // if we should fold these into pieces
276      if (pieces > 1) {
277        byte[] array = this.bloom.array();
278        int start = this.bloom.arrayOffset();
279        int end = start + newByteSize;
280        int off = end;
281        for(int p = 1; p < pieces; ++p) {
282          for(int pos = start; pos < end; ++pos) {
283            array[pos] |= array[off++];
284          }
285        }
286        // folding done, only use a subset of this array
287        this.bloom.rewind();
288        this.bloom.limit(newByteSize);
289        this.bloom = this.bloom.slice();
290        this.byteSize = newByteSize;
291        this.maxKeys = newMaxKeys;
292      }
293    }
294  }
295
296  /**
297   * Writes just the bloom filter to the output array
298   * @param out OutputStream to place bloom
299   * @throws IOException Error writing bloom array
300   */
301  public void writeBloom(final DataOutput out)
302      throws IOException {
303    if (!this.bloom.hasArray()) {
304      throw new IOException("Only writes ByteBuffer with underlying array.");
305    }
306    out.write(this.bloom.array(), this.bloom.arrayOffset(), this.bloom.limit());
307  }
308
309  public int getHashCount() {
310    return hashCount;
311  }
312
313  @Override
314  public String toString() {
315    return BloomFilterUtil.toString(this);
316  }
317
318}