HBASE-16405 Change read path Bloom check to work with Cells with out any copy.
This commit is contained in:
parent
ce88270340
commit
5f61f9cc5c
@ -36,8 +36,10 @@ import org.apache.hadoop.hbase.master.RegionState;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo;
|
||||
import org.apache.hadoop.hbase.util.ByteArrayHashKey;
|
||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.HashKey;
|
||||
import org.apache.hadoop.hbase.util.JenkinsHash;
|
||||
import org.apache.hadoop.hbase.util.MD5Hash;
|
||||
import org.apache.hadoop.io.DataInputBuffer;
|
||||
@ -152,8 +154,8 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
|
||||
} else {
|
||||
// old format region name. First hbase:meta region also
|
||||
// use this format.EncodedName is the JenkinsHash value.
|
||||
int hashVal = Math.abs(JenkinsHash.getInstance().hash(regionName,
|
||||
regionName.length, 0));
|
||||
HashKey<byte[]> key = new ByteArrayHashKey(regionName, 0, regionName.length);
|
||||
int hashVal = Math.abs(JenkinsHash.getInstance().hash(key, 0));
|
||||
encodedName = String.valueOf(hashVal);
|
||||
}
|
||||
return encodedName;
|
||||
|
@ -22,12 +22,26 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
@InterfaceAudience.Private
|
||||
public class ByteArrayHashKey extends HashKey<byte[]> {
|
||||
|
||||
public ByteArrayHashKey(byte[] t) {
|
||||
private final int offset;
|
||||
private final int length;
|
||||
|
||||
public ByteArrayHashKey(byte[] t, int offset, int length) {
|
||||
super(t);
|
||||
this.offset = offset;
|
||||
this.length = length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte get(int pos) {
|
||||
return t[pos];
|
||||
return t[getAbsolutePos(pos)];
|
||||
}
|
||||
|
||||
private int getAbsolutePos(int pos) {
|
||||
return this.offset + pos;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int length() {
|
||||
return this.length;
|
||||
}
|
||||
}
|
||||
|
@ -98,61 +98,11 @@ public abstract class Hash {
|
||||
}
|
||||
|
||||
/**
|
||||
* Calculate a hash using all bytes from the input argument, and
|
||||
* a seed of -1.
|
||||
* @param bytes input bytes
|
||||
* @return hash value
|
||||
*/
|
||||
public int hash(byte[] bytes) {
|
||||
return hash(bytes, bytes.length, -1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Calculate a hash using all bytes from the input argument,
|
||||
* and a provided seed value.
|
||||
* @param bytes input bytes
|
||||
* @param initval seed value
|
||||
* @return hash value
|
||||
*/
|
||||
public int hash(byte[] bytes, int initval) {
|
||||
return hash(bytes, 0, bytes.length, initval);
|
||||
}
|
||||
|
||||
/**
|
||||
* Calculate a hash using bytes from 0 to <code>length</code>, and
|
||||
* the provided seed value
|
||||
* @param bytes input bytes
|
||||
* @param length length of the valid bytes after offset to consider
|
||||
* @param initval seed value
|
||||
* @return hash value
|
||||
*/
|
||||
public int hash(byte[] bytes, int length, int initval) {
|
||||
return hash(bytes, 0, length, initval);
|
||||
}
|
||||
|
||||
/**
|
||||
* Calculate a hash using bytes from <code>offset</code> to <code>offset +
|
||||
* length</code>, and the provided seed value.
|
||||
* @param bytes input bytes
|
||||
* @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
|
||||
*/
|
||||
// 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);
|
||||
}
|
||||
|
||||
/**
|
||||
* Calculate a hash using bytes from <code>offset</code> to <code>offset +
|
||||
* length</code>, and the provided seed value.
|
||||
* Calculate a hash using bytes from HashKey and the provided seed value.
|
||||
* @param <T>
|
||||
* @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);
|
||||
public abstract <T> int hash(HashKey<T> hashKey, int initval);
|
||||
}
|
||||
|
@ -32,5 +32,14 @@ public abstract class HashKey<T> {
|
||||
this.t = t;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param pos
|
||||
* @return The byte at the given position in this HashKey
|
||||
*/
|
||||
public abstract byte get(int pos);
|
||||
|
||||
/**
|
||||
* @return The number of bytes in this HashKey
|
||||
*/
|
||||
public abstract int length();
|
||||
}
|
||||
|
@ -69,7 +69,7 @@ public class JenkinsHash extends Hash {
|
||||
JenkinsHash hash = new JenkinsHash();
|
||||
try {
|
||||
for (int length = in.read(bytes); length > 0; length = in.read(bytes)) {
|
||||
value = hash.hash(bytes, length, value);
|
||||
value = hash.hash(new ByteArrayHashKey(bytes, 0, length), value);
|
||||
}
|
||||
} finally {
|
||||
in.close();
|
||||
@ -81,7 +81,6 @@ public class JenkinsHash extends Hash {
|
||||
* taken from hashlittle() -- hash a variable-length key into a 32-bit value
|
||||
*
|
||||
* @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
|
||||
* return value. Two keys differing by one or two bits will have totally
|
||||
@ -104,11 +103,11 @@ public class JenkinsHash extends Hash {
|
||||
*/
|
||||
@SuppressWarnings("fallthrough")
|
||||
@Override
|
||||
public int hash(HashKey hashKey, int off, int nbytes, int initval) {
|
||||
int length = nbytes;
|
||||
public <T> int hash(HashKey<T> hashKey, int initval) {
|
||||
int length = hashKey.length();
|
||||
int a, b, c;
|
||||
a = b = c = 0xdeadbeef + length + initval;
|
||||
int offset = off;
|
||||
int offset = 0;
|
||||
for (; length > 12; offset += 12, length -= 12) {
|
||||
a += (hashKey.get(offset) & BYTE_MASK);
|
||||
a += ((hashKey.get(offset + 1) & BYTE_MASK) << 8);
|
||||
|
@ -39,16 +39,16 @@ public class MurmurHash extends Hash {
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hash(HashKey hashKey, int offset, int length, int seed) {
|
||||
public <T> int hash(HashKey<T> hashKey, int seed) {
|
||||
int m = 0x5bd1e995;
|
||||
int r = 24;
|
||||
|
||||
int length = hashKey.length();
|
||||
int h = seed ^ length;
|
||||
|
||||
int len_4 = length >> 2;
|
||||
|
||||
for (int i = 0; i < len_4; i++) {
|
||||
int i_4 = (i << 2) + offset;
|
||||
int i_4 = (i << 2);
|
||||
int k = hashKey.get(i_4 + 3);
|
||||
k = k << 8;
|
||||
k = k | (hashKey.get(i_4 + 2) & 0xff);
|
||||
@ -67,7 +67,7 @@ public class MurmurHash extends Hash {
|
||||
// avoid calculating modulo
|
||||
int len_m = len_4 << 2;
|
||||
int left = length - len_m;
|
||||
int i_m = len_m + offset;
|
||||
int i_m = len_m;
|
||||
|
||||
if (left != 0) {
|
||||
if (left >= 3) {
|
||||
|
@ -40,14 +40,14 @@ public class MurmurHash3 extends Hash {
|
||||
/** Returns the MurmurHash3_x86_32 hash. */
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings("SF")
|
||||
@Override
|
||||
public int hash(HashKey hashKey, int offset, int length, int initval) {
|
||||
public <T> int hash(HashKey<T> hashKey, int initval) {
|
||||
final int c1 = 0xcc9e2d51;
|
||||
final int c2 = 0x1b873593;
|
||||
|
||||
int length = hashKey.length();
|
||||
int h1 = initval;
|
||||
int roundedEnd = offset + (length & 0xfffffffc); // round down to 4 byte block
|
||||
int roundedEnd = (length & 0xfffffffc); // round down to 4 byte block
|
||||
|
||||
for (int i = offset; i < roundedEnd; i += 4) {
|
||||
for (int i = 0; i < roundedEnd; i += 4) {
|
||||
// little endian load order
|
||||
int k1 =
|
||||
(hashKey.get(i) & 0xff) | ((hashKey.get(i + 1) & 0xff) << 8)
|
||||
|
@ -32,4 +32,9 @@ public class RowBloomHashKey extends CellHashKey {
|
||||
public byte get(int offset) {
|
||||
return CellUtil.getRowByte(t, offset);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int length() {
|
||||
return this.t.getRowLength();
|
||||
}
|
||||
}
|
||||
|
@ -43,8 +43,8 @@ public class RowColBloomHashKey extends CellHashKey {
|
||||
|
||||
@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
|
||||
// For ROW_COL blooms we use bytes
|
||||
// <RK length> (2 bytes) , <RK>, 0 (one byte CF length), <CQ>, <TS> (8 btes), <TYPE> ( 1 byte)
|
||||
if (offset < Bytes.SIZEOF_SHORT) {
|
||||
// assign locally
|
||||
int rowlen = rowLength;
|
||||
@ -77,4 +77,12 @@ public class RowColBloomHashKey extends CellHashKey {
|
||||
}
|
||||
return MAX_TYPE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int length() {
|
||||
// For ROW_COL blooms we use bytes
|
||||
// <RK length> (2 bytes) , <RK>, 0 (one byte CF length), <CQ>, <TS> (8 btes), <TYPE> ( 1 byte)
|
||||
return KeyValue.ROW_LENGTH_SIZE + this.t.getRowLength() + KeyValue.FAMILY_LENGTH_SIZE
|
||||
+ this.t.getQualifierLength() + KeyValue.TIMESTAMP_TYPE_SIZE;
|
||||
}
|
||||
}
|
||||
|
@ -72,6 +72,7 @@ import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
|
||||
import org.apache.hadoop.hbase.rest.client.Client;
|
||||
import org.apache.hadoop.hbase.rest.client.Cluster;
|
||||
import org.apache.hadoop.hbase.rest.client.RemoteAdmin;
|
||||
import org.apache.hadoop.hbase.util.ByteArrayHashKey;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Hash;
|
||||
import org.apache.hadoop.hbase.util.MurmurHash;
|
||||
@ -738,7 +739,8 @@ public class PerformanceEvaluation extends Configured implements Tool {
|
||||
", writeToWAL=" + this.writeToWAL +
|
||||
", useTags=" + this.useTags +
|
||||
", noOfTags=" + this.noOfTags;
|
||||
int hash = h.hash(Bytes.toBytes(s));
|
||||
byte[] b = Bytes.toBytes(s);
|
||||
int hash = h.hash(new ByteArrayHashKey(b, 0, b.length), -1);
|
||||
m.put(hash, s);
|
||||
}
|
||||
}
|
||||
|
@ -23,10 +23,9 @@ import java.io.DataInput;
|
||||
import java.io.IOException;
|
||||
|
||||
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;
|
||||
import org.apache.hadoop.hbase.nio.ByteBuff;
|
||||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||
import org.apache.hadoop.hbase.util.BloomFilter;
|
||||
import org.apache.hadoop.hbase.util.BloomFilterUtil;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
@ -95,68 +94,66 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase
|
||||
|
||||
@Override
|
||||
public boolean contains(byte[] key, int keyOffset, int keyLength, ByteBuff bloom) {
|
||||
// We try to store the result in this variable so we can update stats for
|
||||
// testing, but when an error happens, we log a message and return.
|
||||
|
||||
int block = index.rootBlockContainingKey(key, keyOffset,
|
||||
keyLength);
|
||||
return checkContains(key, keyOffset, keyLength, block);
|
||||
}
|
||||
|
||||
private boolean checkContains(byte[] key, int keyOffset, int keyLength, int block) {
|
||||
boolean result;
|
||||
int block = index.rootBlockContainingKey(key, keyOffset, keyLength);
|
||||
if (block < 0) {
|
||||
result = false; // This key is not in the file.
|
||||
} else {
|
||||
HFileBlock bloomBlock;
|
||||
try {
|
||||
// We cache the block and use a positional read.
|
||||
bloomBlock = reader.readBlock(index.getRootBlockOffset(block),
|
||||
index.getRootBlockDataSize(block), true, true, false, true,
|
||||
BlockType.BLOOM_CHUNK, null);
|
||||
} catch (IOException ex) {
|
||||
// The Bloom filter is broken, turn it off.
|
||||
throw new IllegalArgumentException(
|
||||
"Failed to load Bloom block for key "
|
||||
+ Bytes.toStringBinary(key, keyOffset, keyLength), ex);
|
||||
}
|
||||
try {
|
||||
ByteBuff bloomBuf = bloomBlock.getBufferReadOnly();
|
||||
result =
|
||||
BloomFilterUtil.contains(key, keyOffset, keyLength, bloomBuf, bloomBlock.headerSize(),
|
||||
bloomBlock.getUncompressedSizeWithoutHeader(), hash, hashCount);
|
||||
} finally {
|
||||
// After the use return back the block if it was served from a cache.
|
||||
reader.returnBlock(bloomBlock);
|
||||
}
|
||||
return false; // This key is not in the file.
|
||||
}
|
||||
|
||||
if (numQueriesPerChunk != null && block >= 0) {
|
||||
boolean result;
|
||||
HFileBlock bloomBlock = getBloomBlock(block);
|
||||
try {
|
||||
ByteBuff bloomBuf = bloomBlock.getBufferReadOnly();
|
||||
result = BloomFilterUtil.contains(key, keyOffset, keyLength, bloomBuf,
|
||||
bloomBlock.headerSize(), bloomBlock.getUncompressedSizeWithoutHeader(), hash, hashCount);
|
||||
} finally {
|
||||
// After the use return back the block if it was served from a cache.
|
||||
reader.returnBlock(bloomBlock);
|
||||
}
|
||||
if (numPositivesPerChunk != null && result) {
|
||||
// Update statistics. Only used in unit tests.
|
||||
++numQueriesPerChunk[block];
|
||||
if (result)
|
||||
++numPositivesPerChunk[block];
|
||||
++numPositivesPerChunk[block];
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean contains(Cell keyCell, ByteBuff bloom) {
|
||||
// We try to store the result in this variable so we can update stats for
|
||||
// testing, but when an error happens, we log a message and return.
|
||||
int block = index.rootBlockContainingKey(keyCell);
|
||||
// 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) {
|
||||
// TODO : directly use Cell here
|
||||
return checkContains(((KeyValue) keyCell).getBuffer(), ((KeyValue) keyCell).getKeyOffset(),
|
||||
((KeyValue) keyCell).getKeyLength(), block);
|
||||
private HFileBlock getBloomBlock(int block) {
|
||||
HFileBlock bloomBlock;
|
||||
try {
|
||||
// We cache the block and use a positional read.
|
||||
bloomBlock = reader.readBlock(index.getRootBlockOffset(block),
|
||||
index.getRootBlockDataSize(block), true, true, false, true, BlockType.BLOOM_CHUNK, null);
|
||||
} catch (IOException ex) {
|
||||
// The Bloom filter is broken, turn it off.
|
||||
throw new IllegalArgumentException("Failed to load Bloom block", ex);
|
||||
}
|
||||
// TODO : Avoid this copy in read path also
|
||||
byte[] key = CellUtil.getCellKeySerializedAsKeyValueKey(keyCell);
|
||||
return checkContains(key, 0, key.length, block);
|
||||
|
||||
if (numQueriesPerChunk != null) {
|
||||
// Update statistics. Only used in unit tests.
|
||||
++numQueriesPerChunk[block];
|
||||
}
|
||||
return bloomBlock;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean contains(Cell keyCell, ByteBuff bloom, BloomType type) {
|
||||
int block = index.rootBlockContainingKey(keyCell);
|
||||
if (block < 0) {
|
||||
return false; // This key is not in the file.
|
||||
}
|
||||
boolean result;
|
||||
HFileBlock bloomBlock = getBloomBlock(block);
|
||||
try {
|
||||
ByteBuff bloomBuf = bloomBlock.getBufferReadOnly();
|
||||
result = BloomFilterUtil.contains(keyCell, bloomBuf, bloomBlock.headerSize(),
|
||||
bloomBlock.getUncompressedSizeWithoutHeader(), hash, hashCount, type);
|
||||
} finally {
|
||||
// After the use return back the block if it was served from a cache.
|
||||
reader.returnBlock(bloomBlock);
|
||||
}
|
||||
if (numPositivesPerChunk != null && result) {
|
||||
// Update statistics. Only used in unit tests.
|
||||
++numPositivesPerChunk[block];
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
public boolean supportsAutoLoading() {
|
||||
|
@ -374,8 +374,8 @@ public class StoreFileReader {
|
||||
exists = false;
|
||||
} else {
|
||||
exists =
|
||||
bloomFilter.contains(kvKey, bloom) ||
|
||||
bloomFilter.contains(rowBloomKey, bloom);
|
||||
bloomFilter.contains(kvKey, bloom, BloomType.ROWCOL) ||
|
||||
bloomFilter.contains(rowBloomKey, bloom, BloomType.ROWCOL);
|
||||
}
|
||||
} else {
|
||||
exists = !keyIsAfterLast
|
||||
|
@ -22,6 +22,7 @@ package org.apache.hadoop.hbase.util;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.nio.ByteBuff;
|
||||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||
|
||||
/**
|
||||
*
|
||||
@ -77,17 +78,16 @@ public interface BloomFilter extends BloomFilterBase {
|
||||
|
||||
/**
|
||||
* Check if the specified key is contained in the bloom filter.
|
||||
* Used in ROW_COL blooms where the blooms are serialized as KeyValues
|
||||
* @param keyCell the key to check for the existence of
|
||||
* @param bloom bloom filter data to search. This can be null if auto-loading
|
||||
* is supported.
|
||||
* @param type The type of Bloom ROW/ ROW_COL
|
||||
* @return true if matched by bloom, false if not
|
||||
*/
|
||||
boolean contains(Cell keyCell, ByteBuff bloom);
|
||||
boolean contains(Cell keyCell, ByteBuff bloom, BloomType type);
|
||||
|
||||
/**
|
||||
* Check if the specified key is contained in the bloom filter.
|
||||
* Used in ROW bloom where the blooms are just plain byte[]
|
||||
* @param buf data to check for existence of
|
||||
* @param offset offset into the data
|
||||
* @param length length of the data
|
||||
|
@ -25,7 +25,6 @@ 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;
|
||||
|
||||
@ -168,18 +167,15 @@ public class BloomFilterChunk implements BloomFilterBase {
|
||||
}
|
||||
}
|
||||
|
||||
public void add(byte [] buf) {
|
||||
add(buf, 0, buf.length);
|
||||
}
|
||||
|
||||
public void add(byte [] buf, int offset, int len) {
|
||||
// Used only by tests
|
||||
void add(byte [] 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);
|
||||
|
||||
HashKey<byte[]> hashKey = new ByteArrayHashKey(buf, offset, len);
|
||||
int hash1 = this.hash.hash(hashKey, 0);
|
||||
int hash2 = this.hash.hash(hashKey, hash1);
|
||||
setHashLoc(hash1, hash2);
|
||||
}
|
||||
|
||||
@ -192,22 +188,14 @@ public class BloomFilterChunk implements BloomFilterBase {
|
||||
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);
|
||||
hash1 = this.hash.hash(hashKey, 0);
|
||||
hash2 = this.hash.hash(hashKey, 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);
|
||||
hash1 = this.hash.hash(hashKey, 0);
|
||||
hash2 = this.hash.hash(hashKey, hash1);
|
||||
}
|
||||
|
||||
setHashLoc(hash1, hash2);
|
||||
}
|
||||
|
||||
|
@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.util;
|
||||
import java.text.NumberFormat;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.nio.ByteBuff;
|
||||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||
@ -197,10 +198,14 @@ public final class BloomFilterUtil {
|
||||
public static boolean contains(byte[] buf, int offset, int length,
|
||||
ByteBuff bloomBuf, int bloomOffset, int bloomSize, Hash hash,
|
||||
int hashCount) {
|
||||
// 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);
|
||||
HashKey<byte[]> hashKey = new ByteArrayHashKey(buf, offset, length);
|
||||
return contains(bloomBuf, bloomOffset, bloomSize, hash, hashCount, hashKey);
|
||||
}
|
||||
|
||||
private static <T> boolean contains(ByteBuff bloomBuf, int bloomOffset, int bloomSize, Hash hash,
|
||||
int hashCount, HashKey<T> hashKey) {
|
||||
int hash1 = hash.hash(hashKey, 0);
|
||||
int hash2 = hash.hash(hashKey, hash1);
|
||||
int bloomBitSize = bloomSize << 3;
|
||||
|
||||
if (randomGeneratorForTest == null) {
|
||||
@ -225,6 +230,13 @@ public final class BloomFilterUtil {
|
||||
return true;
|
||||
}
|
||||
|
||||
public static boolean contains(Cell cell, ByteBuff bloomBuf, int bloomOffset, int bloomSize,
|
||||
Hash hash, int hashCount, BloomType type) {
|
||||
HashKey<Cell> hashKey = type == BloomType.ROW ? new RowBloomHashKey(cell)
|
||||
: new RowColBloomHashKey(cell);
|
||||
return contains(bloomBuf, bloomOffset, bloomSize, hash, hashCount, hashKey);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if bit at specified index is 1.
|
||||
*
|
||||
|
@ -101,7 +101,6 @@ import org.apache.htrace.impl.ProbabilitySampler;
|
||||
import com.google.common.base.Objects;
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
import com.codahale.metrics.Histogram;
|
||||
import com.codahale.metrics.Snapshot;
|
||||
import com.codahale.metrics.UniformReservoir;
|
||||
|
||||
/**
|
||||
@ -550,7 +549,8 @@ public class PerformanceEvaluation extends Configured implements Tool {
|
||||
next.perClientRunRows = perClientRows / 10;
|
||||
String s = MAPPER.writeValueAsString(next);
|
||||
LOG.info("Client=" + j + ", maptask=" + i + ", input=" + s);
|
||||
int hash = h.hash(Bytes.toBytes(s));
|
||||
byte[] b = Bytes.toBytes(s);
|
||||
int hash = h.hash(new ByteArrayHashKey(b, 0, b.length), -1);
|
||||
m.put(hash, s);
|
||||
}
|
||||
}
|
||||
|
@ -600,8 +600,7 @@ public class TestStoreFile extends HBaseTestCase {
|
||||
for (int i = 0; i < 2000; i++) {
|
||||
String row = String.format(localFormatter, i);
|
||||
byte[] rowKey = Bytes.toBytes(row);
|
||||
boolean exists = reader.passesDeleteFamilyBloomFilter(rowKey, 0,
|
||||
rowKey.length);
|
||||
boolean exists = reader.passesDeleteFamilyBloomFilter(rowKey, 0, rowKey.length);
|
||||
if (i % 2 == 0) {
|
||||
if (!exists)
|
||||
falseNeg++;
|
||||
|
@ -43,8 +43,8 @@ public class TestBloomFilterChunk extends TestCase {
|
||||
byte[] key1 = {1,2,3,4,5,6,7,8,9};
|
||||
byte[] key2 = {1,2,3,4,5,6,7,8,7};
|
||||
|
||||
bf1.add(key1);
|
||||
bf2.add(key2);
|
||||
bf1.add(key1, 0, key1.length);
|
||||
bf2.add(key2, 0, key2.length);
|
||||
|
||||
assertTrue(BloomFilterUtil.contains(key1, 0, key1.length, new MultiByteBuff(bf1.bloom), 0,
|
||||
(int) bf1.byteSize, bf1.hash, bf1.hashCount));
|
||||
@ -58,7 +58,7 @@ public class TestBloomFilterChunk extends TestCase {
|
||||
byte [] bkey = {1,2,3,4};
|
||||
byte [] bval = "this is a much larger byte array".getBytes();
|
||||
|
||||
bf1.add(bkey);
|
||||
bf1.add(bkey, 0, bkey.length);
|
||||
bf1.add(bval, 1, bval.length-1);
|
||||
|
||||
assertTrue(BloomFilterUtil.contains(bkey, 0, bkey.length, new MultiByteBuff(bf1.bloom), 0,
|
||||
@ -100,7 +100,8 @@ public class TestBloomFilterChunk extends TestCase {
|
||||
long origSize = b.getByteSize();
|
||||
assertEquals(1204, origSize);
|
||||
for (int i = 0; i < 12; ++i) {
|
||||
b.add(Bytes.toBytes(i));
|
||||
byte[] ib = Bytes.toBytes(i);
|
||||
b.add(ib, 0, ib.length);
|
||||
}
|
||||
b.compactBloom();
|
||||
assertEquals(origSize>>2, b.getByteSize());
|
||||
@ -128,7 +129,8 @@ public class TestBloomFilterChunk extends TestCase {
|
||||
long startTime = System.currentTimeMillis();
|
||||
long origSize = b.getByteSize();
|
||||
for (int i = 0; i < 1*1000*1000; ++i) {
|
||||
b.add(Bytes.toBytes(i));
|
||||
byte[] ib = Bytes.toBytes(i);
|
||||
b.add(ib, 0, ib.length);
|
||||
}
|
||||
long endTime = System.currentTimeMillis();
|
||||
System.out.println("Total Add time = " + (endTime - startTime) + "ms");
|
||||
|
Loading…
x
Reference in New Issue
Block a user