From 11b727f3fd26f5c23f9bd2e707c5a7ea187ed497 Mon Sep 17 00:00:00 2001 From: Walter Koetke Date: Tue, 5 Apr 2016 14:50:02 -0400 Subject: [PATCH] HBASE-15480: added api for bloom check for array of keys, to avoid overhead of calling for one key at a time. --- .../hadoop/hbase/io/hfile/CompoundBloomFilter.java | 99 ++++++++++++++-------- .../org/apache/hadoop/hbase/util/BloomFilter.java | 14 ++- 2 files changed, 78 insertions(+), 35 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundBloomFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundBloomFilter.java index 2d773bb..e35f530 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundBloomFilter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundBloomFilter.java @@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.io.hfile; import java.io.DataInput; import java.io.IOException; +import java.util.BitSet; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; @@ -94,12 +95,39 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase } @Override + public BitSet contains(byte[][] key, int[] keyOffset, int[] keyLength, ByteBuff bloom) { + BitSet bitSet = new BitSet(key.length); + HFileBlock[] bloomBlocks = new HFileBlock[numChunks]; + boolean updateStats = numQueriesPerChunk != null; + for (int i =0; i< key.length; i++) { + if (key[i] == null) + continue; + boolean result; + int block = index.rootBlockContainingKey(key[i], keyOffset[i], keyLength[i]); + if (block<0) + result = false; + else { + HFileBlock bloomBlock; + if (bloomBlocks[block]!=null) + bloomBlock = bloomBlocks[block]; + else { + bloomBlock = getHFileBlock(block, key[i], keyOffset[i], keyLength[i]); + bloomBlocks[block] = bloomBlock; + } + result = bloomContains(bloomBlock, key[i], keyOffset[i], keyLength[i]); + } + if (updateStats) + updateTestingStats(block, result); + bitSet.set(i, result); + } + return bitSet; + } + + @Override public boolean contains(byte[] key, int keyOffset, int keyLength, ByteBuff 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. - - int block = index.rootBlockContainingKey(key, keyOffset, - keyLength); + int block = index.rootBlockContainingKey(key, keyOffset, keyLength); return checkContains(key, keyOffset, keyLength, block); } @@ -108,36 +136,10 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase 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, true, - BlockType.BLOOM_CHUNK, null); - } 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); - } - try { - ByteBuff bloomBuf = bloomBlock.getBufferReadOnly(); - result = - BloomFilterUtil.contains(key, keyOffset, keyLength, bloomBuf, bloomBlock.headerSize(), - bloomBlock.getUncompressedSizeWithoutHeader(), hash, hashCount); - } finally { - // After the use return back the block if it was served from a cache. - reader.returnBlock(bloomBlock); - } - } - - if (numQueriesPerChunk != null && block >= 0) { - // Update statistics. Only used in unit tests. - ++numQueriesPerChunk[block]; - if (result) - ++numPositivesPerChunk[block]; + HFileBlock bloomBlock = getHFileBlock(block, key, keyOffset, keyLength); + result = bloomContains(bloomBlock, key, keyOffset, keyLength); } - + updateTestingStats(block,result); return result; } @@ -198,13 +200,42 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase public String toString() { StringBuilder sb = new StringBuilder(); sb.append(BloomFilterUtil.formatStats(this)); - sb.append(BloomFilterUtil.STATS_RECORD_SEP + + sb.append(BloomFilterUtil.STATS_RECORD_SEP + "Number of chunks: " + numChunks); - sb.append(BloomFilterUtil.STATS_RECORD_SEP + + sb.append(BloomFilterUtil.STATS_RECORD_SEP + ((comparator != null) ? "Comparator: " + comparator.getClass().getSimpleName() : "Comparator: " + Bytes.BYTES_RAWCOMPARATOR.getClass().getSimpleName())); return sb.toString(); } + private HFileBlock getHFileBlock(int block, byte[] key, int keyOffset, int keyLength) { + try { + // We cache the block and use a positional read. + return reader.readBlock(index.getRootBlockOffset(block), + index.getRootBlockDataSize(block), true, true, false, true, + BlockType.BLOOM_CHUNK, null); + } 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); + } + } + + private void updateTestingStats(int block, boolean result) { + if (numQueriesPerChunk != null && block >= 0) { + ++numQueriesPerChunk[block]; + if (result) + ++numPositivesPerChunk[block]; + } + } + + private boolean bloomContains(HFileBlock bloomBlock, byte[] key, int keyOffset, int keyLength) { + ByteBuff bloomBuf = bloomBlock.getBufferReadOnly(); + return BloomFilterUtil.contains(key, keyOffset, keyLength, + bloomBuf, bloomBlock.headerSize(), + bloomBlock.getUncompressedSizeWithoutHeader(), hash, hashCount); + } + } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilter.java index 2062244..5c3ec8d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilter.java @@ -18,7 +18,7 @@ */ package org.apache.hadoop.hbase.util; - +import java.util.BitSet; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.nio.ByteBuff; @@ -98,6 +98,18 @@ public interface BloomFilter extends BloomFilterBase { boolean contains(byte[] buf, int offset, int length, ByteBuff bloom); /** + * Check if the specified keys are contained in the bloom filter. + * + * @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. This can be null if auto-loading + * is supported. + * @return true if matched by bloom, false if not + */ + BitSet contains(byte [][] buf, int[] offset, int[] length, ByteBuff bloom); + + /** * @return true if this Bloom filter can automatically load its data * and thus allows a null byte buffer to be passed to contains() */ -- 2.3.2 (Apple Git-55)