HADOOP-1757 Bloomfilters: single argument constructor, use enum for bloom filter types

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@570270 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Jim Kellerman 2007-08-27 23:35:15 +00:00
parent e9aafde1f1
commit f56ee6b375
5 changed files with 201 additions and 59 deletions

View File

@ -25,6 +25,8 @@ Trunk (unreleased changes)
IMPROVEMENTS
HADOOP-1737 Make HColumnDescriptor data publically members settable
HADOOP-1746 Clean up findbugs warnings
HADOOP-1757 Bloomfilters: single argument constructor, use enum for bloom
filter types
Below are the list of changes before 2007-08-18

View File

@ -27,40 +27,88 @@ import org.apache.hadoop.io.WritableComparable;
/**
* Supplied as a parameter to HColumnDescriptor to specify what kind of
* bloom filter to use for a column, and its configuration parameters
* bloom filter to use for a column, and its configuration parameters.
*
* There is no way to automatically determine the vector size and the number of
* hash functions to use. In particular, bloom filters are very sensitive to the
* number of elements inserted into them. For HBase, the number of entries
* depends on the size of the data stored in the column. Currently the default
* region size is 64MB, so the number of entries is approximately
* 64MB / (average value size for column).
*
* If m denotes the number of bits in the Bloom filter (vectorSize),
* n denotes the number of elements inserted into the Bloom filter and
* k represents the number of hash functions used (nbHash), then according to
* Broder and Mitzenmacher,
*
* ( http://www.eecs.harvard.edu/~michaelm/NEWWORK/postscripts/BloomFilterSurvey.pdf )
*
* the probability of false positives is minimized when k is approximately
* m/n ln(2).
*
*/
public class BloomFilterDescriptor implements WritableComparable {
private static final double DEFAULT_NUMBER_OF_HASH_FUNCTIONS = 4.0;
/*
* Specify the kind of bloom filter that will be instantiated
*/
/** The type of bloom filter */
public static enum BloomFilterType {
/** <i>Bloom filter</i>, as defined by Bloom in 1970. */
BLOOMFILTER,
/**
* <i>Bloom filter</i>, as defined by Bloom in 1970.
* <i>Counting Bloom filter</i>, as defined by Fan et al. in a ToN 2000 paper.
*/
public static final int BLOOMFILTER = 1;
COUNTING_BLOOMFILTER,
/**
* <i>counting Bloom filter</i>, as defined by Fan et al. in a ToN 2000 paper.
* <i>Retouched Bloom filter</i>, as defined in the CoNEXT 2006 paper.
*/
public static final int COUNTING_BLOOMFILTER = 2;
/**
* <i>retouched Bloom filter</i>, as defined in the CoNEXT 2006 paper.
*/
public static final int RETOUCHED_BLOOMFILTER = 3;
RETOUCHED_BLOOMFILTER
}
/** Default constructor - used in conjunction with Writable */
public BloomFilterDescriptor() {
super();
}
/**
* Creates a BloomFilterDescriptor for the specified type of filter, fixes
* the number of hash functions to 4 and computes a vector size using:
*
* vectorSize = ceil((4 * n) / ln(2))
*
* @param type
* @param numberOfEntries
*/
public BloomFilterDescriptor(final BloomFilterType type,
final int numberOfEntries) {
switch(type) {
case BLOOMFILTER:
case COUNTING_BLOOMFILTER:
case RETOUCHED_BLOOMFILTER:
this.filterType = type;
break;
default:
throw new IllegalArgumentException("Invalid bloom filter type: " + type);
}
this.nbHash = (int) DEFAULT_NUMBER_OF_HASH_FUNCTIONS;
this.vectorSize = (int) Math.ceil(
(DEFAULT_NUMBER_OF_HASH_FUNCTIONS * (1.0 * numberOfEntries)) /
Math.log(2.0));
}
/**
* @param type The kind of bloom filter to use.
* @param vectorSize The vector size of <i>this</i> filter.
* @param nbHash The number of hash functions to consider.
*/
public BloomFilterDescriptor(int type, int vectorSize, int nbHash) {
public BloomFilterDescriptor(final BloomFilterType type, final int vectorSize,
final int nbHash) {
switch(type) {
case BLOOMFILTER:
case COUNTING_BLOOMFILTER:
@ -75,7 +123,7 @@ public class BloomFilterDescriptor implements WritableComparable {
this.nbHash = nbHash;
}
int filterType;
BloomFilterType filterType;
int vectorSize;
int nbHash;
@ -113,7 +161,7 @@ public class BloomFilterDescriptor implements WritableComparable {
/** {@inheritDoc} */
@Override
public int hashCode() {
int result = Integer.valueOf(this.filterType).hashCode();
int result = this.filterType.hashCode();
result ^= Integer.valueOf(this.vectorSize).hashCode();
result ^= Integer.valueOf(this.nbHash).hashCode();
return result;
@ -123,14 +171,15 @@ public class BloomFilterDescriptor implements WritableComparable {
/** {@inheritDoc} */
public void readFields(DataInput in) throws IOException {
filterType = in.readInt();
int ordinal = in.readInt();
this.filterType = BloomFilterType.values()[ordinal];
vectorSize = in.readInt();
nbHash = in.readInt();
}
/** {@inheritDoc} */
public void write(DataOutput out) throws IOException {
out.writeInt(filterType);
out.writeInt(filterType.ordinal());
out.writeInt(vectorSize);
out.writeInt(nbHash);
}
@ -140,7 +189,7 @@ public class BloomFilterDescriptor implements WritableComparable {
/** {@inheritDoc} */
public int compareTo(Object o) {
BloomFilterDescriptor other = (BloomFilterDescriptor)o;
int result = this.filterType - other.filterType;
int result = this.filterType.ordinal() - other.filterType.ordinal();
if(result == 0) {
result = this.vectorSize - other.vectorSize;

View File

@ -31,6 +31,11 @@ import org.apache.hadoop.io.WritableComparable;
/**
* An HColumnDescriptor contains information about a column family such as the
* number of versions, compression settings, etc.
*
* It is used as input when creating a table or adding a column. Once set, the
* parameters that specify a column cannot be changed without deleting the
* column and recreating it. If there is data stored in the column, it will be
* deleted when the column is deleted.
*/
public class HColumnDescriptor implements WritableComparable {

View File

@ -317,17 +317,20 @@ class HStore implements HConstants {
LOG.debug("loading bloom filter for " + this.storeName);
}
switch(family.getBloomFilter().filterType) {
BloomFilterDescriptor.BloomFilterType type =
family.getBloomFilter().filterType;
case BloomFilterDescriptor.BLOOMFILTER:
switch(type) {
case BLOOMFILTER:
bloomFilter = new BloomFilter();
break;
case BloomFilterDescriptor.COUNTING_BLOOMFILTER:
case COUNTING_BLOOMFILTER:
bloomFilter = new CountingBloomFilter();
break;
case BloomFilterDescriptor.RETOUCHED_BLOOMFILTER:
case RETOUCHED_BLOOMFILTER:
bloomFilter = new RetouchedBloomFilter();
}
FSDataInputStream in = fs.open(filterFile);
@ -339,20 +342,23 @@ class HStore implements HConstants {
LOG.debug("creating bloom filter for " + this.storeName);
}
switch(family.getBloomFilter().filterType) {
BloomFilterDescriptor.BloomFilterType type =
family.getBloomFilter().filterType;
case BloomFilterDescriptor.BLOOMFILTER:
switch(type) {
case BLOOMFILTER:
bloomFilter = new BloomFilter(family.getBloomFilter().vectorSize,
family.getBloomFilter().nbHash);
break;
case BloomFilterDescriptor.COUNTING_BLOOMFILTER:
case COUNTING_BLOOMFILTER:
bloomFilter =
new CountingBloomFilter(family.getBloomFilter().vectorSize,
family.getBloomFilter().nbHash);
break;
case BloomFilterDescriptor.RETOUCHED_BLOOMFILTER:
case RETOUCHED_BLOOMFILTER:
bloomFilter =
new RetouchedBloomFilter(family.getBloomFilter().vectorSize,
family.getBloomFilter().nbHash);

View File

@ -19,17 +19,15 @@
*/
package org.apache.hadoop.hbase;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.io.Text;
/** Tests per-column bloom filters */
public class TestBloomFilters extends HBaseClusterTestCase {
private static final Text CONTENTS = new Text("contents:");
static final Log LOG = LogFactory.getLog(TestBloomFilters.class);
private HTableDescriptor desc = null;
private HTable table = null;
private static final Text CONTENTS = new Text("contents:");
private static final Text[] rows = {
new Text("wmjwjzyv"),
@ -144,28 +142,40 @@ public class TestBloomFilters extends HBaseClusterTestCase {
/** constructor */
public TestBloomFilters() {
super();
conf.set("hbase.hregion.maxunflushed", "90"); // flush cache every 100 writes
conf.set("hbase.hregion.memcache.flush.size", "100");// flush cache every 100 bytes
conf.set("hbase.regionserver.maxlogentries", "90"); // and roll log too
Logger.getLogger(HRegion.class).setLevel(Level.DEBUG);
Logger.getLogger(HStore.class).setLevel(Level.DEBUG);
}
/** {@inheritDoc} */
@Override
public void setUp() {
/** Test that specifies explicit parameters for the bloom filter */
public void testExplicitParameters() {
HTable table = null;
try {
super.setUp();
this.desc = new HTableDescriptor("test");
desc.addFamily(
new HColumnDescriptor(CONTENTS, 1, HColumnDescriptor.CompressionType.NONE,
false, Integer.MAX_VALUE,
// Setup
HTableDescriptor desc = new HTableDescriptor(getName());
BloomFilterDescriptor bloomFilter =
new BloomFilterDescriptor( // if we insert 1000 values
BloomFilterDescriptor.BLOOMFILTER, // plain old bloom filter
BloomFilterDescriptor.BloomFilterType.BLOOMFILTER, // plain old bloom filter
12499, // number of bits
4 // number of hash functions
))); // false positive = 0.0000001
);
desc.addFamily(
new HColumnDescriptor(CONTENTS, // Column name
1, // Max versions
HColumnDescriptor.CompressionType.NONE, // no compression
HColumnDescriptor.DEFAULT_IN_MEMORY, // not in memory
HColumnDescriptor.DEFAULT_MAX_VALUE_LENGTH,
bloomFilter
)
);
// Create the table
HBaseAdmin admin = new HBaseAdmin(conf);
admin.createTable(desc);
// Open table
table = new HTable(conf, desc.getName());
// Store some values
@ -181,10 +191,78 @@ public class TestBloomFilters extends HBaseClusterTestCase {
e.printStackTrace();
fail();
}
try {
// Give cache flusher and log roller a chance to run
// Otherwise we'll never hit the bloom filter, just the memcache
Thread.sleep(conf.getLong(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000) * 2);
} catch (InterruptedException e) {
// ignore
}
/** the test */
public void testBloomFilters() {
try {
if (table != null) {
for(int i = 0; i < testKeys.length; i++) {
byte[] value = table.get(testKeys[i], CONTENTS);
if(value != null && value.length != 0) {
LOG.info("non existant key: " + testKeys[i] + " returned value: " +
new String(value, HConstants.UTF8_ENCODING));
}
}
}
} catch (Exception e) {
e.printStackTrace();
fail();
}
}
/** Test that uses computed for the bloom filter */
public void testComputedParameters() {
HTable table = null;
try {
// Setup
HTableDescriptor desc = new HTableDescriptor(getName());
BloomFilterDescriptor bloomFilter =
new BloomFilterDescriptor(
BloomFilterDescriptor.BloomFilterType.BLOOMFILTER, // plain old bloom filter
1000 // estimated number of entries
);
LOG.info("vector size: " + bloomFilter.vectorSize);
desc.addFamily(
new HColumnDescriptor(CONTENTS, // Column name
1, // Max versions
HColumnDescriptor.CompressionType.NONE, // no compression
HColumnDescriptor.DEFAULT_IN_MEMORY, // not in memory
HColumnDescriptor.DEFAULT_MAX_VALUE_LENGTH,
bloomFilter
)
);
// Create the table
HBaseAdmin admin = new HBaseAdmin(conf);
admin.createTable(desc);
// Open table
table = new HTable(conf, desc.getName());
// Store some values
for(int i = 0; i < 100; i++) {
Text row = rows[i];
String value = row.toString();
long lockid = table.startUpdate(rows[i]);
table.put(lockid, CONTENTS, value.getBytes(HConstants.UTF8_ENCODING));
table.commit(lockid);
}
} catch (Exception e) {
e.printStackTrace();
fail();
}
try {
// Give cache flusher and log roller a chance to run
// Otherwise we'll never hit the bloom filter, just the memcache
@ -195,11 +273,13 @@ public class TestBloomFilters extends HBaseClusterTestCase {
}
try {
if (table != null) {
for(int i = 0; i < testKeys.length; i++) {
byte[] value = table.get(testKeys[i], CONTENTS);
if(value != null && value.length != 0) {
System.err.println("non existant key: " + testKeys[i] +
" returned value: " + new String(value, HConstants.UTF8_ENCODING));
LOG.info("non existant key: " + testKeys[i] + " returned value: " +
new String(value, HConstants.UTF8_ENCODING));
}
}
}
} catch (Exception e) {