HADOOP-6546. BloomMapFile can return false negatives. Contributed by Clark Jefcoat.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@925479 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Thomas White 2010-03-19 23:19:13 +00:00
parent a0a4bf08d3
commit c8ffb2249a
3 changed files with 56 additions and 2 deletions

View File

@ -281,6 +281,9 @@ Trunk (unreleased changes)
HADOOP-6504. Invalid example in the documentation of
org.apache.hadoop.util.Tool. (Benoit Sigoure via tomwhite)
HADOOP-6546. BloomMapFile can return false negatives. (Clark Jefcoat
via tomwhite)
Release 0.21.0 - Unreleased
INCOMPATIBLE CHANGES

View File

@ -58,6 +58,16 @@ public static void delete(FileSystem fs, String name) throws IOException {
fs.delete(bloom, true);
fs.delete(dir, true);
}
private static byte[] byteArrayForBloomKey(DataOutputBuffer buf) {
int cleanLength = buf.getLength();
byte [] ba = buf.getData();
if (cleanLength != ba.length) {
ba = new byte[cleanLength];
System.arraycopy(buf.getData(), 0, ba, 0, cleanLength);
}
return ba;
}
public static class Writer extends MapFile.Writer {
private DynamicBloomFilter bloomFilter;
@ -163,7 +173,7 @@ public synchronized void append(WritableComparable key, Writable val)
super.append(key, val);
buf.reset();
key.write(buf);
bloomKey.set(buf.getData(), 1.0);
bloomKey.set(byteArrayForBloomKey(buf), 1.0);
bloomFilter.add(bloomKey);
}
@ -228,7 +238,7 @@ public boolean probablyHasKey(WritableComparable key) throws IOException {
}
buf.reset();
key.write(buf);
bloomKey.set(buf.getData(), 1.0);
bloomKey.set(byteArrayForBloomKey(buf), 1.0);
return bloomFilter.membershipTest(bloomKey);
}

View File

@ -18,6 +18,10 @@
package org.apache.hadoop.io;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -67,4 +71,41 @@ public void testMembershipTest() throws Exception {
assertTrue(falsePos < 2);
}
private void checkMembershipVaryingSizedKeys(String name, List<Text> keys) throws Exception {
Path dirName = new Path(System.getProperty("test.build.data",".") +
name + ".bloommapfile");
FileSystem fs = FileSystem.getLocal(conf);
Path qualifiedDirName = fs.makeQualified(dirName);
BloomMapFile.Writer writer = new BloomMapFile.Writer(conf, fs,
qualifiedDirName.toString(), Text.class, NullWritable.class);
for (Text key : keys) {
writer.append(key, NullWritable.get());
}
writer.close();
// will check for membership in the opposite order of how keys were inserted
BloomMapFile.Reader reader = new BloomMapFile.Reader(fs,
qualifiedDirName.toString(), conf);
Collections.reverse(keys);
for (Text key : keys) {
assertTrue("False negative for existing key " + key, reader.probablyHasKey(key));
}
reader.close();
fs.delete(qualifiedDirName, true);
}
public void testMembershipVaryingSizedKeysTest1() throws Exception {
ArrayList<Text> list = new ArrayList<Text>();
list.add(new Text("A"));
list.add(new Text("BB"));
checkMembershipVaryingSizedKeys(getName(), list);
}
public void testMembershipVaryingSizedKeysTest2() throws Exception {
ArrayList<Text> list = new ArrayList<Text>();
list.add(new Text("AA"));
list.add(new Text("B"));
checkMembershipVaryingSizedKeys(getName(), list);
}
}