HBASE-15554 - StoreFile$Writer.appendGeneralBloomFilter generates extra KV (Ram)
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/285e1041 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/285e1041 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/285e1041 Branch: refs/heads/hbase-12439 Commit: 285e104176bf848dc81657ac5358ad3beb49cc57 Parents: fb668a0 Author: Ramkrishna <ramkrishna.s.vasude...@intel.com> Authored: Fri Aug 12 10:21:41 2016 +0530 Committer: Ramkrishna <ramkrishna.s.vasude...@intel.com> Committed: Fri Aug 12 10:21:41 2016 +0530 ---------------------------------------------------------------------- .../java/org/apache/hadoop/hbase/CellUtil.java | 30 ++++- .../hadoop/hbase/util/ByteArrayHashKey.java | 33 +++++ .../hadoop/hbase/util/ByteBufferUtils.java | 16 +++ .../apache/hadoop/hbase/util/CellHashKey.java | 37 ++++++ .../java/org/apache/hadoop/hbase/util/Hash.java | 17 ++- .../org/apache/hadoop/hbase/util/HashKey.java | 36 ++++++ .../apache/hadoop/hbase/util/JenkinsHash.java | 102 +++++++-------- .../apache/hadoop/hbase/util/MurmurHash.java | 18 +-- .../apache/hadoop/hbase/util/MurmurHash3.java | 14 +- .../hadoop/hbase/util/RowBloomHashKey.java | 35 +++++ .../hadoop/hbase/util/RowColBloomHashKey.java | 80 ++++++++++++ .../hbase/io/hfile/CompoundBloomFilter.java | 2 + .../io/hfile/CompoundBloomFilterWriter.java | 70 +++++----- .../hbase/regionserver/StoreFileWriter.java | 129 +++++-------------- .../apache/hadoop/hbase/util/BloomContext.java | 71 ++++++++++ .../hadoop/hbase/util/BloomFilterChunk.java | 45 ++++++- .../hadoop/hbase/util/BloomFilterFactory.java | 4 +- .../hadoop/hbase/util/BloomFilterUtil.java | 15 ++- .../hadoop/hbase/util/BloomFilterWriter.java | 9 +- .../hadoop/hbase/util/RowBloomContext.java | 52 ++++++++ .../hadoop/hbase/util/RowColBloomContext.java | 56 ++++++++ 21 files changed, 656 insertions(+), 215 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java ---------------------------------------------------------------------- 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..c6a0a93 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 @@ -26,6 +26,7 @@ import java.io.IOException; import java.math.BigDecimal; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Arrays; import java.util.Iterator; import java.util.List; import java.util.Map.Entry; @@ -33,6 +34,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; @@ -139,6 +141,22 @@ public final class CellUtil { return destinationOffset + rowLen; } + /** + * Copies the row to a new byte[] + * @param cell the cell from which row has to copied + * @return the byte[] containing the row + */ + public static byte[] copyRow(Cell cell) { + if (cell instanceof ByteBufferedCell) { + return ByteBufferUtils.copyOfRange(((ByteBufferedCell) cell).getRowByteBuffer(), + ((ByteBufferedCell) cell).getRowPosition(), + ((ByteBufferedCell) cell).getRowPosition() + cell.getRowLength()); + } else { + return Arrays.copyOfRange(cell.getRowArray(), cell.getRowOffset(), + cell.getRowOffset() + cell.getRowLength()); + } + } + public static int copyFamilyTo(Cell cell, byte[] destination, int destinationOffset) { byte fLen = cell.getFamilyLength(); if (cell instanceof ByteBufferedCell) { @@ -200,6 +218,7 @@ public final class CellUtil { /********************* misc *************************************/ + @Private public static byte getRowByte(Cell cell, int index) { if (cell instanceof ByteBufferedCell) { return ((ByteBufferedCell) cell).getRowByteBuffer().get( @@ -208,6 +227,15 @@ public final class CellUtil { 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 ByteBuffer getValueBufferShallowCopy(Cell cell) { ByteBuffer buffer = ByteBuffer.wrap(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()); @@ -1701,7 +1729,7 @@ public final class CellUtil { /** * Create a Cell that is smaller than all other possible Cells for the given Cell's row. - * + * The family length is considered to be 0 * @param cell * @return First possible Cell on passed Cell's row. */ http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteArrayHashKey.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteArrayHashKey.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteArrayHashKey.java new file mode 100644 index 0000000..1c6e369 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteArrayHashKey.java @@ -0,0 +1,33 @@ +/** + * 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 org.apache.hadoop.hbase.classification.InterfaceAudience; + +@InterfaceAudience.Private +public class ByteArrayHashKey extends HashKey<byte[]> { + + public ByteArrayHashKey(byte[] t) { + super(t); + } + + @Override + public byte get(int pos) { + return t[pos]; + } +} http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java ---------------------------------------------------------------------- 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..d788c70 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(); http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CellHashKey.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CellHashKey.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CellHashKey.java new file mode 100644 index 0000000..3c30cc6 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CellHashKey.java @@ -0,0 +1,37 @@ +/** + * 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 org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * Extracts the byte for the hash calculation from the given cell + */ +@InterfaceAudience.Private +public abstract class CellHashKey extends HashKey<Cell> { + + protected static final byte[] LATEST_TS = Bytes.toBytes(HConstants.LATEST_TIMESTAMP); + protected static final byte MAX_TYPE = KeyValue.Type.Maximum.getCode(); + + public CellHashKey(Cell cell) { + super(cell); + } +} http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Hash.java ---------------------------------------------------------------------- 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..dafda2a 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 @@ -139,7 +139,20 @@ public abstract class Hash { * @param initval seed value * @return hash value */ - public abstract int hash(byte[] bytes, int offset, int length, int initval); + // TODO : remove this once read path is updated to work with Cells + public int hash(byte[] bytes, int offset, int length, int initval) { + return hash(new ByteArrayHashKey(bytes), offset, length, initval); + } - // TODO : a buffer based hash function would be needed.. Not adding it for now + /** + * Calculate a hash using bytes from <code>offset</code> to <code>offset + + * length</code>, and the provided seed value. + * @param hashKey key to extract the hash + * @param offset offset to be used by the hash algo + * @param length length to be used by the hash algo + * @param initval the seed value + * @return hash value + */ + // TODO : Remove offset and length when the byte[] version of hash() is removed + public abstract int hash(HashKey hashKey, int offset, int length, int initval); } http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-common/src/main/java/org/apache/hadoop/hbase/util/HashKey.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/HashKey.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/HashKey.java new file mode 100644 index 0000000..83bba3c --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/HashKey.java @@ -0,0 +1,36 @@ +/** + * 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 org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * Used to calculate the hash {@link Hash} algorithms for Bloomfilters. + * + * @param <T> the type of HashKey + */ +@InterfaceAudience.Private +public abstract class HashKey<T> { + protected final T t; + + public HashKey(T t) { + this.t = t; + } + + public abstract byte get(int pos); +} http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-common/src/main/java/org/apache/hadoop/hbase/util/JenkinsHash.java ---------------------------------------------------------------------- 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..e874bcf 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 @@ -54,9 +54,33 @@ public class JenkinsHash extends Hash { } /** + * Compute the hash of the specified file + * @param args name of file to compute hash of. + * @throws IOException e + */ + public static void main(String[] args) throws IOException { + if (args.length != 1) { + System.err.println("Usage: JenkinsHash filename"); + System.exit(-1); + } + FileInputStream in = new FileInputStream(args[0]); + byte[] bytes = new byte[512]; + int value = 0; + JenkinsHash hash = new JenkinsHash(); + try { + for (int length = in.read(bytes); length > 0; length = in.read(bytes)) { + value = hash.hash(bytes, length, value); + } + } finally { + in.close(); + } + System.out.println(Math.abs(value)); + } + + /** * taken from hashlittle() -- hash a variable-length key into a 32-bit value * - * @param key the key (the unaligned variable-length array of bytes) + * @param hashKey the key to extract the bytes for hash algo * @param nbytes number of bytes to include in hash * @param initval can be any integer value * @return a 32-bit value. Every bit of the key affects every bit of the @@ -78,26 +102,26 @@ public class JenkinsHash extends Hash { * <p>Use for hash table lookup, or anything where one collision in 2^^32 is * acceptable. Do NOT use for cryptographic purposes. */ - @Override @SuppressWarnings("fallthrough") - public int hash(byte[] key, int off, int nbytes, int initval) { + @Override + public int hash(HashKey hashKey, 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 += (key[offset] & BYTE_MASK); - a += ((key[offset + 1] & BYTE_MASK) << 8); - a += ((key[offset + 2] & BYTE_MASK) << 16); - a += ((key[offset + 3] & BYTE_MASK) << 24); - b += (key[offset + 4] & BYTE_MASK); - b += ((key[offset + 5] & BYTE_MASK) << 8); - b += ((key[offset + 6] & BYTE_MASK) << 16); - b += ((key[offset + 7] & BYTE_MASK) << 24); - c += (key[offset + 8] & BYTE_MASK); - c += ((key[offset + 9] & BYTE_MASK) << 8); - c += ((key[offset + 10] & BYTE_MASK) << 16); - c += ((key[offset + 11] & BYTE_MASK) << 24); + a += (hashKey.get(offset) & BYTE_MASK); + a += ((hashKey.get(offset + 1) & BYTE_MASK) << 8); + a += ((hashKey.get(offset + 2) & BYTE_MASK) << 16); + a += ((hashKey.get(offset + 3) & BYTE_MASK) << 24); + b += (hashKey.get(offset + 4) & BYTE_MASK); + b += ((hashKey.get(offset + 5) & BYTE_MASK) << 8); + b += ((hashKey.get(offset + 6) & BYTE_MASK) << 16); + b += ((hashKey.get(offset + 7) & BYTE_MASK) << 24); + c += (hashKey.get(offset + 8) & BYTE_MASK); + c += ((hashKey.get(offset + 9) & BYTE_MASK) << 8); + c += ((hashKey.get(offset + 10) & BYTE_MASK) << 16); + c += ((hashKey.get(offset + 11) & BYTE_MASK) << 24); /* * mix -- mix 3 32-bit values reversibly. @@ -164,30 +188,30 @@ public class JenkinsHash extends Hash { //-------------------------------- last block: affect all 32 bits of (c) switch (length) { // all the case statements fall through case 12: - c += ((key[offset + 11] & BYTE_MASK) << 24); + c += ((hashKey.get(offset + 11) & BYTE_MASK) << 24); case 11: - c += ((key[offset + 10] & BYTE_MASK) << 16); + c += ((hashKey.get(offset + 10) & BYTE_MASK) << 16); case 10: - c += ((key[offset + 9] & BYTE_MASK) << 8); + c += ((hashKey.get(offset + 9) & BYTE_MASK) << 8); case 9: - c += (key[offset + 8] & BYTE_MASK); + c += (hashKey.get(offset + 8) & BYTE_MASK); case 8: - b += ((key[offset + 7] & BYTE_MASK) << 24); + b += ((hashKey.get(offset + 7) & BYTE_MASK) << 24); case 7: - b += ((key[offset + 6] & BYTE_MASK) << 16); + b += ((hashKey.get(offset + 6) & BYTE_MASK) << 16); case 6: - b += ((key[offset + 5] & BYTE_MASK) << 8); + b += ((hashKey.get(offset + 5) & BYTE_MASK) << 8); case 5: - b += (key[offset + 4] & BYTE_MASK); + b += (hashKey.get(offset + 4) & BYTE_MASK); case 4: - a += ((key[offset + 3] & BYTE_MASK) << 24); + a += ((hashKey.get(offset + 3) & BYTE_MASK) << 24); case 3: - a += ((key[offset + 2] & BYTE_MASK) << 16); + a += ((hashKey.get(offset + 2) & BYTE_MASK) << 16); case 2: - a += ((key[offset + 1] & BYTE_MASK) << 8); + a += ((hashKey.get(offset + 1) & BYTE_MASK) << 8); case 1: //noinspection PointlessArithmeticExpression - a += (key[offset + 0] & BYTE_MASK); + a += (hashKey.get(offset + 0) & BYTE_MASK); break; case 0: return c; @@ -238,28 +262,4 @@ public class JenkinsHash extends Hash { c ^= b; c -= rotateLeft(b, 24); return c; } - - /** - * Compute the hash of the specified file - * @param args name of file to compute hash of. - * @throws IOException e - */ - public static void main(String[] args) throws IOException { - if (args.length != 1) { - System.err.println("Usage: JenkinsHash filename"); - System.exit(-1); - } - FileInputStream in = new FileInputStream(args[0]); - byte[] bytes = new byte[512]; - int value = 0; - JenkinsHash hash = new JenkinsHash(); - try { - for (int length = in.read(bytes); length > 0; length = in.read(bytes)) { - value = hash.hash(bytes, length, value); - } - } finally { - in.close(); - } - System.out.println(Math.abs(value)); - } } http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-common/src/main/java/org/apache/hadoop/hbase/util/MurmurHash.java ---------------------------------------------------------------------- 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..86f0875 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 @@ -39,7 +39,7 @@ public class MurmurHash extends Hash { } @Override - public int hash(byte[] data, int offset, int length, int seed) { + public int hash(HashKey hashKey, int offset, int length, int seed) { int m = 0x5bd1e995; int r = 24; @@ -49,14 +49,14 @@ public class MurmurHash extends Hash { for (int i = 0; i < len_4; i++) { int i_4 = (i << 2) + offset; - int k = data[i_4 + 3]; + int k = hashKey.get(i_4 + 3); k = k << 8; - k = k | (data[i_4 + 2] & 0xff); + k = k | (hashKey.get(i_4 + 2) & 0xff); k = k << 8; - k = k | (data[i_4 + 1] & 0xff); + k = k | (hashKey.get(i_4 + 1) & 0xff); k = k << 8; - //noinspection PointlessArithmeticExpression - k = k | (data[i_4 + 0] & 0xff); + // noinspection PointlessArithmeticExpression + k = k | (hashKey.get(i_4 + 0) & 0xff); k *= m; k ^= k >>> r; k *= m; @@ -71,13 +71,13 @@ public class MurmurHash extends Hash { if (left != 0) { if (left >= 3) { - h ^= data[i_m + 2] << 16; + h ^= hashKey.get(i_m + 2) << 16; } if (left >= 2) { - h ^= data[i_m + 1] << 8; + h ^= hashKey.get(i_m + 1) << 8; } if (left >= 1) { - h ^= data[i_m]; + h ^= hashKey.get(i_m); } h *= m; http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-common/src/main/java/org/apache/hadoop/hbase/util/MurmurHash3.java ---------------------------------------------------------------------- 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..22a23ae 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 @@ -40,7 +40,7 @@ public class MurmurHash3 extends Hash { /** Returns the MurmurHash3_x86_32 hash. */ @edu.umd.cs.findbugs.annotations.SuppressWarnings("SF") @Override - public int hash(byte[] bytes, int offset, int length, int initval) { + public int hash(HashKey hashKey, int offset, int length, int initval) { final int c1 = 0xcc9e2d51; final int c2 = 0x1b873593; @@ -49,8 +49,10 @@ public class MurmurHash3 extends Hash { for (int i = offset; i < roundedEnd; i += 4) { // little endian load order - int k1 = (bytes[i] & 0xff) | ((bytes[i + 1] & 0xff) << 8) | ((bytes[i + 2] & 0xff) << 16) - | (bytes[i + 3] << 24); + int k1 = + (hashKey.get(i) & 0xff) | ((hashKey.get(i + 1) & 0xff) << 8) + | ((hashKey.get(i + 2) & 0xff) << 16) + | (hashKey.get(i + 3) << 24); k1 *= c1; k1 = (k1 << 15) | (k1 >>> 17); // ROTL32(k1,15); k1 *= c2; @@ -65,13 +67,13 @@ public class MurmurHash3 extends Hash { switch (length & 0x03) { case 3: - k1 = (bytes[roundedEnd + 2] & 0xff) << 16; + k1 = (hashKey.get(roundedEnd + 2) & 0xff) << 16; // FindBugs SF_SWITCH_FALLTHROUGH case 2: - k1 |= (bytes[roundedEnd + 1] & 0xff) << 8; + k1 |= (hashKey.get(roundedEnd + 1) & 0xff) << 8; // FindBugs SF_SWITCH_FALLTHROUGH case 1: - k1 |= (bytes[roundedEnd] & 0xff); + k1 |= (hashKey.get(roundedEnd) & 0xff); k1 *= c1; k1 = (k1 << 15) | (k1 >>> 17); // ROTL32(k1,15); k1 *= c2; http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-common/src/main/java/org/apache/hadoop/hbase/util/RowBloomHashKey.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/RowBloomHashKey.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/RowBloomHashKey.java new file mode 100644 index 0000000..5253bf9 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/RowBloomHashKey.java @@ -0,0 +1,35 @@ +/** + * 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 org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +@InterfaceAudience.Private +public class RowBloomHashKey extends CellHashKey { + + public RowBloomHashKey(Cell cell) { + super(cell); + } + + @Override + public byte get(int offset) { + return CellUtil.getRowByte(t, offset); + } +} http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-common/src/main/java/org/apache/hadoop/hbase/util/RowColBloomHashKey.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/RowColBloomHashKey.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/RowColBloomHashKey.java new file mode 100644 index 0000000..c35f307 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/RowColBloomHashKey.java @@ -0,0 +1,80 @@ +/** + * 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 org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * An hash key for ROWCOL bloom. This assumes the cells to be serialized in the Keyvalue + * serialization format with Empty column family. Note that the byte representing the family length + * is considered to be 0 + */ +@InterfaceAudience.Private +public class RowColBloomHashKey extends CellHashKey { + + private final int rowLength; + private final int qualLength; + + public RowColBloomHashKey(Cell cell) { + super(cell); + rowLength = cell.getRowLength(); + // 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 + qualLength = cell.getQualifierLength(); + } + + @Override + public byte get(int offset) { + // Always assume that this cell has keyvalue serialized key structure. + // rk len + row key + 0 byte for family length + qual + ts + type + if (offset < Bytes.SIZEOF_SHORT) { + // assign locally + int rowlen = rowLength; + byte b = (byte) rowlen; + if (offset == 0) { + rowlen >>= 8; + b = (byte) rowlen; + } + return b; + } + int refLen = Bytes.SIZEOF_SHORT + rowLength; + if (offset < refLen) { + return CellUtil.getRowByte(t, offset - Bytes.SIZEOF_SHORT); + } + if (offset == refLen) { + // The fam length should return 0 assuming there is no column family. + // Because for ROWCOL blooms family is not considered + return 0; + } + refLen += qualLength + Bytes.SIZEOF_BYTE; + // skip the family len because actual cells may have family also + if (offset < refLen) { + return CellUtil.getQualifierByte(t, + offset - (Bytes.SIZEOF_SHORT + rowLength + Bytes.SIZEOF_BYTE)); + } + // TODO : check if ts and type can be removed + refLen += KeyValue.TIMESTAMP_SIZE; + if (offset < refLen) { + return LATEST_TS[offset - (Bytes.SIZEOF_SHORT + rowLength + qualLength + Bytes.SIZEOF_BYTE)]; + } + return MAX_TYPE; + } +} http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundBloomFilter.java ---------------------------------------------------------------------- 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..79e8e0c 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 @@ -150,9 +150,11 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase // 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) { + // TODO : directly use Cell here return checkContains(((KeyValue) keyCell).getBuffer(), ((KeyValue) keyCell).getKeyOffset(), ((KeyValue) keyCell).getKeyLength(), block); } + // TODO : Avoid this copy in read path also byte[] key = CellUtil.getCellKeySerializedAsKeyValueKey(keyCell); return checkContains(key, 0, key.length, block); } http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundBloomFilterWriter.java ---------------------------------------------------------------------- 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..3193a17 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,15 +21,16 @@ package org.apache.hadoop.hbase.io.hfile; import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; -import java.util.Arrays; import java.util.LinkedList; 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.regionserver.StoreFileWriter; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.util.BloomFilterChunk; import org.apache.hadoop.hbase.util.BloomFilterUtil; import org.apache.hadoop.hbase.util.BloomFilterWriter; @@ -78,6 +79,8 @@ public class CompoundBloomFilterWriter extends CompoundBloomFilterBase /** Whether to cache-on-write compound Bloom filter chunks */ private boolean cacheOnWrite; + private BloomType bloomType; + /** * @param chunkByteSizeHint * each chunk's size in bytes. The real chunk size might be different @@ -88,10 +91,12 @@ public class CompoundBloomFilterWriter extends CompoundBloomFilterBase * hash function type to use * @param maxFold * maximum degree of folding allowed + * @param bloomType + * the bloom type */ public CompoundBloomFilterWriter(int chunkByteSizeHint, float errorRate, int hashType, int maxFold, boolean cacheOnWrite, - CellComparator comparator) { + CellComparator comparator, BloomType bloomType) { chunkByteSize = BloomFilterUtil.computeFoldableByteSize( chunkByteSizeHint * 8L, maxFold); @@ -100,6 +105,7 @@ public class CompoundBloomFilterWriter extends CompoundBloomFilterBase this.maxFold = maxFold; this.cacheOnWrite = cacheOnWrite; this.comparator = comparator; + this.bloomType = bloomType; } @Override @@ -152,16 +158,9 @@ public class CompoundBloomFilterWriter extends CompoundBloomFilterBase chunk = null; } - /** - * Adds a Bloom filter key. This key must be greater than the previous key, - * as defined by the comparator this compound Bloom filter is configured - * with. For efficiency, key monotonicity is not checked here. See - * {@link StoreFileWriter#append( - * org.apache.hadoop.hbase.Cell)} for the details of deduplication. - */ @Override - public void add(byte[] bloomKey, int keyOffset, int keyLength) { - if (bloomKey == null) + public void add(Cell cell) { + if (cell == null) throw new NullPointerException(); enqueueReadyChunk(false); @@ -171,32 +170,39 @@ public class CompoundBloomFilterWriter extends CompoundBloomFilterBase throw new IllegalStateException("First key in chunk already set: " + Bytes.toStringBinary(firstKeyInChunk)); } - firstKeyInChunk = Arrays.copyOfRange(bloomKey, keyOffset, keyOffset - + keyLength); - - if (prevChunk == null) { - // First chunk - chunk = BloomFilterUtil.createBySize(chunkByteSize, errorRate, - hashType, maxFold); + // This will be done only once per chunk + if (bloomType == BloomType.ROW) { + firstKeyInChunk = CellUtil.copyRow(cell); } 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"); + firstKeyInChunk = + CellUtil.getCellKeySerializedAsKeyValueKey(CellUtil.createFirstOnRowCol(cell)); } - - chunk.allocBloom(); - ++numChunks; + allocateNewChunk(); } - chunk.add(bloomKey, keyOffset, keyLength); + chunk.add(cell); ++totalKeyCount; } + private void allocateNewChunk() { + if (prevChunk == null) { + // First chunk + chunk = BloomFilterUtil.createBySize(chunkByteSize, errorRate, + hashType, maxFold, bloomType); + } 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 http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java ---------------------------------------------------------------------- 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..47a8b25 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,87 +214,30 @@ public class StoreFileWriter implements Compactor.CellSink { private void appendGeneralBloomfilter(final Cell cell) throws IOException { if (this.generalBloomFilterWriter != null) { // only add to the bloom filter on a new, unique key - boolean newKey = true; - 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)"); - } - } - 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; - + if (this.bloomContext == null) { + // init bloom context 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)"); + 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)"); } - 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; } + + /* + * 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 + */ + bloomContext.writeBloom(cell); } } @@ -317,9 +256,9 @@ 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()); + this.deleteFamilyBloomFilterWriter.add(cell); this.lastDeleteFamilyCell = cell; } } @@ -365,11 +304,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; } http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomContext.java ---------------------------------------------------------------------- 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..aab30e0 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomContext.java @@ -0,0 +1,71 @@ +/** + * 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 { + + // TODO : Avoid holding references to lastCell + 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 void writeBloom(Cell cell) throws IOException { + if (isNewKey(cell)) { + generalBloomFilterWriter.add(cell); + this.lastCell = cell; + } + } + + /** + * 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; + + /** + * Returns true if the cell is a new key as per the bloom type + * @param cell the cell to be verified + * @return true if a new key else false + */ + protected abstract boolean isNewKey(Cell cell); +} http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterChunk.java ---------------------------------------------------------------------- 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..de09938 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,7 +24,10 @@ import java.io.DataOutput; import java.io.IOException; import java.nio.ByteBuffer; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.regionserver.BloomType; /** * The basic building block for the {@link org.apache.hadoop.hbase.io.hfile.CompoundBloomFilter} @@ -46,6 +49,8 @@ public class BloomFilterChunk implements BloomFilterBase { protected int maxKeys; /** Bloom bits */ protected ByteBuffer bloom; + /** The type of bloom */ + protected BloomType bloomType; /** * Loads bloom filter meta data from file input. @@ -80,9 +85,10 @@ public class BloomFilterChunk implements BloomFilterBase { return BloomFilterUtil.actualErrorRate(keyCount, byteSize * 8, hashCount); } - public BloomFilterChunk(int hashType) { + public BloomFilterChunk(int hashType, BloomType bloomType) { this.hashType = hashType; this.hash = Hash.getInstance(hashType); + this.bloomType = bloomType; } /** @@ -100,9 +106,10 @@ public class BloomFilterChunk implements BloomFilterBase { * than maxKeys. * @throws IllegalArgumentException */ + // Used only in testcases public BloomFilterChunk(int maxKeys, double errorRate, int hashType, int foldFactor) throws IllegalArgumentException { - this(hashType); + this(hashType, BloomType.ROW); long bitSize = BloomFilterUtil.computeBitSize(maxKeys, errorRate); hashCount = BloomFilterUtil.optimalFunctionCount(maxKeys, bitSize); @@ -121,7 +128,7 @@ public class BloomFilterChunk implements BloomFilterBase { * @return a Bloom filter with the same configuration as this */ public BloomFilterChunk createAnother() { - BloomFilterChunk bbf = new BloomFilterChunk(hashType); + BloomFilterChunk bbf = new BloomFilterChunk(hashType, this.bloomType); bbf.byteSize = byteSize; bbf.hashCount = hashCount; bbf.maxKeys = maxKeys; @@ -173,6 +180,38 @@ 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; + int hash2; + HashKey<Cell> hashKey; + if (this.bloomType == BloomType.ROW) { + // TODO : Move this length to the HashKey when we do the read path to work with + // extractor so that the byte[] version of hash() function is removed + int length = cell.getRowLength(); + hashKey = new RowBloomHashKey(cell); + hash1 = this.hash.hash(hashKey, 0, length, 0); + hash2 = this.hash.hash(hashKey, 0, length, hash1); + } else { + int famLen = cell.getFamilyLength(); + // TODO : Move this length to the HashKey when we do the read path to work with + // extractor so that the byte[] version of hash() function is removed + int length = KeyValueUtil.keyLength(cell) - famLen; + hashKey = new RowColBloomHashKey(cell); + hash1 = this.hash.hash(hashKey, 0, length, 0); + hash2 = this.hash.hash(hashKey, 0, length, 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); http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java index 99951f0..22d6fe1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java @@ -196,7 +196,7 @@ public final class BloomFilterFactory { // In case of compound Bloom filters we ignore the maxKeys hint. CompoundBloomFilterWriter bloomWriter = new CompoundBloomFilterWriter(getBloomBlockSize(conf), err, Hash.getHashType(conf), maxFold, cacheConf.shouldCacheBloomsOnWrite(), - bloomType == BloomType.ROWCOL ? CellComparator.COMPARATOR : null); + bloomType == BloomType.ROWCOL ? CellComparator.COMPARATOR : null, bloomType); writer.addInlineBlockWriter(bloomWriter); return bloomWriter; } @@ -227,7 +227,7 @@ public final class BloomFilterFactory { // In case of compound Bloom filters we ignore the maxKeys hint. CompoundBloomFilterWriter bloomWriter = new CompoundBloomFilterWriter(getBloomBlockSize(conf), err, Hash.getHashType(conf), maxFold, cacheConf.shouldCacheBloomsOnWrite(), - null); + null, BloomType.ROW); writer.addInlineBlockWriter(bloomWriter); return bloomWriter; } http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterUtil.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterUtil.java index 08ed753..4b4e87d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterUtil.java @@ -22,6 +22,7 @@ import java.util.Random; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.nio.ByteBuff; +import org.apache.hadoop.hbase.regionserver.BloomType; /** * Utility methods related to BloomFilters @@ -173,11 +174,12 @@ public final class BloomFilterUtil { * @param errorRate target false positive rate of the Bloom filter * @param hashType Bloom filter hash function type * @param foldFactor + * @param bloomType * @return the new Bloom filter of the desired size */ public static BloomFilterChunk createBySize(int byteSizeHint, - double errorRate, int hashType, int foldFactor) { - BloomFilterChunk bbf = new BloomFilterChunk(hashType); + double errorRate, int hashType, int foldFactor, BloomType bloomType) { + BloomFilterChunk bbf = new BloomFilterChunk(hashType, bloomType); bbf.byteSize = computeFoldableByteSize(byteSizeHint * 8L, foldFactor); long bitSize = bbf.byteSize * 8; @@ -195,11 +197,12 @@ public final class BloomFilterUtil { public static boolean contains(byte[] buf, int offset, int length, ByteBuff bloomBuf, int bloomOffset, int bloomSize, Hash hash, int hashCount) { - - int hash1 = hash.hash(buf, offset, length, 0); - int hash2 = hash.hash(buf, offset, length, hash1); + // TODO : this will get removed once read path also work with Cell for blooms. + ByteArrayHashKey hashKey = new ByteArrayHashKey(buf); + int hash1 = hash.hash(hashKey, offset, length, 0); + int hash2 = hash.hash(hashKey, offset, length, hash1); int bloomBitSize = bloomSize << 3; - + if (randomGeneratorForTest == null) { // Production mode. int compositeHash = hash1; http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterWriter.java ---------------------------------------------------------------------- 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..32a9ff4 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,7 @@ package org.apache.hadoop.hbase.util; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.io.Writable; @@ -48,11 +49,7 @@ public interface BloomFilterWriter extends BloomFilterBase { /** * 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 + * @param cell the cell data to be added to the bloom */ - void add(byte[] buf, int offset, int len); - + void add(Cell cell); } http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowBloomContext.java ---------------------------------------------------------------------- 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..c61e840 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowBloomContext.java @@ -0,0 +1,52 @@ +/** + * 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.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); + } + + public void addLastBloomKey(Writer writer) throws IOException { + if (lastCell != null) { + byte[] key = CellUtil.copyRow(this.lastCell); + writer.appendFileInfo(StoreFile.LAST_BLOOM_KEY, key); + } + } + + @Override + protected boolean isNewKey(Cell cell) { + if (this.lastCell != null) { + return !CellUtil.matchingRows(cell, this.lastCell); + } + return true; + } +} http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowColBloomContext.java ---------------------------------------------------------------------- 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..c1b47af --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowColBloomContext.java @@ -0,0 +1,56 @@ +/** + * 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.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 { + + public RowColBloomContext(BloomFilterWriter generalBloomFilterWriter) { + super(generalBloomFilterWriter); + } + + @Override + public void addLastBloomKey(Writer writer) throws IOException { + if (this.lastCell != null) { + Cell firstOnRow = CellUtil.createFirstOnRowCol(this.lastCell); + // This copy happens only once when the writer is closed + byte[] key = CellUtil.getCellKeySerializedAsKeyValueKey(firstOnRow); + writer.appendFileInfo(StoreFile.LAST_BLOOM_KEY, key); + } + } + + @Override + protected boolean isNewKey(Cell cell) { + if (this.lastCell != null) { + return !CellUtil.matchingRowColumn(cell, this.lastCell); + } + return true; + } +}