.../java/org/apache/hadoop/hbase/CellUtil.java | 61 ++++- .../apache/hadoop/hbase/util/ByteBufferUtils.java | 15 ++ .../java/org/apache/hadoop/hbase/util/Hash.java | 52 +++- .../org/apache/hadoop/hbase/util/JenkinsHash.java | 283 +++++++++++++++++++++ .../org/apache/hadoop/hbase/util/MurmurHash.java | 122 +++++++++ .../org/apache/hadoop/hbase/util/MurmurHash3.java | 129 ++++++++++ .../hadoop/hbase/io/hfile/CompoundBloomFilter.java | 4 +- .../hbase/io/hfile/CompoundBloomFilterWriter.java | 81 +++++- .../hadoop/hbase/regionserver/StoreFileWriter.java | 101 ++------ .../org/apache/hadoop/hbase/util/BloomContext.java | 62 +++++ .../apache/hadoop/hbase/util/BloomFilterChunk.java | 27 ++ .../hadoop/hbase/util/BloomFilterWriter.java | 17 ++ .../apache/hadoop/hbase/util/RowBloomContext.java | 128 ++++++++++ .../hadoop/hbase/util/RowColBloomContext.java | 69 +++++ 14 files changed, 1058 insertions(+), 93 deletions(-) diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java index b769f19..11dfa4d 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java @@ -33,6 +33,7 @@ import java.util.NavigableMap; import org.apache.hadoop.hbase.KeyValue.Type; import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceAudience.Private; import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.io.TagCompressionContext; @@ -200,14 +201,45 @@ public final class CellUtil { /********************* misc *************************************/ + @Private public static byte getRowByte(Cell cell, int index) { if (cell instanceof ByteBufferedCell) { - return ((ByteBufferedCell) cell).getRowByteBuffer().get( - ((ByteBufferedCell) cell).getRowPosition() + index); + return ((ByteBufferedCell) cell).getRowByteBuffer() + .get(((ByteBufferedCell) cell).getRowPosition() + index); } return cell.getRowArray()[cell.getRowOffset() + index]; } + @Private + public static byte getQualifierByte(Cell cell, int index) { + if (cell instanceof ByteBufferedCell) { + return ((ByteBufferedCell) cell).getQualifierByteBuffer().get( + ((ByteBufferedCell) cell).getQualifierPosition() + index); + } + return cell.getQualifierArray()[cell.getQualifierOffset() + index]; + } + + public static int getRowOffset(Cell cell) { + if (cell instanceof ByteBufferedCell) { + return ((ByteBufferedCell) cell).getRowPosition(); + } + return cell.getRowOffset(); + } + + public static int getQualifierOffset(Cell cell) { + if (cell instanceof ByteBufferedCell) { + return ((ByteBufferedCell) cell).getQualifierPosition(); + } + return cell.getQualifierOffset(); + } + + public static int getFamilyOffset(Cell cell) { + if (cell instanceof ByteBufferedCell) { + return (((ByteBufferedCell) cell).getFamilyPosition()); + } + return cell.getFamilyOffset(); + } + public static ByteBuffer getValueBufferShallowCopy(Cell cell) { ByteBuffer buffer = ByteBuffer.wrap(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()); @@ -1752,6 +1784,31 @@ public final class CellUtil { } /** + * Create a Cell that is smaller than all other possible Cells for the given Cell's rk and + * passed qualifier. Assumes family to be empty + * + * @param cell + * @param qArray + * @param qoffest + * @param qlength + * @return Last possible Cell on passed Cell's rk and passed qualifier. Assumes family to be empty + */ + @Private + public static Cell createFirstOnRowColWithEmptyFamily(final Cell cell) { + if (cell instanceof ByteBufferedCell) { + return new FirstOnRowColByteBufferedCell(((ByteBufferedCell) cell).getRowByteBuffer(), + ((ByteBufferedCell) cell).getRowPosition(), cell.getRowLength(), + HConstants.EMPTY_BYTE_BUFFER, 0, (byte) 0, + ((ByteBufferedCell) cell).getQualifierByteBuffer(), + ((ByteBufferedCell) cell).getQualifierPosition(), + ((ByteBufferedCell) cell).getQualifierLength()); + } + return new FirstOnRowColCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), + HConstants.EMPTY_BYTE_ARRAY, 0, (byte) 0, cell.getQualifierArray(), + cell.getQualifierOffset(), cell.getQualifierLength()); + } + + /** * Creates the first cell with the row/family/qualifier of this cell and the given timestamp. * Uses the "maximum" type that guarantees that the new cell is the lowest possible for this * combination of row, family, qualifier, and timestamp. This cell's own timestamp is ignored. 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 9909f19..6682e71 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 @@ -984,6 +984,21 @@ public final class ByteBufferUtils { } } + /** + * Similar to {@link Arrays#copyOfRange(byte[], int, int)} + * @param original the buffer from which the copy has to happen + * @param from the starting index + * @param to the ending index + * @return a byte[] created out of the copy + */ + 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..ecd8a32 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,7 +19,11 @@ package org.apache.hadoop.hbase.util; +import java.nio.ByteBuffer; + import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; @@ -37,6 +41,8 @@ public abstract class Hash { public static final int MURMUR_HASH = 1; /** Constant to denote {@link MurmurHash3}. */ public static final int MURMUR_HASH3 = 2; + // Assuming the cell that comes here is of KeyValue serialized + protected static final int KEY_OFFSET = 8; /** * This utility method converts String representation of hash function name @@ -141,5 +147,49 @@ public abstract class Hash { */ public abstract int hash(byte[] bytes, int offset, int length, int initval); - // TODO : a buffer based hash function would be needed.. Not adding it for now + /** + * 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); + + /** + * Calculate a hash using the Cell data with the provided seed value + * @param bloomCell the cell data + * @param initval seed value + * @return has value + */ + public abstract int hash(Cell cell, int initval); + + protected byte getByte(Cell data, int offset, int rowOffset, short rowLength, int qualOffset, + int qualLength, int famOffset, int famLen) { + // Always assume that this cell has keyvalue serialized key structure. + // TODO : Add Key interface to always assert the above key structure + if (offset <= KEY_OFFSET + Bytes.SIZEOF_SHORT - 1) { + byte b = (byte) rowLength; + if (offset - KEY_OFFSET == 0) { + rowLength >>= 8; + b = (byte) rowLength; + } + return b; + } + if (offset < (rowOffset + rowLength)) { + return CellUtil.getRowByte(data, offset - rowOffset); + } + if (offset == (famOffset - 1)) { + // The fam length should return 0 assuming there is no column family. + // Because for ROWCOL blooms family is not considered + return 0; + } + if (offset + famLen <= qualOffset + qualLength) { + return CellUtil.getQualifierByte(data, offset + famLen - qualOffset); + } + return 0; + // Processing ts and type may really not be needed + } } 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..0d9316a 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,7 +23,12 @@ import static java.lang.Integer.rotateLeft; import java.io.FileInputStream; import java.io.IOException; +import java.nio.ByteBuffer; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; @@ -239,6 +244,165 @@ 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. @@ -262,4 +426,123 @@ public class JenkinsHash extends Hash { } System.out.println(Math.abs(value)); } + + @Override + public int hash(Cell bloomCell, int initval) { + int famLen = bloomCell.getFamilyLength(); + // We don't consider the family length for ROWCOL bloom. So subtract the famLen from the + // length calculation. Timestamp and type are of no relevance here + int length = KeyValueUtil.keyLength(bloomCell) - KeyValue.TIMESTAMP_TYPE_SIZE - famLen; + int a, b, c; + a = b = c = 0xdeadbeef + length + initval; + int rowOffset = CellUtil.getRowOffset(bloomCell); + short rowLen = bloomCell.getRowLength(); + int qualOffset = CellUtil.getQualifierOffset(bloomCell); + int qualLen = bloomCell.getQualifierLength(); + int famOffset = CellUtil.getFamilyOffset(bloomCell); + int offset = KEY_OFFSET; + for (; length > 12; offset += 12, length -= 12) { + a += (getByte(bloomCell, offset, rowOffset, rowLen, qualOffset, qualLen, famOffset, famLen) + & BYTE_MASK); + a += ((getByte(bloomCell, offset + 1, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) & BYTE_MASK) << 8); + a += ((getByte(bloomCell, offset + 2, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) & BYTE_MASK) << 16); + a += ((getByte(bloomCell, offset + 3, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) & BYTE_MASK) << 24); + b += (getByte(bloomCell, offset + 4, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) & BYTE_MASK); + b += ((getByte(bloomCell, offset + 5, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) & BYTE_MASK) << 8); + b += ((getByte(bloomCell, offset + 6, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) & BYTE_MASK) << 16); + b += ((getByte(bloomCell, offset + 7, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) & BYTE_MASK) << 24); + c += (getByte(bloomCell, offset + 8, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) & BYTE_MASK); + c += ((getByte(bloomCell, offset + 9, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) & BYTE_MASK) << 8); + c += ((getByte(bloomCell, offset + 10, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) & BYTE_MASK) << 16); + c += ((getByte(bloomCell, offset + 11, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) & BYTE_MASK) << 24); + 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 += ((getByte(bloomCell, offset + 11, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) & BYTE_MASK) << 24); + case 11: + c += ((getByte(bloomCell, offset + 10, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) & BYTE_MASK) << 16); + case 10: + c += ((getByte(bloomCell, offset + 9, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) & BYTE_MASK) << 8); + case 9: + c += (getByte(bloomCell, offset + 8, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) & BYTE_MASK); + case 8: + b += ((getByte(bloomCell, offset + 7, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) & BYTE_MASK) << 24); + case 7: + b += ((getByte(bloomCell, offset + 6, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) & BYTE_MASK) << 16); + case 6: + b += ((getByte(bloomCell, offset + 5, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) & BYTE_MASK) << 8); + case 5: + b += (getByte(bloomCell, offset + 4, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) & BYTE_MASK); + case 4: + a += ((getByte(bloomCell, offset + 3, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) & BYTE_MASK) << 24); + case 3: + a += ((getByte(bloomCell, offset + 2, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) & BYTE_MASK) << 16); + case 2: + a += ((getByte(bloomCell, offset + 1, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) & BYTE_MASK) << 8); + case 1: + // noinspection PointlessArithmeticExpression + a += (getByte(bloomCell, offset + 0, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) & BYTE_MASK); + break; + case 0: + return c; + } + 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; + } } 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..f92763c 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,12 @@ package org.apache.hadoop.hbase.util; +import java.nio.ByteBuffer; + +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; @@ -38,6 +44,7 @@ public class MurmurHash extends Hash { return _instance; } + @Override public int hash(byte[] data, int offset, int length, int seed) { int m = 0x5bd1e995; @@ -89,4 +96,119 @@ 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; + } + + public int hash(Cell bloomCell, int seed) { + int famLen = bloomCell.getFamilyLength(); + // We don't consider the family length for ROWCOL bloom. So subtract the famLen from the + // length calculation. Timestamp and type are of no relevance here + int length = KeyValueUtil.keyLength(bloomCell) - KeyValue.TIMESTAMP_TYPE_SIZE - famLen; + int m = 0x5bd1e995; + int r = 24; + + int h = seed ^ length; + + int len_4 = length >> 2; + int rowOffset = CellUtil.getRowOffset(bloomCell); + short rowLen = bloomCell.getRowLength(); + int qualOffset = CellUtil.getQualifierOffset(bloomCell); + int qualLen = bloomCell.getQualifierLength(); + int famOffset = CellUtil.getFamilyOffset(bloomCell); + for (int i = 0; i < len_4; i++) { + int i_4 = (i << 2) + KEY_OFFSET; + int k = getByte(bloomCell, i_4 + 3, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen); + k = k << 8; + k = k | (getByte(bloomCell, i_4 + 2, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) & 0xff); + k = k << 8; + k = k | (getByte(bloomCell, i_4 + 1, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) & 0xff); + k = k << 8; + // noinspection PointlessArithmeticExpression + k = k | (getByte(bloomCell, i_4 + 0, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) & 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 + KEY_OFFSET; + + if (left != 0) { + if (left >= 3) { + h ^= getByte(bloomCell, i_m + 2, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) << 16; + } + if (left >= 2) { + h ^= getByte(bloomCell, i_m + 1, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) << 8; + } + if (left >= 1) { + h ^= getByte(bloomCell, i_m, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen); + } + 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..4d919b2 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,12 @@ */ package org.apache.hadoop.hbase.util; +import java.nio.ByteBuffer; + +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; @@ -90,4 +96,127 @@ 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; + } + + @Override + public int hash(Cell bloomCell, int initval) { + int famLen = bloomCell.getFamilyLength(); + // We don't consider the family length for ROWCOL bloom. So subtract the famLen from the + // length calculation. Timestamp and type are of no relevance here + int length = KeyValueUtil.keyLength(bloomCell) - KeyValue.TIMESTAMP_TYPE_SIZE - famLen; + final int c1 = 0xcc9e2d51; + final int c2 = 0x1b873593; + int rowOffset = CellUtil.getRowOffset(bloomCell); + short rowLen = bloomCell.getRowLength(); + int qualOffset = CellUtil.getQualifierOffset(bloomCell); + int qualLen = bloomCell.getQualifierLength(); + int famOffset = CellUtil.getFamilyOffset(bloomCell); + int h1 = initval; + int roundedEnd = KEY_OFFSET + (length & 0xfffffffc); // round down to 4 byte block + + for (int i = KEY_OFFSET; i < roundedEnd; i += 4) { + // little endian load order + int k1 = (getByte(bloomCell, i, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) & 0xff) + | ((getByte(bloomCell, i + 1, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) & 0xff) << 8) + | ((getByte(bloomCell, i + 2, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) & 0xff) << 16) + | (getByte(bloomCell, i + 3, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) << 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 = (getByte(bloomCell, roundedEnd + 2, rowOffset, rowLen, qualOffset, qualLen, + famOffset, famLen) & 0xff) << 16; + // FindBugs SF_SWITCH_FALLTHROUGH + case 2: + k1 |= (getByte(bloomCell, roundedEnd + 1, rowOffset, rowLen, qualOffset, qualLen, + famOffset, famLen) & 0xff) << 8; + // FindBugs SF_SWITCH_FALLTHROUGH + case 1: + k1 |= (getByte(bloomCell, roundedEnd, rowOffset, rowLen, qualOffset, qualLen, famOffset, + famLen) & 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/CompoundBloomFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundBloomFilter.java index 2d773bb..1d851a0 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 @@ -151,10 +151,10 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase // with BBs then the Hash.java APIs should also be changed to work with BBs. if (keyCell instanceof KeyValue) { return checkContains(((KeyValue) keyCell).getBuffer(), ((KeyValue) keyCell).getKeyOffset(), - ((KeyValue) keyCell).getKeyLength(), block); + ((KeyValue) keyCell).getKeyLength() - KeyValue.TIMESTAMP_TYPE_SIZE, block); } byte[] key = CellUtil.getCellKeySerializedAsKeyValueKey(keyCell); - return checkContains(key, 0, key.length, block); + return checkContains(key, 0, key.length - KeyValue.TIMESTAMP_TYPE_SIZE, block); } public boolean supportsAutoLoading() { 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..f287925 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; @@ -28,11 +29,14 @@ import java.util.Queue; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.CellUtil; 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; @@ -174,23 +178,29 @@ public class CompoundBloomFilterWriter extends CompoundBloomFilterBase firstKeyInChunk = Arrays.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(); - } + allocateNewChunk(); + } + + chunk.add(bloomKey, keyOffset, keyLength); + ++totalKeyCount; + } + + @Override + public void add(ByteBuffer bloomKey, int keyOffset, int keyLength) { + if (bloomKey == null) + throw new NullPointerException(); + + enqueueReadyChunk(false); - if (chunk.getKeyCount() != 0) { - throw new IllegalStateException("keyCount=" + chunk.getKeyCount() - + " > 0"); + 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); - chunk.allocBloom(); - ++numChunks; + allocateNewChunk(); } chunk.add(bloomKey, keyOffset, keyLength); @@ -198,6 +208,49 @@ public class CompoundBloomFilterWriter extends CompoundBloomFilterBase } @Override + public void add(Cell cell) { + if (cell == null) + throw new NullPointerException(); + + enqueueReadyChunk(false); + + if (chunk == null) { + if (firstKeyInChunk != null) { + throw new IllegalStateException("First key in chunk already set: " + + Bytes.toStringBinary(firstKeyInChunk)); + } + // This will be done only once per chunk + // TODO : Adding key interface should help here + firstKeyInChunk = CellUtil + .getCellKeySerializedAsKeyValueKey(CellUtil.createFirstOnRowColWithEmptyFamily(cell)); + + allocateNewChunk(); + } + + chunk.add(cell); + ++totalKeyCount; + } + + private void allocateNewChunk() { + 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; + } + @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 442b90d..e529d6d 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 @@ -18,7 +18,9 @@ */ package org.apache.hadoop.hbase.regionserver; -import com.google.common.base.Preconditions; +import java.io.IOException; +import java.net.InetSocketAddress; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -29,20 +31,20 @@ import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.regionserver.compactions.Compactor; +import org.apache.hadoop.hbase.util.BloomContext; import org.apache.hadoop.hbase.util.BloomFilterFactory; import org.apache.hadoop.hbase.util.BloomFilterWriter; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.RowBloomContext; +import org.apache.hadoop.hbase.util.RowColBloomContext; import org.apache.hadoop.io.WritableUtils; -import java.io.IOException; -import java.net.InetSocketAddress; -import java.util.Arrays; +import com.google.common.base.Preconditions; /** * A StoreFile writer. Use this to read/write HBase Store Files. It is package @@ -55,12 +57,10 @@ public class StoreFileWriter implements Compactor.CellSink { private final BloomFilterWriter generalBloomFilterWriter; private final BloomFilterWriter deleteFamilyBloomFilterWriter; private final BloomType bloomType; - private byte[] lastBloomKey; - private int lastBloomKeyOffset, lastBloomKeyLen; - private Cell lastCell = null; private long earliestPutTs = HConstants.LATEST_TIMESTAMP; private Cell lastDeleteFamilyCell = null; private long deleteFamilyCnt = 0; + private BloomContext bloomContext = null; /** * timeRangeTrackerSet is used to figure if we were passed a filled-out TimeRangeTracker or not. @@ -73,7 +73,6 @@ public class StoreFileWriter implements Compactor.CellSink { final TimeRangeTracker timeRangeTracker; protected HFile.Writer writer; - private KeyValue.KeyOnlyKeyValue lastBloomKeyOnlyKV = null; /** * Creates an HFile.Writer that also write helpful meta data. @@ -134,9 +133,6 @@ public class StoreFileWriter implements Compactor.CellSink { if (generalBloomFilterWriter != null) { this.bloomType = bloomType; - if(this.bloomType == BloomType.ROWCOL) { - lastBloomKeyOnlyKV = new KeyValue.KeyOnlyKeyValue(); - } if (LOG.isTraceEnabled()) { LOG.trace("Bloom filter type for " + path + ": " + this.bloomType + ", " + generalBloomFilterWriter.getClass().getSimpleName()); @@ -218,14 +214,27 @@ 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 + if (this.bloomContext == null) { + switch (bloomType) { + case ROW: + bloomContext = new RowBloomContext(generalBloomFilterWriter); + break; + case ROWCOL: + bloomContext = new RowColBloomContext(generalBloomFilterWriter); + break; + default: + throw new IOException( + "Invalid Bloom filter type: " + bloomType + " (ROW or ROWCOL expected)"); + } + } boolean newKey = true; - if (this.lastCell != null) { + if (this.bloomContext.getLastCell() != null) { switch(bloomType) { case ROW: - newKey = ! CellUtil.matchingRows(cell, lastCell); + newKey = ! CellUtil.matchingRows(cell, this.bloomContext.getLastCell()); break; case ROWCOL: - newKey = ! CellUtil.matchingRowColumn(cell, lastCell); + newKey = !CellUtil.matchingRowColumn(cell, this.bloomContext.getLastCell()); break; case NONE: newKey = false; @@ -244,60 +253,7 @@ public class StoreFileWriter implements Compactor.CellSink { * 1. Row = Row * 2. RowCol = Row + Qualifier */ - byte[] bloomKey = null; - // Used with ROW_COL bloom - KeyValue bloomKeyKV = null; - int bloomKeyOffset, bloomKeyLen; - - switch (bloomType) { - case ROW: - bloomKey = cell.getRowArray(); - bloomKeyOffset = cell.getRowOffset(); - bloomKeyLen = cell.getRowLength(); - break; - case ROWCOL: - // 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(); - 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: " - + 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; + bloomContext.processBloom(cell); } } } @@ -317,6 +273,7 @@ public class StoreFileWriter implements Compactor.CellSink { // of the hbase:meta cells newKey = !CellUtil.matchingRows(cell, lastDeleteFamilyCell); } + // TODO : Use bloom context for delete family bloom filter also if (newKey) { this.deleteFamilyBloomFilterWriter.add(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()); @@ -365,11 +322,7 @@ public class StoreFileWriter implements Compactor.CellSink { writer.addGeneralBloomFilter(generalBloomFilterWriter); writer.appendFileInfo(StoreFile.BLOOM_FILTER_TYPE_KEY, Bytes.toBytes(bloomType.toString())); - if (lastBloomKey != null) { - writer.appendFileInfo(StoreFile.LAST_BLOOM_KEY, Arrays.copyOfRange( - lastBloomKey, lastBloomKeyOffset, lastBloomKeyOffset - + lastBloomKeyLen)); - } + bloomContext.addLastBloomKey(writer); } return hasGeneralBloom; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomContext.java new file mode 100644 index 0000000..6501b4a --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomContext.java @@ -0,0 +1,62 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.util; + +import java.io.IOException; + +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.io.hfile.HFile; + +/** + * The bloom context that is used by the StorefileWriter to add the bloom details + * per cell + */ +@InterfaceAudience.Private +public abstract class BloomContext { + + protected Cell lastCell; + + protected BloomFilterWriter generalBloomFilterWriter; + + public BloomContext(BloomFilterWriter generalBloomFilterWriter) { + this.generalBloomFilterWriter = generalBloomFilterWriter; + } + + public Cell getLastCell() { + return this.lastCell; + } + + /** + * Bloom information from the cell is retrieved + * @param cell + * @throws IOException + */ + public abstract void processBloom(Cell cell) throws IOException; + + /** + * Adds the last bloom key to the HFile Writer as part of StorefileWriter close. + * @param writer + * @throws IOException + */ + public abstract void addLastBloomKey(HFile.Writer writer) throws IOException; + + protected void setLastCell(Cell lastCell) { + this.lastCell = lastCell; + } +} 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..2a7c879 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 @@ -24,6 +24,7 @@ import java.io.DataOutput; import java.io.IOException; import java.nio.ByteBuffer; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.classification.InterfaceAudience; /** @@ -173,6 +174,32 @@ public class BloomFilterChunk implements BloomFilterBase { int hash1 = this.hash.hash(buf, offset, len, 0); int hash2 = this.hash.hash(buf, offset, len, hash1); + setHashLoc(hash1, hash2); + } + + + public void add(Cell cell) { + /* + * For faster hashing, use combinatorial generation + * http://www.eecs.harvard.edu/~kirsch/pubs/bbbf/esa06.pdf + */ + int hash1 = this.hash.hash(cell, 0); + int hash2 = this.hash.hash(cell, hash1); + + setHashLoc(hash1, hash2); + } + 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); + + setHashLoc(hash1, hash2); + } + + private void setHashLoc(int hash1, int hash2) { for (int i = 0; i < this.hashCount; i++) { long hashLoc = Math.abs((hash1 + i * hash2) % (this.byteSize * 8)); set(hashLoc); 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..a3da5c5 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,9 @@ package org.apache.hadoop.hbase.util; +import java.nio.ByteBuffer; + +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.io.Writable; @@ -55,4 +58,18 @@ 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); + + /** + * Add the specified binary to the bloom filter. + * @param cell the cell data to be added to the bloom + */ + void add(Cell cell); + } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowBloomContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowBloomContext.java new file mode 100644 index 0000000..a9d8467 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowBloomContext.java @@ -0,0 +1,128 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.util; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; + +import org.apache.hadoop.hbase.ByteBufferedCell; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.io.hfile.HFile.Writer; +import org.apache.hadoop.hbase.regionserver.StoreFile; + +/** + * Handles ROW bloom related context. It works with both ByteBufferedCell and byte[] backed cells + */ +@InterfaceAudience.Private +public class RowBloomContext extends BloomContext { + + public RowBloomContext(BloomFilterWriter generalBloomFilterWriter) { + super(generalBloomFilterWriter); + } + // for byte[] backed cells + private byte[] lastBloomKey; + + private int lastBloomKeyOffset, lastBloomKeyLength; + + // for ByteBuffer backed cells + private ByteBuffer lastBloomKeyBuffer; + + /** + * Bloom information from the cell is retrieved + * @param cell + * @throws IOException + */ + public void processBloom(Cell cell) throws IOException { + byte[] bloomKey = null; + ByteBuffer bloomKeyBuffer = null; + if (cell instanceof ByteBufferedCell) { + bloomKeyBuffer = ((ByteBufferedCell) cell).getRowByteBuffer(); + } else { + bloomKey = cell.getRowArray(); + } + int bloomKeyOffset = CellUtil.getRowOffset(cell); + int bloomKeyLen = cell.getRowLength(); + int res = 0; + // Handle all four cases + if (bloomKey != null && lastBloomKey != null) { + res = Bytes.BYTES_RAWCOMPARATOR.compare(bloomKey, bloomKeyOffset, bloomKeyLen, lastBloomKey, + lastBloomKeyOffset, lastBloomKeyLength); + if (res < 0) { + throw new IOException("Non-increasing Bloom keys: " + + Bytes.toStringBinary(bloomKey, bloomKeyOffset, bloomKeyLen) + " after " + + Bytes.toStringBinary(lastBloomKey, lastBloomKeyOffset, lastBloomKeyLength)); + } + } else if (bloomKey != null && lastBloomKeyBuffer != null) { + res = ByteBufferUtils.compareTo(bloomKey, bloomKeyOffset, bloomKeyLen, lastBloomKeyBuffer, + lastBloomKeyOffset, lastBloomKeyLength); + if (res < 0) { + throw new IOException("Non-increasing Bloom keys: " + + Bytes.toStringBinary(bloomKey, bloomKeyOffset, bloomKeyLen) + " after " + + ByteBufferUtils.toStringBinary(lastBloomKeyBuffer, lastBloomKeyOffset, + lastBloomKeyLength)); + } + } else if (bloomKeyBuffer != null && lastBloomKey != null) { + res = ByteBufferUtils.compareTo(bloomKeyBuffer, bloomKeyOffset, bloomKeyLen, lastBloomKey, + lastBloomKeyOffset, lastBloomKeyLength); + if (res < 0) { + throw new IOException("Non-increasing Bloom keys: " + + ByteBufferUtils.toStringBinary(bloomKeyBuffer, bloomKeyOffset, bloomKeyLen) + + " after " + + Bytes.toStringBinary(lastBloomKey, lastBloomKeyOffset, lastBloomKeyLength)); + } + } else if (bloomKeyBuffer != null && lastBloomKeyBuffer != null) { + res = ByteBufferUtils.compareTo(bloomKeyBuffer, bloomKeyOffset, bloomKeyLen, + lastBloomKeyBuffer, lastBloomKeyOffset, lastBloomKeyLength); + if (res < 0) { + throw new IOException("Non-increasing Bloom keys: " + + ByteBufferUtils.toStringBinary(bloomKeyBuffer, bloomKeyOffset, bloomKeyLen) + + " after " + ByteBufferUtils.toStringBinary(lastBloomKeyBuffer, lastBloomKeyOffset, + lastBloomKeyLength)); + } + } + setContextParam(bloomKey, bloomKeyBuffer, bloomKeyOffset, bloomKeyLen); + if (bloomKey != null) { + generalBloomFilterWriter.add(bloomKey, bloomKeyOffset, bloomKeyLen); + } else { + generalBloomFilterWriter.add(bloomKeyBuffer, bloomKeyOffset, bloomKeyLen); + } + setLastCell(lastCell); + } + + private void setContextParam(byte[] bloomKey, ByteBuffer bloomKeyBuffer, int bloomKeyOffset, + int bloomKeyLen) { + lastBloomKey = bloomKey; + lastBloomKeyOffset = bloomKeyOffset; + lastBloomKeyLength = bloomKeyLen; + // either one will be set + lastBloomKeyBuffer = bloomKeyBuffer; + } + + public void addLastBloomKey(Writer writer) throws IOException { + if (lastBloomKeyBuffer != null) { + writer.appendFileInfo(StoreFile.LAST_BLOOM_KEY, ByteBufferUtils.copyOfRange( + lastBloomKeyBuffer, lastBloomKeyOffset, lastBloomKeyOffset + lastBloomKeyLength)); + } else if (lastBloomKey != null) { + writer.appendFileInfo(StoreFile.LAST_BLOOM_KEY, Arrays.copyOfRange(lastBloomKey, + lastBloomKeyOffset, lastBloomKeyOffset + lastBloomKeyLength)); + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowColBloomContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowColBloomContext.java new file mode 100644 index 0000000..8890806 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowColBloomContext.java @@ -0,0 +1,69 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.util; + +import java.io.IOException; +import java.util.Arrays; + +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.io.hfile.HFile.Writer; +import org.apache.hadoop.hbase.regionserver.StoreFile; + +/** + * Handles ROWCOL bloom related context. It can work with both BytebufferdCells + * and byte[] backed cells + */ +@InterfaceAudience.Private +public class RowColBloomContext extends BloomContext { + + private Cell lastBloomKeyOnlyKV ; + public RowColBloomContext(BloomFilterWriter generalBloomFilterWriter) { + super(generalBloomFilterWriter); + } + + @Override + public void processBloom(Cell cell) throws IOException { + Cell bloomKeyKV = CellUtil.createFirstOnRowColWithEmptyFamily(cell); + if (lastBloomKeyOnlyKV != null) { + int res = CellComparator.COMPARATOR.compare(bloomKeyKV, lastBloomKeyOnlyKV); + if (res < 0) { + byte[] key = CellUtil.getCellKeySerializedAsKeyValueKey(bloomKeyKV); + byte[] lastBloomKey = CellUtil.getCellKeySerializedAsKeyValueKey(lastBloomKeyOnlyKV); + throw new IOException( + "Non-increasing Bloom keys: " + Bytes.toStringBinary(key, 0, key.length) + " after " + + Bytes.toStringBinary(lastBloomKey, 0, lastBloomKey.length)); + } + } + generalBloomFilterWriter.add(cell); + lastBloomKeyOnlyKV = bloomKeyKV; + setLastCell(cell); + } + + @Override + public void addLastBloomKey(Writer writer) throws IOException { + if (this.lastCell != null) { + Cell firstOnRow = CellUtil.createFirstOnRowColWithEmptyFamily(this.lastCell); + // This copy happens only once when the writer is closed + byte[] key = CellUtil.getCellKeySerializedAsKeyValueKey(firstOnRow); + writer.appendFileInfo(StoreFile.LAST_BLOOM_KEY, Arrays.copyOfRange(key, 0, key.length)); + } + } +}