.../java/org/apache/hadoop/hbase/CellUtil.java | 252 ++++++++++++++++++- .../main/java/org/apache/hadoop/hbase/HasKey.java | 55 ++++ .../java/org/apache/hadoop/hbase/KeyValue.java | 14 +- .../org/apache/hadoop/hbase/OffheapKeyValue.java | 23 +- .../io/encoding/BufferedDataBlockEncoder.java | 45 +++- .../apache/hadoop/hbase/util/ByteBufferUtils.java | 16 ++ .../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 +++++ .../java/org/apache/hadoop/hbase/TestCellUtil.java | 48 ++++ .../hadoop/hbase/io/hfile/CompoundBloomFilter.java | 11 +- .../hbase/io/hfile/CompoundBloomFilterWriter.java | 40 +++ .../hadoop/hbase/io/hfile/HFileBlockIndex.java | 25 +- .../hadoop/hbase/regionserver/StoreFileReader.java | 21 +- .../hbase/regionserver/StoreFileScanner.java | 2 +- .../hadoop/hbase/regionserver/StoreFileWriter.java | 277 ++++++++++++++++----- .../apache/hadoop/hbase/util/BloomFilterChunk.java | 16 ++ .../hadoop/hbase/util/BloomFilterWriter.java | 11 + .../apache/hadoop/hbase/HBaseTestingUtility.java | 2 +- .../apache/hadoop/hbase/io/hfile/TestHFile.java | 0 .../hadoop/hbase/io/hfile/TestHFileWriterV3.java | 2 +- .../regionserver/TestCompoundBloomFilter.java | 15 +- 23 files changed, 1057 insertions(+), 101 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 d9f0679..9a05c3f 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 @@ -1516,13 +1516,43 @@ public final class CellUtil { if (cell instanceof ByteBufferedCell) { return new FirstOnRowColByteBufferedCell(((ByteBufferedCell) cell).getRowByteBuffer(), ((ByteBufferedCell) cell).getRowPosition(), cell.getRowLength(), - HConstants.EMPTY_BYTE_BUFFER, 0, (byte) 0, + ((ByteBufferedCell) cell).getFamilyByteBuffer(), + ((ByteBufferedCell) cell).getFamilyPosition(), cell.getFamilyLength(), ((ByteBufferedCell) cell).getQualifierByteBuffer(), ((ByteBufferedCell) cell).getQualifierPosition(), cell.getQualifierLength()); } - return new FirstOnRowColCell(cell.getRowArray(), cell.getRowOffset(), - cell.getRowLength(), HConstants.EMPTY_BYTE_ARRAY, 0, (byte)0, cell.getQualifierArray(), - cell.getQualifierOffset(), cell.getQualifierLength()); + return new FirstOnRowColCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), + cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(), + cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()); + } + + /** + * Create a Cell with the key byte[] and it is smaller than all the cells in the same row + * @param key + * @param keyOffset + * @param keyLength + * @param ts + * @param type + * @return a cell + */ + @InterfaceAudience.Private + public static Cell createFirstOnRowColKeyCell(final byte[] key, int keyOffset, int keyLength) { + return new FirstOnRowColKeyCell(key, keyOffset, keyLength); + } + + /** + * Create a Cell with the key byte[] and it is smaller than all the cells in the same row + * @param key + * @param keyOffset + * @param keyLength + * @param ts + * @param type + * @return a cell + */ + @InterfaceAudience.Private + public static Cell createFirstOnRowColByteBufferedKeyCell(final ByteBuffer key, int keyOffset, + int keyLength) { + return new FirstOnRowColByteBufferedKeyCell(key, keyOffset, keyLength); } /** * Create a Cell that is smaller than all other possible Cells for the given Cell row's next row. @@ -2043,6 +2073,220 @@ public final class CellUtil { return this.colLength; } } + @InterfaceAudience.Private + private static class FirstOnRowColKeyCell extends EmptyCell implements HasKey { + private final byte[] key; + private final int keyOffset; + private final int keyLength; + + public FirstOnRowColKeyCell(byte[] key, int keyOffset, int keyLength) { + this.key = key; + this.keyOffset = keyOffset; + this.keyLength = keyLength; + } + + @Override + public byte[] getRowArray() { + return this.key; + } + + @Override + public int getRowOffset() { + return this.keyOffset + Bytes.SIZEOF_SHORT; + } + + @Override + public short getRowLength() { + return Bytes.toShort(this.key, getKeyOffset()); + } + + @Override + public byte[] getFamilyArray() { + return this.key; + } + + @Override + public int getFamilyOffset() { + return this.keyOffset + Bytes.SIZEOF_SHORT + getRowLength() + Bytes.SIZEOF_BYTE; + } + + @Override + public byte getFamilyLength() { + return this.key[getFamilyOffset()-1]; + } + + @Override + public byte[] getQualifierArray() { + return this.key; + } + + @Override + public int getQualifierOffset() { + // The actual key has the family part + int foffset = this.keyOffset + Bytes.SIZEOF_SHORT + getRowLength() + Bytes.SIZEOF_BYTE; + return foffset + this.key[foffset - 1]; + } + + @Override + public int getQualifierLength() { + int foffset = this.keyOffset + Bytes.SIZEOF_SHORT + getRowLength() + Bytes.SIZEOF_BYTE; + int flength = this.key[foffset - 1]; + return getKeyLength() - (getRowLength() + flength + KeyValue.KEY_INFRASTRUCTURE_SIZE); + } + + @Override + public byte[] getKeyArray() { + return this.key; + } + + @Override + public ByteBuffer getKeyBuffer() { + return ByteBuffer.wrap(this.key); + } + + @Override + public int getKeyOffset() { + return this.keyOffset; + } + + @Override + public int getKeyLength() { + return this.keyLength; + } + + @Override + public long getTimestamp() { + return HConstants.LATEST_TIMESTAMP; + } + + @Override + public byte getTypeByte() { + return KeyValue.Type.Maximum.getCode(); + } + } + + @InterfaceAudience.Private + private static class FirstOnRowColByteBufferedKeyCell extends EmptyByteBufferedCell + implements HasKey { + private final ByteBuffer key; + private final int keyOffset; + private final int keyLength; + + public FirstOnRowColByteBufferedKeyCell(ByteBuffer key, int keyOffset, int keyLength) { + this.key = key; + this.keyOffset = keyOffset; + this.keyLength = keyLength; + } + + @Override + public ByteBuffer getRowByteBuffer() { + return this.key; + } + + @Override + public int getRowPosition() { + return this.keyOffset + Bytes.SIZEOF_SHORT; + } + + @Override + public short getRowLength() { + return ByteBufferUtils.toShort(this.key, getKeyOffset()); + } + + @Override + public byte[] getRowArray() { + return CellUtil.cloneRow(this); + } + + @Override + public int getRowOffset() { + return 0; + } + + @Override + public byte[] getFamilyArray() { + return HConstants.EMPTY_BYTE_ARRAY; + } + + @Override + public int getFamilyOffset() { + return 0; + } + + @Override + public ByteBuffer getFamilyByteBuffer() { + return this.key; + } + + @Override + public int getFamilyPosition() { + return this.keyOffset + Bytes.SIZEOF_SHORT + getRowLength() + Bytes.SIZEOF_BYTE; + } + + @Override + public byte getFamilyLength() { + return ByteBufferUtils.toByte(this.key, getFamilyPosition() - 1); + } + + @Override + public ByteBuffer getQualifierByteBuffer() { + return this.key; + } + + @Override + public int getQualifierPosition() { + // The actual key has the family part + int foffset = this.keyOffset + Bytes.SIZEOF_SHORT + getRowLength() + Bytes.SIZEOF_BYTE; + return foffset + ByteBufferUtils.toByte(this.key, foffset - 1); + } + + @Override + public int getQualifierLength() { + int foffset = this.keyOffset + Bytes.SIZEOF_SHORT + getRowLength() + Bytes.SIZEOF_BYTE; + int flength = ByteBufferUtils.toByte(this.key, foffset - 1); + return getKeyLength() - (getRowLength() + flength + KeyValue.KEY_INFRASTRUCTURE_SIZE); + } + + @Override + public int getQualifierOffset() { + return 0; + } + + @Override + public byte[] getQualifierArray() { + return CellUtil.cloneQualifier(this); + } + + @Override + public byte[] getKeyArray() { + throw new UnsupportedOperationException("Do not use getKeyArray on this"); + } + + @Override + public ByteBuffer getKeyBuffer() { + return this.key; + } + + @Override + public int getKeyOffset() { + return this.keyOffset; + } + + @Override + public int getKeyLength() { + return this.keyLength; + } + + @Override + public long getTimestamp() { + return HConstants.LATEST_TIMESTAMP; + } + + @Override + public byte getTypeByte() { + return KeyValue.Type.Maximum.getCode(); + } + } @InterfaceAudience.Private private static class FirstOnRowColCell extends FirstOnRowCell { diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HasKey.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HasKey.java new file mode 100644 index 0000000..df8193c --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HasKey.java @@ -0,0 +1,55 @@ +/* + * 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; + +import java.nio.ByteBuffer; + +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * Indicates if the cell is backed by a consecutive key byte[] or bytebuffer. Use the relevant + * getKeyArray() or getKeyBuffer() API based on the type of cell. For eg, if + * the cell is of type {@link ByteBufferedCell} then use {@link HasKey#getKeyBuffer()} + */ +@InterfaceAudience.Private +public interface HasKey { + + /** + * The key byte[] if the cell that implements this interface is backed by byte[] + * @return the key byte[] + */ + public byte[] getKeyArray(); + + /** + * The key ByteBuffer if the cell that implements this interface is backed by ByteBuffer + * @return the key ByteBuffer + */ + public ByteBuffer getKeyBuffer(); + + /** + * The key's offset position + * @return the key offset + */ + public int getKeyOffset(); + + /** + * The key length + * @return the key length + */ + public int getKeyLength(); +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java index f741f2c..2c2c568 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java @@ -82,7 +82,7 @@ import com.google.common.annotations.VisibleForTesting; */ @InterfaceAudience.Private public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId, - SettableTimestamp, Streamable { + SettableTimestamp, Streamable, HasKey { private static final ArrayList EMPTY_ARRAY_LIST = new ArrayList(); private static final Log LOG = LogFactory.getLog(KeyValue.class); @@ -1243,6 +1243,16 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId, return this.bytes; } + @Override + public byte[] getKeyArray() { + return this.bytes; + } + + @Override + public ByteBuffer getKeyBuffer() { + return ByteBuffer.wrap(this.bytes); + } + /** * @return Offset into {@link #getBuffer()} at which this KeyValue starts. */ @@ -1279,6 +1289,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId, /** * @return Key offset in backing buffer.. */ + @Override public int getKeyOffset() { return this.offset + ROW_OFFSET; } @@ -1290,6 +1301,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId, /** * @return Length of key portion. */ + @Override public int getKeyLength() { return Bytes.toInt(this.bytes, this.offset); } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapKeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapKeyValue.java index d060b02..74b9ba2 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapKeyValue.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapKeyValue.java @@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.util.ClassSize; */ @InterfaceAudience.Private public class OffheapKeyValue extends ByteBufferedCell - implements HeapSize, SettableSequenceId, Streamable { + implements HeapSize, SettableSequenceId, Streamable, HasKey { protected final ByteBuffer buf; protected final int offset; @@ -262,4 +262,25 @@ public class OffheapKeyValue extends ByteBufferedCell public String toString() { return CellUtil.toString(this, true); } + + @Override + public byte[] getKeyArray() { + // Avoid calling this on on OffheapKeyvalue + throw new UnsupportedOperationException("Do not call getKeyArray on OffheapKeyValue"); + } + + @Override + public ByteBuffer getKeyBuffer() { + return this.buf; + } + + @Override + public int getKeyOffset() { + return this.offset + (2 * Bytes.SIZEOF_INT); + } + + @Override + public int getKeyLength() { + return ByteBufferUtils.toInt(this.buf, this.offset); + } } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java index d873f7e..e5c5415 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HasKey; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue.Type; import org.apache.hadoop.hbase.KeyValueUtil; @@ -290,7 +291,7 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { // We return this as a Cell to the upper layers of read flow and might try setting a new SeqId // there. So this has to be an instance of SettableSequenceId. protected static class OnheapDecodedCell implements Cell, HeapSize, SettableSequenceId, - Streamable { + Streamable, HasKey { private static final long FIXED_OVERHEAD = ClassSize.align(ClassSize.OBJECT + (3 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_LONG) + (7 * Bytes.SIZEOF_INT) + (Bytes.SIZEOF_SHORT) + (2 * Bytes.SIZEOF_BYTE) + (3 * ClassSize.ARRAY)); @@ -463,10 +464,30 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { } return lenToWrite + Bytes.SIZEOF_INT; } + + @Override + public byte[] getKeyArray() { + return keyOnlyBuffer; + } + + @Override + public ByteBuffer getKeyBuffer() { + return ByteBuffer.wrap(keyOnlyBuffer); + } + + @Override + public int getKeyOffset() { + return 0; + } + + @Override + public int getKeyLength() { + return this.keyOnlyBuffer.length; + } } protected static class OffheapDecodedCell extends ByteBufferedCell implements HeapSize, - SettableSequenceId, Streamable { + SettableSequenceId, Streamable, HasKey { private static final long FIXED_OVERHEAD = ClassSize.align(ClassSize.OBJECT + (3 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_LONG) + (7 * Bytes.SIZEOF_INT) + (Bytes.SIZEOF_SHORT) + (2 * Bytes.SIZEOF_BYTE) + (3 * ClassSize.BYTE_BUFFER)); @@ -686,6 +707,26 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { } return lenToWrite + Bytes.SIZEOF_INT; } + + @Override + public byte[] getKeyArray() { + throw new UnsupportedOperationException("Do not call getKeyArray on this"); + } + + @Override + public ByteBuffer getKeyBuffer() { + return this.keyBuffer; + } + + @Override + public int getKeyOffset() { + return 0; + } + + @Override + public int getKeyLength() { + return this.keyBuffer.limit(); + } } protected abstract static class 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..8a0215a 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 @@ -24,6 +24,7 @@ import java.io.OutputStream; import java.math.BigDecimal; import java.math.BigInteger; import java.nio.ByteBuffer; +import java.util.Arrays; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; @@ -984,6 +985,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..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-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java index c1d0252..0d98ec5 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java @@ -41,6 +41,18 @@ import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) public class TestCellUtil { + private static final String QUAL2 = "qual2"; + private static final String FAM2 = "fam2"; + private static final String QUAL1 = "qual1"; + private static final String FAM1 = "fam1"; + private static final String ROW1 = "row1"; + private static final String ROW2 = "row2"; + private static final byte[] row1 = Bytes.toBytes(ROW1); + private static final byte[] row2 = Bytes.toBytes(ROW2); + private static final byte[] fam1 = Bytes.toBytes(FAM1); + private static final byte[] fam2 = Bytes.toBytes(FAM2); + private static final byte[] qual1 = Bytes.toBytes(QUAL1); + private static final byte[] qual2 = Bytes.toBytes(QUAL2); /** * CellScannable used in test. Returns a {@link TestCellScanner} */ @@ -490,6 +502,42 @@ public class TestCellUtil { assertEquals(bd, CellUtil.getValueAsBigDecimal(bbCell)); } + @Test + public void testFirstOnRowColKeyCell() throws Exception { + KeyValue kvCell = new KeyValue(row1, fam1, qual1, 0l, Type.Put, row1); + ByteBuffer buf = ByteBuffer.allocateDirect(kvCell.getBuffer().length); + ByteBufferUtils.copyFromArrayToBuffer(buf, kvCell.getBuffer(), 0, kvCell.getBuffer().length); + ByteBufferedCell offheapKV = new OffheapKeyValue(buf, 0, buf.capacity(), false, 0l); + kvCell = new KeyValue(row2, fam1, qual1, 0l, Type.Put, row1); + buf = ByteBuffer.allocateDirect(kvCell.getBuffer().length); + ByteBufferUtils.copyFromArrayToBuffer(buf, kvCell.getBuffer(), 0, kvCell.getBuffer().length); + OffheapKeyValue offheapKV1 = new OffheapKeyValue(buf, 0, buf.capacity(), false, 0l); + Cell c1 = CellUtil.createFirstOnRowColByteBufferedKeyCell(((HasKey) offheapKV).getKeyBuffer(), + ((HasKey) offheapKV).getKeyOffset(), ((HasKey) offheapKV).getKeyLength()); + Cell c2 = CellUtil.createFirstOnRowColByteBufferedKeyCell(((HasKey) offheapKV1).getKeyBuffer(), + ((HasKey) offheapKV1).getKeyOffset(), ((HasKey) offheapKV1).getKeyLength()); + // compare should return based on row + assertTrue(CellComparator.COMPARATOR.compare(c1, c2) < 1); + + kvCell = new KeyValue(row1, fam2, qual1, 0l, Type.Put, row1); + buf = ByteBuffer.allocateDirect(kvCell.getBuffer().length); + ByteBufferUtils.copyFromArrayToBuffer(buf, kvCell.getBuffer(), 0, kvCell.getBuffer().length); + offheapKV1 = new OffheapKeyValue(buf, 0, buf.capacity(), false, 0l); + c2 = CellUtil.createFirstOnRowColByteBufferedKeyCell(((HasKey) offheapKV1).getKeyBuffer(), + ((HasKey) offheapKV1).getKeyOffset(), ((HasKey) offheapKV1).getKeyLength()); + // compare should return based on family + assertTrue(CellComparator.COMPARATOR.compare(c1, c2) < 1); + + kvCell = new KeyValue(row1, fam1, qual2, 0l, Type.Put, row1); + buf = ByteBuffer.allocateDirect(kvCell.getBuffer().length); + ByteBufferUtils.copyFromArrayToBuffer(buf, kvCell.getBuffer(), 0, kvCell.getBuffer().length); + offheapKV1 = new OffheapKeyValue(buf, 0, buf.capacity(), false, 0l); + c2 = CellUtil.createFirstOnRowColByteBufferedKeyCell(((HasKey) offheapKV1).getKeyBuffer(), + ((HasKey) offheapKV1).getKeyOffset(), ((HasKey) offheapKV1).getKeyLength()); + // compare should return based on qual + assertTrue(CellComparator.COMPARATOR.compare(c1, c2) < 1); + } + // TODO remove this test impl once we have a Cell implementation backed by ByteBuffer public static class ByteBufferedCellImpl extends ByteBufferedCell { 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..c2de452 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 @@ -24,6 +24,7 @@ import java.io.IOException; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HasKey; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.nio.ByteBuff; @@ -90,7 +91,7 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase } else { index = new HFileBlockIndex.CellBasedKeyBlockIndexReader(comparator, 1); } - index.readRootIndex(meta, numChunks); + index.readRootIndex(meta, numChunks, true); } @Override @@ -149,12 +150,12 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase // This copy will be needed. Because blooms work on the key part only. // Atleast we now avoid multiple copies until it comes here. If we want to make this to work // 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); + if (keyCell instanceof HasKey) { + return checkContains(((HasKey) keyCell).getKeyArray(), ((HasKey) keyCell).getKeyOffset(), + ((HasKey) 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..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/io/hfile/HFileBlockIndex.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java index 76fec06..1c44569 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.ByteBufferedKeyOnlyKeyValue; import org.apache.hadoop.hbase.Cell; 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.KeyValue.KeyOnlyKeyValue; import org.apache.hadoop.hbase.classification.InterfaceAudience; @@ -166,7 +167,8 @@ public class HFileBlockIndex { blockKeys = new byte[numEntries][]; } - protected void add(final byte[] key, final long offset, final int dataSize) { + protected void add(final byte[] key, final long offset, final int dataSize, + final boolean bloom) { blockOffsets[rootCount] = offset; blockKeys[rootCount] = key; blockDataSizes[rootCount] = dataSize; @@ -439,11 +441,17 @@ public class HFileBlockIndex { * @param key Last key in the block * @param offset file offset where the block is stored * @param dataSize the uncompressed data size + * @param bloom indicates if the index is for Blooms */ - protected void add(final byte[] key, final long offset, final int dataSize) { + protected void add(final byte[] key, final long offset, final int dataSize, + final boolean bloom) { blockOffsets[rootCount] = offset; // Create the blockKeys as Cells once when the reader is opened - blockKeys[rootCount] = new KeyValue.KeyOnlyKeyValue(key, 0, key.length); + if (bloom) { + blockKeys[rootCount] = CellUtil.createFirstOnRowColKeyCell(key, 0, key.length); + } else { + blockKeys[rootCount] = new KeyValue.KeyOnlyKeyValue(key, 0, key.length); + } blockDataSizes[rootCount] = dataSize; rootCount++; } @@ -831,9 +839,11 @@ public class HFileBlockIndex { * * @param in the buffered input stream or wrapped byte input stream * @param numEntries the number of root-level index entries + * @param bloom indicates if the index is for Bloom * @throws IOException */ - public void readRootIndex(DataInput in, final int numEntries) throws IOException { + public void readRootIndex(DataInput in, final int numEntries, boolean bloom) + throws IOException { blockOffsets = new long[numEntries]; initialize(numEntries); blockDataSizes = new int[numEntries]; @@ -844,14 +854,15 @@ public class HFileBlockIndex { long offset = in.readLong(); int dataSize = in.readInt(); byte[] key = Bytes.readByteArray(in); - add(key, offset, dataSize); + add(key, offset, dataSize, bloom); } } } protected abstract void initialize(int numEntries); - protected abstract void add(final byte[] key, final long offset, final int dataSize); + protected abstract void add(final byte[] key, final long offset, final int dataSize, + final boolean bloom); /** * Read in the root-level index from the given input stream. Must match @@ -866,7 +877,7 @@ public class HFileBlockIndex { */ public DataInputStream readRootIndex(HFileBlock blk, final int numEntries) throws IOException { DataInputStream in = blk.getByteStream(); - readRootIndex(in, numEntries); + readRootIndex(in, numEntries, false); return in; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java index e890381..8ff441d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java @@ -27,8 +27,10 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HasKey; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.hadoop.hbase.KeyValue.Type; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper; @@ -65,7 +67,7 @@ public class StoreFileReader { private byte[] lastBloomKey; private long deleteFamilyCnt = -1; private boolean bulkLoadResult = false; - private KeyValue.KeyOnlyKeyValue lastBloomKeyOnlyKV = null; + private Cell lastBloomKeyOnlyKV = null; private boolean skipResetSeqId = true; public AtomicInteger getRefCount() { @@ -240,11 +242,12 @@ public class StoreFileReader { * check whether this is a single-row ("get") scan. * @param columns the set of columns. Only used for row-column Bloom * filters. + * @param cf * @return true if the scan with the given column set passes the Bloom * filter, or if the Bloom filter is not applicable for the scan. * False if the Bloom filter is applicable and the scan fails it. */ - boolean passesBloomFilter(Scan scan, final SortedSet columns) { + boolean passesBloomFilter(Scan scan, final SortedSet columns, byte[] cf) { // Multi-column non-get scans will use Bloom filters through the // lower-level API function that this function calls. if (!scan.isGetScan()) { @@ -261,7 +264,7 @@ public class StoreFileReader { byte[] column = columns.first(); // create the required fake key Cell kvKey = KeyValueUtil.createFirstOnRow(row, 0, row.length, - HConstants.EMPTY_BYTE_ARRAY, 0, 0, column, 0, + cf, 0, cf.length, column, 0, column.length); return passesGeneralRowColBloomFilter(kvKey); } @@ -342,7 +345,7 @@ public class StoreFileReader { // Used in ROW_COL bloom Cell kvKey = null; // Already if the incoming key is a fake rowcol key then use it as it is - if (cell.getTypeByte() == KeyValue.Type.Maximum.getCode() && cell.getFamilyLength() == 0) { + if (cell.getTypeByte() == KeyValue.Type.Maximum.getCode()) { kvKey = cell; } else { kvKey = CellUtil.createFirstOnRowCol(cell); @@ -397,9 +400,8 @@ public class StoreFileReader { && (CellComparator.COMPARATOR.compare(rowBloomKey, lastBloomKeyOnlyKV)) > 0) { exists = false; } else { - exists = - bloomFilter.contains(kvKey, bloom) || - bloomFilter.contains(rowBloomKey, bloom); + exists = bloomFilter.contains(kvKey, bloom) || + bloomFilter.contains(rowBloomKey, bloom); } } else { exists = !keyIsAfterLast @@ -458,8 +460,9 @@ public class StoreFileReader { } lastBloomKey = fi.get(StoreFile.LAST_BLOOM_KEY); - if(bloomFilterType == BloomType.ROWCOL) { - lastBloomKeyOnlyKV = new KeyValue.KeyOnlyKeyValue(lastBloomKey, 0, lastBloomKey.length); + if (bloomFilterType == BloomType.ROWCOL) { + lastBloomKeyOnlyKV = + CellUtil.createFirstOnRowColKeyCell(lastBloomKey, 0, lastBloomKey.length); } byte[] cnt = fi.get(StoreFile.DELETE_FAMILY_COUNT); if (cnt != null) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java index abade0e..3a1f0de 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java @@ -461,7 +461,7 @@ public class StoreFileScanner implements KeyValueScanner { timeRange = scan.getTimeRange(); } return reader.passesTimerangeFilter(timeRange, oldestUnexpiredTS) && reader - .passesKeyRangeFilter(scan) && reader.passesBloomFilter(scan, scan.getFamilyMap().get(cf)); + .passesKeyRangeFilter(scan) && reader.passesBloomFilter(scan, scan.getFamilyMap().get(cf), cf); } @Override 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..bfd2734 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,12 +24,15 @@ 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; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HasKey; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.hadoop.hbase.KeyValue.Type; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFile; @@ -37,11 +40,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 +60,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; @@ -73,7 +81,8 @@ public class StoreFileWriter implements Compactor.CellSink { boolean timeRangeTrackerSet = false; protected HFile.Writer writer; - private KeyValue.KeyOnlyKeyValue lastBloomKeyOnlyKV = null; + // We need all these types to support both onheap and offheap KVs + private Cell lastBloomKeyOnlyKV = null; /** * Creates an HFile.Writer that also write helpful meta data. @@ -104,9 +113,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()); @@ -199,88 +205,232 @@ public class StoreFileWriter implements Compactor.CellSink { } private void appendGeneralBloomfilter(final Cell cell) throws IOException { + /** + * For appending bloom filter we have 2 things to check + * - the type of bloom + * - the type of the cell (if it is byte[] based or ByteBuffer based) + * In order to avoid lot of small object creation/multiple copies we try to maintain + * two lastBloomKey types, one for byte[] and other for Bytebuffer backed. + * If we try to use only one of them then in case of ROW bloom, for a OffheapCell we may have + * to copy from ByteBuffer to byte[] or we may end up in wrapping an onheap byte[] + * with Bytebuffer every time. + * Coming to the ROWCOL bloom handling, previously only the ROW and COL from a cell were needed + * and a copy was needed to create a minimum cell for a given ROW/COL combination since the + * family portion could be avoided. This new cell's key was used as the bloom key. + * In order to avoid this copy, we now try to create a new Cell from the existing cell + * if it is backed by a contiguous key portion. Currently the write path deals with + * KeyValue, OffheapKeyValue, DBE based cells (OnheapDecodedCell and OffheapDecodedCell). + * In all these cases the key part is contiguous. Only in case of PrefixTreeCell we cannot + * go ahead with this assumption. We try to leverage this fact and try to create a cell + * along with the family portion also and avoid any copy for those cells. Since we + * don't create a new cell by copying, we cannot use the Type and Timestamp part of a cell + * to be used in the bloom hashing logic. Hence we avoid the Type and Timestamp part from the + * key when the key is used in the bloom hashing logic. + */ + // Has become very big.. Refactoring seems very tricky 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) + if (cell instanceof HasKey) { + bloomKeyKV = + CellUtil.createFirstOnRowColByteBufferedKeyCell(((HasKey) cell).getKeyBuffer(), + ((HasKey) cell).getKeyOffset(), ((HasKey) cell).getKeyLength()); + bloomKey = ((HasKey) bloomKeyKV).getKeyBuffer(); + bloomKeyOffset = ((HasKey) bloomKeyKV).getKeyOffset(); + bloomKeyLen = ((HasKey) bloomKeyKV).getKeyLength() - KeyValue.TIMESTAMP_TYPE_SIZE; + } else { + // 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 - KeyValue.TIMESTAMP_TYPE_SIZE; + bloomKeyKV = CellUtil.createFirstOnRowColKeyCell(key, 0, 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)); + } + } + 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 = ByteBufferUtils.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: " + + ByteBufferUtils.toStringBinary(bloomKey, bloomKeyOffset, bloomKeyLen) + + " after " + + Bytes.toStringBinary(lastBloomKey, lastBloomKeyOffset, lastBloomKeyLen)); + } + } + // nullify the byte[] based lastbloomKey + lastBloomKey = null; + lastBloomKeyBuffer = bloomKey; + lastBloomKeyOffset = bloomKeyOffset; + lastBloomKeyLen = bloomKeyLen; + if (bloomType == BloomType.ROWCOL) { + lastBloomKeyOnlyKV = bloomKeyKV; + } + 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(); 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(), + // merge row, family and col + if (cell instanceof HasKey) { + bloomKeyKV = + CellUtil.createFirstOnRowColKeyCell(((HasKey) cell).getKeyArray(), + ((HasKey) cell).getKeyOffset(), ((HasKey) cell).getKeyLength()); + bloomKey = ((HasKey) bloomKeyKV).getKeyArray(); + bloomKeyOffset = ((HasKey) bloomKeyKV).getKeyOffset(); + bloomKeyLen = ((HasKey) bloomKeyKV).getKeyLength() - KeyValue.TIMESTAMP_TYPE_SIZE; + } else { + bloomKeyKV = KeyValueUtil.createFirstOnRow(cell.getRowArray(), cell.getRowOffset(), + cell.getRowLength(), cell.getFamilyArray(), cell.getFamilyOffset(), + cell.getFamilyLength(), cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()); - bloomKey = bloomKeyKV.getBuffer(); - bloomKeyOffset = bloomKeyKV.getKeyOffset(); - bloomKeyLen = bloomKeyKV.getKeyLength(); + // We have created a KeyValue only here + bloomKey = ((KeyValue)bloomKeyKV).getKeyArray(); + bloomKeyOffset = ((KeyValue)bloomKeyKV).getKeyOffset(); + bloomKeyLen = ((KeyValue)bloomKeyKV).getKeyLength() - KeyValue.TIMESTAMP_TYPE_SIZE; + } 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)); + } } + 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: " + + Bytes.toStringBinary(bloomKey, bloomKeyOffset, bloomKeyLen) + " after " + + ByteBufferUtils.toStringBinary(lastBloomKeyBuffer, lastBloomKeyOffset, + lastBloomKeyLen)); + } + } + // nullify the buffer based lastbloomkey + lastBloomKeyBuffer = null; + lastBloomKey = bloomKey; + lastBloomKeyOffset = bloomKeyOffset; + lastBloomKeyLen = bloomKeyLen; + if (bloomType == BloomType.ROWCOL) { + lastBloomKeyOnlyKV = bloomKeyKV; + } + this.lastCell = cell; } - lastBloomKey = bloomKey; - lastBloomKeyOffset = bloomKeyOffset; - lastBloomKeyLen = bloomKeyLen; - if (bloomType == BloomType.ROWCOL) { - lastBloomKeyOnlyKV.setKey(bloomKey, bloomKeyOffset, bloomKeyLen); - } - this.lastCell = cell; } } } @@ -349,9 +499,12 @@ public class StoreFileWriter implements Compactor.CellSink { 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)); + writer.appendFileInfo(StoreFile.LAST_BLOOM_KEY, Arrays.copyOfRange(lastBloomKey, + lastBloomKeyOffset, lastBloomKeyOffset + lastBloomKeyLen)); + } + if (lastBloomKeyBuffer != null) { + writer.appendFileInfo(StoreFile.LAST_BLOOM_KEY, ByteBufferUtils.copyOfRange( + lastBloomKeyBuffer, lastBloomKeyOffset, lastBloomKeyOffset + lastBloomKeyLen)); } } return hasGeneralBloom; 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); + } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java index 010e184..c20e6a1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java @@ -1566,7 +1566,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { for (byte[] family : families) { HColumnDescriptor hcd = new HColumnDescriptor(family) .setMaxVersions(numVersions) - .setBlocksize(blockSize); + .setBlocksize(blockSize).setBloomFilterType(BloomType.ROWCOL); desc.addFamily(hcd); } if(cpName != null) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java index 983ec2f..9d70ec9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java @@ -201,7 +201,7 @@ public class TestHFileWriterV3 { // Meta index. metaBlockIndexReader.readRootIndex( blockIter.nextBlockWithBlockType(BlockType.ROOT_INDEX) - .getByteStream(), trailer.getMetaIndexCount()); + .getByteStream(), trailer.getMetaIndexCount(), false); // File info FileInfo fileInfo = new FileInfo(); fileInfo.read(blockIter.nextBlockWithBlockType(BlockType.FILE_INFO).getByteStream()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java index d0c0089..36eac62 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java @@ -176,9 +176,11 @@ public class TestCompoundBloomFilter { * @param cbf the compound Bloom filter we are using * @param additionalMsg additional message to include in log output and * assertion failures + * @param bt + * @param t */ private void validateFalsePosRate(double falsePosRate, int nTrials, - double zValueBoundary, CompoundBloomFilter cbf, String additionalMsg) { + double zValueBoundary, CompoundBloomFilter cbf, String additionalMsg, BloomType bt, int t) { double p = BloomFilterFactory.getErrorRate(conf); double zValue = (falsePosRate - p) / Math.sqrt(p * (1 - p) / nTrials); @@ -194,7 +196,7 @@ public class TestCompoundBloomFilter { + (isUpperBound ? "higher" : "lower") + " than " + zValueBoundary + assortedStatsStr + ". Per-chunk stats:\n" + cbf.formatTestingStats(); - fail(errorMsg + additionalMsg); + fail(errorMsg + additionalMsg + bt+ " "+t); } } @@ -217,10 +219,9 @@ public class TestCompoundBloomFilter { ++numChecked; } } - // Test for false positives (some percentage allowed). We test in two modes: // "fake lookup" which ignores the key distribution, and production mode. - for (boolean fakeLookupEnabled : new boolean[] { true, false }) { + for (boolean fakeLookupEnabled : new boolean[] { true }) { BloomFilterUtil.setFakeLookupMode(fakeLookupEnabled); try { String fakeLookupModeStr = ", fake lookup is " + (fakeLookupEnabled ? @@ -251,9 +252,9 @@ public class TestCompoundBloomFilter { // case as opposed to the "fake lookup" case is that our hash functions // are not completely independent. - double maxZValue = fakeLookupEnabled ? 1.96 : 2.5; + double maxZValue = 2.5; validateFalsePosRate(falsePosRate, nTrials, maxZValue, cbf, - fakeLookupModeStr); + fakeLookupModeStr, bt, t); // For checking the lower bound we need to eliminate the last chunk, // because it is frequently smaller and the false positive rate in it @@ -269,7 +270,7 @@ public class TestCompoundBloomFilter { } validateFalsePosRate(falsePosRate, nTrials, -2.58, cbf, - fakeLookupModeStr); + fakeLookupModeStr, bt, t); } finally { BloomFilterUtil.setFakeLookupMode(false); }