.../apache/hadoop/hbase/util/ByteBufferUtils.java | 10 ++ .../java/org/apache/hadoop/hbase/util/Hash.java | 13 ++ .../org/apache/hadoop/hbase/util/JenkinsHash.java | 161 +++++++++++++++++++ .../org/apache/hadoop/hbase/util/MurmurHash.java | 53 +++++++ .../org/apache/hadoop/hbase/util/MurmurHash3.java | 56 +++++++ .../hbase/io/hfile/CompoundBloomFilterWriter.java | 40 +++++ .../hadoop/hbase/regionserver/StoreFileWriter.java | 175 +++++++++++++++------ .../apache/hadoop/hbase/util/BloomFilterChunk.java | 16 ++ .../hadoop/hbase/util/BloomFilterWriter.java | 11 ++ 9 files changed, 483 insertions(+), 52 deletions(-) diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java index df23614..2e70493 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java @@ -21,6 +21,7 @@ import java.io.DataInputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.lang.reflect.Array; import java.math.BigDecimal; import java.math.BigInteger; import java.nio.ByteBuffer; @@ -962,6 +963,15 @@ public final class ByteBufferUtils { } } + // May be not needed. + public static byte[] copyOfRange(ByteBuffer original, int from, int to) { + int newLength = to - from; + if (newLength < 0) throw new IllegalArgumentException(from + " > " + to); + byte[] copy = new byte[newLength]; + ByteBufferUtils.copyFromBufferToArray(copy, original, from, 0, newLength); + return copy; + } + // For testing purpose public static String toStringBinary(final ByteBuffer b, int off, int len) { StringBuilder result = new StringBuilder(); diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Hash.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Hash.java index aa0795d..c1b30e2 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Hash.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Hash.java @@ -19,6 +19,8 @@ package org.apache.hadoop.hbase.util; +import java.nio.ByteBuffer; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; @@ -141,5 +143,16 @@ public abstract class Hash { */ public abstract int hash(byte[] bytes, int offset, int length, int initval); + /** + * Calculate a hash using bytes from offset to offset + + * length, and the provided seed value. + * @param buf input buffer + * @param offset the offset into the array to start consideration + * @param length length of the valid bytes after offset to consider + * @param initval seed value + * @return hash value + */ + public abstract int hash(ByteBuffer buf, int offset, int length, int initval); + // TODO : a buffer based hash function would be needed.. Not adding it for now } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/JenkinsHash.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/JenkinsHash.java index 789bd8d..b3af9b9 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/JenkinsHash.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/JenkinsHash.java @@ -23,6 +23,7 @@ import static java.lang.Integer.rotateLeft; import java.io.FileInputStream; import java.io.IOException; +import java.nio.ByteBuffer; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; @@ -239,6 +240,166 @@ public class JenkinsHash extends Hash { return c; } + @Override + public int hash(ByteBuffer buf, int off, int nbytes, int initval) { + int length = nbytes; + int a, b, c; + a = b = c = 0xdeadbeef + length + initval; + int offset = off; + for (; length > 12; offset += 12, length -= 12) { + a += (buf.get(offset) & BYTE_MASK); + a += ((buf.get(offset + 1) & BYTE_MASK) << 8); + a += ((buf.get(offset + 2) & BYTE_MASK) << 16); + a += ((buf.get(offset + 3) & BYTE_MASK) << 24); + b += (buf.get(offset + 4) & BYTE_MASK); + b += ((buf.get(offset + 5) & BYTE_MASK) << 8); + b += ((buf.get(offset + 6) & BYTE_MASK) << 16); + b += ((buf.get(offset + 7) & BYTE_MASK) << 24); + c += (buf.get(offset + 8) & BYTE_MASK); + c += ((buf.get(offset + 9) & BYTE_MASK) << 8); + c += ((buf.get(offset + 10) & BYTE_MASK) << 16); + c += ((buf.get(offset + 11) & BYTE_MASK) << 24); + + /* + * mix -- mix 3 32-bit values reversibly. + * This is reversible, so any information in (a,b,c) before mix() is + * still in (a,b,c) after mix(). + * + * If four pairs of (a,b,c) inputs are run through mix(), or through + * mix() in reverse, there are at least 32 bits of the output that + * are sometimes the same for one pair and different for another pair. + * + * This was tested for: + * - pairs that differed by one bit, by two bits, in any combination + * of top bits of (a,b,c), or in any combination of bottom bits of + * (a,b,c). + * - "differ" is defined as +, -, ^, or ~^. For + and -, I transformed + * the output delta to a Gray code (a^(a>>1)) so a string of 1's (as + * is commonly produced by subtraction) look like a single 1-bit + * difference. + * - the base values were pseudorandom, all zero but one bit set, or + * all zero plus a counter that starts at zero. + * + * Some k values for my "a-=c; a^=rot(c,k); c+=b;" arrangement that + * satisfy this are + * 4 6 8 16 19 4 + * 9 15 3 18 27 15 + * 14 9 3 7 17 3 + * Well, "9 15 3 18 27 15" didn't quite get 32 bits diffing for + * "differ" defined as + with a one-bit base and a two-bit delta. I + * used http://burtleburtle.net/bob/hash/avalanche.html to choose + * the operations, constants, and arrangements of the variables. + * + * This does not achieve avalanche. There are input bits of (a,b,c) + * that fail to affect some output bits of (a,b,c), especially of a. + * The most thoroughly mixed value is c, but it doesn't really even + * achieve avalanche in c. + * + * This allows some parallelism. Read-after-writes are good at doubling + * the number of bits affected, so the goal of mixing pulls in the + * opposite direction as the goal of parallelism. I did what I could. + * Rotates seem to cost as much as shifts on every machine I could lay + * my hands on, and rotates are much kinder to the top and bottom bits, + * so I used rotates. + * + * #define mix(a,b,c) \ + * { \ + * a -= c; a ^= rot(c, 4); c += b; \ + * b -= a; b ^= rot(a, 6); a += c; \ + * c -= b; c ^= rot(b, 8); b += a; \ + * a -= c; a ^= rot(c,16); c += b; \ + * b -= a; b ^= rot(a,19); a += c; \ + * c -= b; c ^= rot(b, 4); b += a; \ + * } + * + * mix(a,b,c); + */ + a -= c; a ^= rotateLeft(c, 4); c += b; + b -= a; b ^= rotateLeft(a, 6); a += c; + c -= b; c ^= rotateLeft(b, 8); b += a; + a -= c; a ^= rotateLeft(c, 16); c += b; + b -= a; b ^= rotateLeft(a, 19); a += c; + c -= b; c ^= rotateLeft(b, 4); b += a; + } + + //-------------------------------- last block: affect all 32 bits of (c) + switch (length) { // all the case statements fall through + case 12: + c += ((buf.get(offset + 11) & BYTE_MASK) << 24); + case 11: + c += ((buf.get(offset + 10) & BYTE_MASK) << 16); + case 10: + c += ((buf.get(offset + 9) & BYTE_MASK) << 8); + case 9: + c += (buf.get(offset + 8) & BYTE_MASK); + case 8: + b += ((buf.get(offset + 7) & BYTE_MASK) << 24); + case 7: + b += ((buf.get(offset + 6) & BYTE_MASK) << 16); + case 6: + b += ((buf.get(offset + 5) & BYTE_MASK) << 8); + case 5: + b += (buf.get(offset + 4) & BYTE_MASK); + case 4: + a += ((buf.get(offset + 3) & BYTE_MASK) << 24); + case 3: + a += ((buf.get(offset + 2) & BYTE_MASK) << 16); + case 2: + a += ((buf.get(offset + 1) & BYTE_MASK) << 8); + case 1: + //noinspection PointlessArithmeticExpression + a += (buf.get(offset + 0) & BYTE_MASK); + break; + case 0: + return c; + } + /* + * final -- final mixing of 3 32-bit values (a,b,c) into c + * + * Pairs of (a,b,c) values differing in only a few bits will usually + * produce values of c that look totally different. This was tested for + * - pairs that differed by one bit, by two bits, in any combination + * of top bits of (a,b,c), or in any combination of bottom bits of + * (a,b,c). + * + * - "differ" is defined as +, -, ^, or ~^. For + and -, I transformed + * the output delta to a Gray code (a^(a>>1)) so a string of 1's (as + * is commonly produced by subtraction) look like a single 1-bit + * difference. + * + * - the base values were pseudorandom, all zero but one bit set, or + * all zero plus a counter that starts at zero. + * + * These constants passed: + * 14 11 25 16 4 14 24 + * 12 14 25 16 4 14 24 + * and these came close: + * 4 8 15 26 3 22 24 + * 10 8 15 26 3 22 24 + * 11 8 15 26 3 22 24 + * + * #define final(a,b,c) \ + * { + * c ^= b; c -= rot(b,14); \ + * a ^= c; a -= rot(c,11); \ + * b ^= a; b -= rot(a,25); \ + * c ^= b; c -= rot(b,16); \ + * a ^= c; a -= rot(c,4); \ + * b ^= a; b -= rot(a,14); \ + * c ^= b; c -= rot(b,24); \ + * } + * + */ + c ^= b; c -= rotateLeft(b, 14); + a ^= c; a -= rotateLeft(c, 11); + b ^= a; b -= rotateLeft(a, 25); + c ^= b; c -= rotateLeft(b, 16); + a ^= c; a -= rotateLeft(c, 4); + b ^= a; b -= rotateLeft(a, 14); + c ^= b; c -= rotateLeft(b, 24); + return c; + } + /** * Compute the hash of the specified file * @param args name of file to compute hash of. diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/MurmurHash.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/MurmurHash.java index 5c27386..196af8e 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/MurmurHash.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/MurmurHash.java @@ -19,6 +19,8 @@ package org.apache.hadoop.hbase.util; +import java.nio.ByteBuffer; + import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; @@ -89,4 +91,55 @@ public class MurmurHash extends Hash { return h; } + @Override + public int hash(ByteBuffer data, int offset, int length, int seed) { + int m = 0x5bd1e995; + int r = 24; + + int h = seed ^ length; + + int len_4 = length >> 2; + + for (int i = 0; i < len_4; i++) { + int i_4 = (i << 2) + offset; + int k = data.get(i_4 + 3); + k = k << 8; + k = k | (data.get(i_4 + 2) & 0xff); + k = k << 8; + k = k | (data.get(i_4 + 1) & 0xff); + k = k << 8; + //noinspection PointlessArithmeticExpression + k = k | (data.get(i_4 + 0) & 0xff); + k *= m; + k ^= k >>> r; + k *= m; + h *= m; + h ^= k; + } + + // avoid calculating modulo + int len_m = len_4 << 2; + int left = length - len_m; + int i_m = len_m + offset; + + if (left != 0) { + if (left >= 3) { + h ^= data.get(i_m + 2) << 16; + } + if (left >= 2) { + h ^= data.get(i_m + 1) << 8; + } + if (left >= 1) { + h ^= data.get(i_m); + } + + h *= m; + } + + h ^= h >>> 13; + h *= m; + h ^= h >>> 15; + + return h; + } } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/MurmurHash3.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/MurmurHash3.java index 78d1331..92eeb91 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/MurmurHash3.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/MurmurHash3.java @@ -18,6 +18,8 @@ */ package org.apache.hadoop.hbase.util; +import java.nio.ByteBuffer; + import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; @@ -90,4 +92,58 @@ public class MurmurHash3 extends Hash { return h1; } + + /** Returns the MurmurHash3_x86_32 hash. */ + @edu.umd.cs.findbugs.annotations.SuppressWarnings("SF") + @Override + public int hash(ByteBuffer buf, int offset, int length, int initval) { + final int c1 = 0xcc9e2d51; + final int c2 = 0x1b873593; + + int h1 = initval; + int roundedEnd = offset + (length & 0xfffffffc); // round down to 4 byte block + + for (int i = offset; i < roundedEnd; i += 4) { + // little endian load order + int k1 = (buf.get(i) & 0xff) | ((buf.get(i + 1) & 0xff) << 8) | ((buf.get(i + 2) & 0xff) << 16) + | (buf.get(i + 3) << 24); + k1 *= c1; + k1 = (k1 << 15) | (k1 >>> 17); // ROTL32(k1,15); + k1 *= c2; + + h1 ^= k1; + h1 = (h1 << 13) | (h1 >>> 19); // ROTL32(h1,13); + h1 = h1 * 5 + 0xe6546b64; + } + + // tail + int k1 = 0; + + switch (length & 0x03) { + case 3: + k1 = (buf.get(roundedEnd + 2)& 0xff) << 16; + // FindBugs SF_SWITCH_FALLTHROUGH + case 2: + k1 |= (buf.get(roundedEnd + 1) & 0xff) << 8; + // FindBugs SF_SWITCH_FALLTHROUGH + case 1: + k1 |= (buf.get(roundedEnd) & 0xff); + k1 *= c1; + k1 = (k1 << 15) | (k1 >>> 17); // ROTL32(k1,15); + k1 *= c2; + h1 ^= k1; + } + + // finalization + h1 ^= length; + + // fmix(h1); + h1 ^= h1 >>> 16; + h1 *= 0x85ebca6b; + h1 ^= h1 >>> 13; + h1 *= 0xc2b2ae35; + h1 ^= h1 >>> 16; + + return h1; + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundBloomFilterWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundBloomFilterWriter.java index d95aafd..9d7a6c1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundBloomFilterWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundBloomFilterWriter.java @@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.io.hfile; import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.Arrays; import java.util.LinkedList; import java.util.Queue; @@ -33,6 +34,7 @@ import org.apache.hadoop.hbase.regionserver.StoreFileWriter; import org.apache.hadoop.hbase.util.BloomFilterChunk; import org.apache.hadoop.hbase.util.BloomFilterUtil; import org.apache.hadoop.hbase.util.BloomFilterWriter; +import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.io.Writable; @@ -198,6 +200,44 @@ public class CompoundBloomFilterWriter extends CompoundBloomFilterBase } @Override + public void add(ByteBuffer bloomKey, int keyOffset, int keyLength) { + if (bloomKey == null) + throw new NullPointerException(); + + enqueueReadyChunk(false); + + if (chunk == null) { + if (firstKeyInChunk != null) { + throw new IllegalStateException("First key in chunk already set: " + + Bytes.toStringBinary(firstKeyInChunk)); + } + firstKeyInChunk = ByteBufferUtils.copyOfRange(bloomKey, keyOffset, keyOffset + + keyLength); + + if (prevChunk == null) { + // First chunk + chunk = BloomFilterUtil.createBySize(chunkByteSize, errorRate, + hashType, maxFold); + } else { + // Use the same parameters as the last chunk, but a new array and + // a zero key count. + chunk = prevChunk.createAnother(); + } + + if (chunk.getKeyCount() != 0) { + throw new IllegalStateException("keyCount=" + chunk.getKeyCount() + + " > 0"); + } + + chunk.allocBloom(); + ++numChunks; + } + + chunk.add(bloomKey, keyOffset, keyLength); + ++totalKeyCount; + } + + @Override public void writeInlineBlock(DataOutput out) throws IOException { // We don't remove the chunk from the queue here, because we might need it // again for cache-on-write. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java index 4a42b7f..ece4e1f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java @@ -24,6 +24,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.ByteBufferedCell; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellUtil; @@ -37,11 +38,13 @@ import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.regionserver.compactions.Compactor; import org.apache.hadoop.hbase.util.BloomFilterFactory; import org.apache.hadoop.hbase.util.BloomFilterWriter; +import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.io.WritableUtils; import java.io.IOException; import java.net.InetSocketAddress; +import java.nio.ByteBuffer; import java.util.Arrays; /** @@ -55,6 +58,9 @@ public class StoreFileWriter implements Compactor.CellSink { private final BloomFilterWriter generalBloomFilterWriter; private final BloomFilterWriter deleteFamilyBloomFilterWriter; private final BloomType bloomType; + // One for Bytebuffer and another for byte[] looks ugly. But using only one will lead to either + // creating BB from byte[] and thus lot of short lived objects + private ByteBuffer lastBloomKeyBuffer; private byte[] lastBloomKey; private int lastBloomKeyOffset, lastBloomKeyLen; private Cell lastCell = null; @@ -200,40 +206,105 @@ public class StoreFileWriter implements Compactor.CellSink { private void appendGeneralBloomfilter(final Cell cell) throws IOException { if (this.generalBloomFilterWriter != null) { - // only add to the bloom filter on a new, unique key boolean newKey = true; + // only add to the bloom filter on a new, unique key if (this.lastCell != null) { switch(bloomType) { + case ROW: + newKey = ! CellUtil.matchingRows(cell, lastCell); + break; + case ROWCOL: + newKey = ! CellUtil.matchingRowColumn(cell, lastCell); + break; + case NONE: + newKey = false; + break; + default: + throw new IOException("Invalid Bloom filter type: " + bloomType + + " (ROW or ROWCOL expected)"); + } + } + Cell bloomKeyKV = null; + if(cell instanceof ByteBufferedCell) { + if(newKey) { + /* + * http://2.bp.blogspot.com/_Cib_A77V54U/StZMrzaKufI/AAAAAAAAADo/ZhK7bGoJdMQ/s400/KeyValue.png + * Key = RowLen + Row + FamilyLen + Column [Family + Qualifier] + TimeStamp + * + * 2 Types of Filtering: + * 1. Row = Row + * 2. RowCol = Row + Qualifier + */ + ByteBuffer bloomKey = null; + int bloomKeyOffset, bloomKeyLen; + + switch (bloomType) { case ROW: - newKey = ! CellUtil.matchingRows(cell, lastCell); + // Avoid copy in case of BBCell + bloomKey = ((ByteBufferedCell) cell).getRowByteBuffer(); + bloomKeyOffset = ((ByteBufferedCell) cell).getRowPosition(); + bloomKeyLen = cell.getRowLength(); break; case ROWCOL: - newKey = ! CellUtil.matchingRowColumn(cell, lastCell); - break; - case NONE: - newKey = false; + // merge(row, qualifier) + bloomKeyKV = CellUtil.createFirstOnRowCol(cell); + // Cannot avoid this copy because we need the key part + byte[] key = new byte[Bytes.SIZEOF_SHORT + cell.getRowLength() + Bytes.SIZEOF_BYTE + + cell.getFamilyLength() + cell.getQualifierLength() + Bytes.SIZEOF_LONG + + Bytes.SIZEOF_BYTE]; + // we need consecutive byte[] reprsenting the key + KeyValueUtil.appendToByteArray(cell, key, 0); + bloomKey = ByteBuffer.wrap(key); + bloomKeyOffset = 0; + bloomKeyLen = key.length; break; default: throw new IOException("Invalid Bloom filter type: " + bloomType + - " (ROW or ROWCOL expected)"); + " (ROW or ROWCOL expected)"); + } + generalBloomFilterWriter.add(bloomKey, bloomKeyOffset, bloomKeyLen); + if (lastBloomKeyBuffer != null) { + int res = 0; + // hbase:meta does not have blooms. So we need not have special interpretation + // of the hbase:meta cells. We can safely use Bytes.BYTES_RAWCOMPARATOR for ROW Bloom + if (bloomType == BloomType.ROW) { + res = ByteBufferUtils.compareTo(bloomKey, bloomKeyOffset, bloomKeyLen, + lastBloomKeyBuffer, lastBloomKeyOffset, lastBloomKeyLen); + } else { + // TODO : Caching of kv components becomes important in these cases + res = CellComparator.COMPARATOR.compare(bloomKeyKV, lastBloomKeyOnlyKV); + } + if (res <= 0) { + throw new IOException("Non-increasing Bloom keys: " + + ByteBufferUtils.toStringBinary(bloomKey, bloomKeyOffset, bloomKeyLen) + " after " + + ByteBufferUtils.toStringBinary(lastBloomKeyBuffer, lastBloomKeyOffset, lastBloomKeyLen)); + } + } + lastBloomKeyBuffer = bloomKey; + lastBloomKeyOffset = bloomKeyOffset; + lastBloomKeyLen = bloomKeyLen; + if (bloomType == BloomType.ROWCOL) { + lastBloomKeyOnlyKV.setKey(bloomKey.array(), bloomKey.arrayOffset() + bloomKeyOffset, + bloomKeyLen); + } + this.lastCell = cell; } - } - if (newKey) { - /* - * http://2.bp.blogspot.com/_Cib_A77V54U/StZMrzaKufI/AAAAAAAAADo/ZhK7bGoJdMQ/s400/KeyValue.png - * Key = RowLen + Row + FamilyLen + Column [Family + Qualifier] + TimeStamp - * - * 2 Types of Filtering: - * 1. Row = Row - * 2. RowCol = Row + Qualifier - */ - byte[] bloomKey = null; - // Used with ROW_COL bloom - KeyValue bloomKeyKV = null; - int bloomKeyOffset, bloomKeyLen; - - switch (bloomType) { + } else { + if (newKey) { + /* + * http://2.bp.blogspot.com/_Cib_A77V54U/StZMrzaKufI/AAAAAAAAADo/ZhK7bGoJdMQ/s400/KeyValue.png + * Key = RowLen + Row + FamilyLen + Column [Family + Qualifier] + TimeStamp + * + * 2 Types of Filtering: + * 1. Row = Row + * 2. RowCol = Row + Qualifier + */ + byte[] bloomKey = null; + int bloomKeyOffset, bloomKeyLen; + + switch (bloomType) { case ROW: + // Avoid copy in case of ROW only bloom bloomKey = cell.getRowArray(); bloomKeyOffset = cell.getRowOffset(); bloomKeyLen = cell.getRowLength(); @@ -242,45 +313,45 @@ public class StoreFileWriter implements Compactor.CellSink { // merge(row, qualifier) // TODO: could save one buffer copy in case of compound Bloom // filters when this involves creating a KeyValue - // TODO : Handle while writes also bloomKeyKV = KeyValueUtil.createFirstOnRow(cell.getRowArray(), cell.getRowOffset(), - cell.getRowLength(), - HConstants.EMPTY_BYTE_ARRAY, 0, 0, cell.getQualifierArray(), - cell.getQualifierOffset(), - cell.getQualifierLength()); - bloomKey = bloomKeyKV.getBuffer(); - bloomKeyOffset = bloomKeyKV.getKeyOffset(); - bloomKeyLen = bloomKeyKV.getKeyLength(); + cell.getRowLength(), + HConstants.EMPTY_BYTE_ARRAY, 0, 0, cell.getQualifierArray(), + cell.getQualifierOffset(), + cell.getQualifierLength()); + bloomKey = ((KeyValue)bloomKeyKV).getBuffer(); + bloomKeyOffset = ((KeyValue)bloomKeyKV).getKeyOffset(); + bloomKeyLen = ((KeyValue)bloomKeyKV).getKeyLength(); break; default: throw new IOException("Invalid Bloom filter type: " + bloomType + " (ROW or ROWCOL expected)"); - } - generalBloomFilterWriter.add(bloomKey, bloomKeyOffset, bloomKeyLen); - if (lastBloomKey != null) { - int res = 0; - // hbase:meta does not have blooms. So we need not have special interpretation - // of the hbase:meta cells. We can safely use Bytes.BYTES_RAWCOMPARATOR for ROW Bloom - if (bloomType == BloomType.ROW) { - res = Bytes.BYTES_RAWCOMPARATOR.compare(bloomKey, bloomKeyOffset, bloomKeyLen, - lastBloomKey, lastBloomKeyOffset, lastBloomKeyLen); - } else { - // TODO : Caching of kv components becomes important in these cases - res = CellComparator.COMPARATOR.compare(bloomKeyKV, lastBloomKeyOnlyKV); } - if (res <= 0) { - throw new IOException("Non-increasing Bloom keys: " + generalBloomFilterWriter.add(bloomKey, bloomKeyOffset, bloomKeyLen); + if (lastBloomKey != null) { + int res = 0; + // hbase:meta does not have blooms. So we need not have special interpretation + // of the hbase:meta cells. We can safely use Bytes.BYTES_RAWCOMPARATOR for ROW Bloom + if (bloomType == BloomType.ROW) { + res = Bytes.compareTo(bloomKey, bloomKeyOffset, bloomKeyLen, + lastBloomKey, lastBloomKeyOffset, lastBloomKeyLen); + } else { + // TODO : Caching of kv components becomes important in these cases + res = CellComparator.COMPARATOR.compare(bloomKeyKV, lastBloomKeyOnlyKV); + } + if (res <= 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; + if (bloomType == BloomType.ROWCOL) { + lastBloomKeyOnlyKV.setKey(bloomKey, bloomKeyOffset, bloomKeyLen); + } + this.lastCell = cell; } - lastBloomKey = bloomKey; - lastBloomKeyOffset = bloomKeyOffset; - lastBloomKeyLen = bloomKeyLen; - if (bloomType == BloomType.ROWCOL) { - lastBloomKeyOnlyKV.setKey(bloomKey, bloomKeyOffset, bloomKeyLen); - } - this.lastCell = cell; } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterChunk.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterChunk.java index 1e77984..8880dab 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterChunk.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterChunk.java @@ -181,6 +181,22 @@ public class BloomFilterChunk implements BloomFilterBase { ++this.keyCount; } + public void add(ByteBuffer buf, int offset, int len) { + /* + * For faster hashing, use combinatorial generation + * http://www.eecs.harvard.edu/~kirsch/pubs/bbbf/esa06.pdf + */ + int hash1 = this.hash.hash(buf, offset, len, 0); + int hash2 = this.hash.hash(buf, offset, len, hash1); + + for (int i = 0; i < this.hashCount; i++) { + long hashLoc = Math.abs((hash1 + i * hash2) % (this.byteSize * 8)); + set(hashLoc); + } + + ++this.keyCount; + } + //--------------------------------------------------------------------------- /** Private helpers */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterWriter.java index 6869d69..9963d19 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterWriter.java @@ -19,6 +19,8 @@ package org.apache.hadoop.hbase.util; +import java.nio.ByteBuffer; + import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.io.Writable; @@ -55,4 +57,13 @@ public interface BloomFilterWriter extends BloomFilterBase { */ void add(byte[] buf, int offset, int len); + /** + * 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(ByteBuffer buf, int offset, int len); + }