HBASE-875 Use MurmurHash instead of JenkinsHash

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@698265 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2008-09-23 18:10:06 +00:00
parent d65e0b2bd6
commit 804155c541
15 changed files with 95 additions and 53 deletions

View File

@ -19,6 +19,8 @@ Release 0.19.0 - Unreleased
(Doğacan Güney via Stack)
NEW FEATURES
HBASE-875 Use MurmurHash instead of JenkinsHash [in bloomfilters]
(Andrzej Bialecki via Stack)
OPTIMIZATIONS
HBASE-887 Fix a hotspot in scanners

View File

@ -301,4 +301,11 @@
memcache limiting.
</description>
</property>
<property>
<name>hbase.hash.type</name>
<value>murmur</value>
<description>The hashing algorithm for use in HashFunction. Two values are
supported now: murmur (MurmurHash) and jenkins (JenkinsHash).
</description>
</property>
</configuration>

View File

@ -42,7 +42,7 @@ public class HRegionInfo extends VersionedWritable implements WritableComparable
* @return the encodedName
*/
public static int encodeRegionName(final byte [] regionName) {
return Math.abs(JenkinsHash.hash(regionName, regionName.length, 0));
return Math.abs(JenkinsHash.getInstance().hash(regionName, regionName.length, 0));
}
/** delimiter used between portions of a region name */

View File

@ -39,12 +39,14 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.io.BlockFSInputStream;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Hash;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.io.MapFile;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableComparable;
import org.onelab.filter.BloomFilter;
import org.onelab.filter.HashFunction;
import org.onelab.filter.Key;
import org.apache.hadoop.hbase.HConstants;
@ -830,7 +832,8 @@ public class HStoreFile implements HConstants {
(int) Math.ceil(
(DEFAULT_NUMBER_OF_HASH_FUNCTIONS * (1.0 * nrows)) /
Math.log(2.0)),
(int) DEFAULT_NUMBER_OF_HASH_FUNCTIONS
(int) DEFAULT_NUMBER_OF_HASH_FUNCTIONS,
Hash.getHashType(conf)
);
} else {
this.bloomFilter = null;

View File

@ -279,7 +279,7 @@ public class ScannerHandler extends GenericHandler {
// Make a scanner id by hashing the object toString value (object name +
// an id). Will make identifier less burdensome and more url friendly.
String scannerid =
Integer.toHexString(JenkinsHash.hash(scanner.toString().getBytes(), -1));
Integer.toHexString(JenkinsHash.getInstance().hash(scanner.toString().getBytes(), -1));
ScannerRecord sr = new ScannerRecord(scanner);
// store the scanner for subsequent requests

View File

@ -38,36 +38,21 @@ import java.io.IOException;
* @see <a href="http://burtleburtle.net/bob/hash/doobs.html">Has update on the
* Dr. Dobbs Article</a>
*/
public class JenkinsHash {
public class JenkinsHash extends Hash {
private static long INT_MASK = 0x00000000ffffffffL;
private static long BYTE_MASK = 0x00000000000000ffL;
private static JenkinsHash _instance = new JenkinsHash();
public static Hash getInstance() {
return _instance;
}
private static long rot(long val, int pos) {
return ((Integer.rotateLeft(
(int)(val & INT_MASK), pos)) & INT_MASK);
}
/**
* Alternate form for hashing an entire byte array
*
* @param bytes
* @return hash value
*/
public static int hash(byte[] bytes) {
return hash(bytes, bytes.length, -1);
}
/**
* Alternate form for hashing an entire byte array
*
* @param bytes
* @param initval
* @return hash value
*/
public static int hash(byte[] bytes, int initval) {
return hash(bytes, bytes.length, initval);
}
/**
* taken from hashlittle() -- hash a variable-length key into a 32-bit value
*
@ -94,7 +79,7 @@ public class JenkinsHash {
* acceptable. Do NOT use for cryptographic purposes.
*/
@SuppressWarnings("fallthrough")
public static int hash(byte[] key, int nbytes, int initval) {
public int hash(byte[] key, int nbytes, int initval) {
int length = nbytes;
long a, b, c; // We use longs because we don't have unsigned ints
a = b = c = (0x00000000deadbeefL + length + initval) & INT_MASK;
@ -266,8 +251,9 @@ public class JenkinsHash {
FileInputStream in = new FileInputStream(args[0]);
byte[] bytes = new byte[512];
int value = 0;
JenkinsHash hash = new JenkinsHash();
for (int length = in.read(bytes); length > 0 ; length = in.read(bytes)) {
value = hash(bytes, length, value);
value = hash.hash(bytes, length, value);
}
System.out.println(Math.abs(value));
}

View File

@ -45,7 +45,7 @@ public class HRegionInfo implements WritableComparable {
* @return the encodedName
*/
public static int encodeRegionName(final byte [] regionName) {
return Math.abs(JenkinsHash.hash(regionName, regionName.length, 0));
return Math.abs(JenkinsHash.getInstance().hash(regionName, regionName.length, 0));
}
/** delimiter used between portions of a region name */

View File

@ -39,6 +39,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.io.BlockFSInputStream;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Hash;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.io.MapFile;
import org.apache.hadoop.io.SequenceFile;
@ -822,7 +823,8 @@ public class HStoreFile implements HConstants {
(int) DEFAULT_NUMBER_OF_HASH_FUNCTIONS,
(int) Math.ceil(
(DEFAULT_NUMBER_OF_HASH_FUNCTIONS * (1.0 * nrows)) /
Math.log(2.0))
Math.log(2.0)),
Hash.JENKINS_HASH
);
} else {
this.bloomFilter = null;

View File

@ -53,6 +53,8 @@ import java.io.IOException;
import java.util.BitSet;
import org.apache.hadoop.hbase.util.Hash;
/**
* Implements a <i>Bloom filter</i>, as defined by Bloom in 1970.
* <p>
@ -97,9 +99,10 @@ public class BloomFilter extends Filter {
* Constructor
* @param vectorSize The vector size of <i>this</i> filter.
* @param nbHash The number of hash function to consider.
* @param hashType type of the hashing function (see {@link Hash}).
*/
public BloomFilter(int vectorSize, int nbHash){
super(vectorSize, nbHash);
public BloomFilter(int vectorSize, int nbHash, int hashType){
super(vectorSize, nbHash, hashType);
bits = new BitSet(this.vectorSize);
}//end constructor
@ -180,7 +183,7 @@ public class BloomFilter extends Filter {
@Override
public Object clone(){
BloomFilter bf = new BloomFilter(vectorSize, nbHash);
BloomFilter bf = new BloomFilter(vectorSize, nbHash, hashType);
bf.or(this);
return bf;
}//end clone()

View File

@ -52,6 +52,8 @@ import java.io.DataOutput;
import java.io.IOException;
import java.util.Arrays; //TODO: remove
import org.apache.hadoop.hbase.util.Hash;
/**
* Implements a <i>counting Bloom filter</i>, as defined by Fan et al. in a ToN
* 2000 paper.
@ -82,9 +84,10 @@ public final class CountingBloomFilter extends Filter {
* Constructor
* @param vectorSize The vector size of <i>this</i> filter.
* @param nbHash The number of hash function to consider.
* @param hashType type of the hashing function (see {@link Hash}).
*/
public CountingBloomFilter(int vectorSize, int nbHash){
super(vectorSize, nbHash);
public CountingBloomFilter(int vectorSize, int nbHash, int hashType){
super(vectorSize, nbHash, hashType);
buckets = new long[buckets2words(vectorSize)];
}//end constructor
@ -245,7 +248,7 @@ public final class CountingBloomFilter extends Filter {
@Override
public Object clone(){
CountingBloomFilter cbf = new CountingBloomFilter(vectorSize, nbHash);
CountingBloomFilter cbf = new CountingBloomFilter(vectorSize, nbHash, hashType);
cbf.buckets = this.buckets.clone();
return cbf;
}//end clone()

View File

@ -51,6 +51,8 @@ import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.hbase.util.Hash;
/**
* Implements a <i>dynamic Bloom filter</i>, as defined in the INFOCOM 2006 paper.
* <p>
@ -111,16 +113,17 @@ public class DynamicBloomFilter extends Filter {
* Builds an empty Dynamic Bloom filter.
* @param vectorSize The number of bits in the vector.
* @param nbHash The number of hash function to consider.
* @param hashType type of the hashing function (see {@link Hash}).
* @param nr The threshold for the maximum number of keys to record in a dynamic Bloom filter row.
*/
public DynamicBloomFilter(int vectorSize, int nbHash, int nr) {
super(vectorSize, nbHash);
public DynamicBloomFilter(int vectorSize, int nbHash, int hashType, int nr) {
super(vectorSize, nbHash, hashType);
this.nr = nr;
this.currentNbRecord = 0;
matrix = new BloomFilter[1];
matrix[0] = new BloomFilter(this.vectorSize, this.nbHash);
matrix[0] = new BloomFilter(this.vectorSize, this.nbHash, this.hashType);
}//end constructor
@Override
@ -235,7 +238,7 @@ public class DynamicBloomFilter extends Filter {
@Override
public Object clone(){
DynamicBloomFilter dbf = new DynamicBloomFilter(vectorSize, nbHash, nr);
DynamicBloomFilter dbf = new DynamicBloomFilter(vectorSize, nbHash, hashType, nr);
dbf.currentNbRecord = this.currentNbRecord;
dbf.matrix = new BloomFilter[this.matrix.length];
for(int i = 0; i < this.matrix.length; i++) {
@ -280,7 +283,7 @@ public class DynamicBloomFilter extends Filter {
tmp[i] = (BloomFilter)matrix[i].clone();
}
tmp[tmp.length-1] = new BloomFilter(vectorSize, nbHash);
tmp[tmp.length-1] = new BloomFilter(vectorSize, nbHash, hashType);
matrix = tmp;
}//end addRow()

View File

@ -54,6 +54,8 @@ import java.io.DataOutput;
import java.io.IOException;
import java.util.Collection;
import java.util.List;
import org.apache.hadoop.hbase.util.Hash;
import org.apache.hadoop.io.Writable;
/**
@ -75,6 +77,7 @@ import org.apache.hadoop.io.Writable;
* @see org.onelab.filter.HashFunction A hash function
*/
public abstract class Filter implements Writable {
private static final int VERSION = -1; // negative to accommodate for old format
/** The vector size of <i>this</i> filter. */
protected int vectorSize;
@ -84,17 +87,22 @@ public abstract class Filter implements Writable {
/** The number of hash function to consider. */
protected int nbHash;
/** Type of hashing function to use. */
protected int hashType;
protected Filter() {}
/**
* Constructor.
* @param vectorSize The vector size of <i>this</i> filter.
* @param nbHash The number of hash functions to consider.
* @param hashType type of the hashing function (see {@link Hash}).
*/
protected Filter(int vectorSize, int nbHash){
protected Filter(int vectorSize, int nbHash, int hashType){
this.vectorSize = vectorSize;
this.nbHash = nbHash;
this.hash = new HashFunction(this.vectorSize, this.nbHash);
this.hashType = hashType;
this.hash = new HashFunction(this.vectorSize, this.nbHash, this.hashType);
}//end constructor
/**
@ -185,13 +193,24 @@ public abstract class Filter implements Writable {
// Writable interface
public void write(DataOutput out) throws IOException {
out.writeInt(VERSION);
out.writeInt(this.nbHash);
out.writeByte(this.hashType);
out.writeInt(this.vectorSize);
}
public void readFields(DataInput in) throws IOException {
int ver = in.readInt();
if (ver > 0) { // old unversioned format
this.nbHash = ver;
this.hashType = Hash.JENKINS_HASH;
} else if (ver == VERSION) {
this.nbHash = in.readInt();
this.hashType = in.readByte();
} else {
throw new IOException("Unsupported version: " + ver);
}
this.vectorSize = in.readInt();
this.hash = new HashFunction(this.vectorSize, this.nbHash);
this.hash = new HashFunction(this.vectorSize, this.nbHash, this.hashType);
}
}//end class

View File

@ -49,7 +49,9 @@
*/
package org.onelab.filter;
import org.apache.hadoop.hbase.util.Hash;
import org.apache.hadoop.hbase.util.JenkinsHash;
import org.apache.hadoop.hbase.util.MurmurHash;
/**
* Implements a hash object that returns a certain number of hashed values.
@ -65,21 +67,25 @@ import org.apache.hadoop.hbase.util.JenkinsHash;
*
* @see <a href="http://www.itl.nist.gov/fipspubs/fip180-1.htm">SHA-1 algorithm</a>
*/
public final class HashFunction{
public final class HashFunction {
/** The number of hashed values. */
private int nbHash;
/** The maximum highest returned value. */
private int maxValue;
/** Hashing algorithm to use. */
private Hash hashFunction;
/**
* Constructor.
* <p>
* Builds a hash function that must obey to a given maximum number of returned values and a highest value.
* @param maxValue The maximum highest returned value.
* @param nbHash The number of resulting hashed values.
* @param hashType type of the hashing function (see {@link Hash}).
*/
public HashFunction(int maxValue, int nbHash) {
public HashFunction(int maxValue, int nbHash, int hashType) {
if(maxValue <= 0) {
throw new IllegalArgumentException("maxValue must be > 0");
}
@ -90,6 +96,9 @@ public final class HashFunction{
this.maxValue = maxValue;
this.nbHash = nbHash;
this.hashFunction = Hash.getInstance(hashType);
if (this.hashFunction == null)
throw new IllegalArgumentException("hashType must be known");
}//end constructor
/** Clears <i>this</i> hash function. A NOOP */
@ -112,7 +121,7 @@ public final class HashFunction{
}
int[] result = new int[nbHash];
for (int i = 0, initval = 0; i < nbHash; i++) {
initval = result[i] = Math.abs(JenkinsHash.hash(b, initval) % maxValue);
initval = result[i] = Math.abs(hashFunction.hash(b, initval) % maxValue);
}
return result;
}//end hash()

View File

@ -56,6 +56,8 @@ import java.util.Collections;
import java.util.List;
import java.util.Random;
import org.apache.hadoop.hbase.util.Hash;
/**
* Implements a <i>retouched Bloom filter</i>, as defined in the CoNEXT 2006 paper.
* <p>
@ -99,9 +101,10 @@ implements RemoveScheme {
* Constructor
* @param vectorSize The vector size of <i>this</i> filter.
* @param nbHash The number of hash function to consider.
* @param hashType type of the hashing function (see {@link Hash}).
*/
public RetouchedBloomFilter(int vectorSize, int nbHash) {
super(vectorSize, nbHash);
public RetouchedBloomFilter(int vectorSize, int nbHash, int hashType) {
super(vectorSize, nbHash, hashType);
this.rand = null;
createVector();

View File

@ -58,6 +58,7 @@ import junit.framework.TestCase;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.util.Hash;
import org.onelab.filter.*;
/**
@ -208,7 +209,8 @@ public class TestFilter extends TestCase {
(int) Math.ceil(
(DEFAULT_NUMBER_OF_HASH_FUNCTIONS * (1.0 * inserted.length)) /
Math.log(2.0)),
DEFAULT_NUMBER_OF_HASH_FUNCTIONS
DEFAULT_NUMBER_OF_HASH_FUNCTIONS,
Hash.JENKINS_HASH
);
for (int i = 0; i < inserted.length; i++) {
@ -264,7 +266,7 @@ public class TestFilter extends TestCase {
* @throws UnsupportedEncodingException
*/
public void testCountingBloomFilter() throws UnsupportedEncodingException {
Filter bf = new CountingBloomFilter(8, 2);
Filter bf = new CountingBloomFilter(8, 2, Hash.JENKINS_HASH);
Key key = new StringKey("toto");
Key k2 = new StringKey("lulu");
Key k3 = new StringKey("mama");
@ -281,7 +283,7 @@ public class TestFilter extends TestCase {
assertFalse(bf.membershipTest(key));
// OR 'key' back into the filter
Filter bf2 = new CountingBloomFilter(8, 2);
Filter bf2 = new CountingBloomFilter(8, 2, Hash.JENKINS_HASH);
bf2.add(key);
bf.or(bf2);
assertTrue(bf.membershipTest(key));
@ -302,7 +304,7 @@ public class TestFilter extends TestCase {
* @throws UnsupportedEncodingException
*/
public void testDynamicBloomFilter() throws UnsupportedEncodingException {
Filter bf = new DynamicBloomFilter(8, 2, 2);
Filter bf = new DynamicBloomFilter(8, 2, Hash.JENKINS_HASH, 2);
Key key = new StringKey("toto");
Key k2 = new StringKey("lulu");
Key k3 = new StringKey("mama");