From 509d2160d58111072212d0837609f37394059890 Mon Sep 17 00:00:00 2001 From: Michael Stack Date: Sat, 17 Jul 2010 00:45:40 +0000 Subject: [PATCH] HBASE-2843 Re-add bloomfilter test over-zealously removed by HBASE-2625 git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@964990 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 1 + .../hbase/util/TestByteBloomFilter.java | 184 ++++++++++++++++++ 2 files changed, 185 insertions(+) create mode 100644 src/test/java/org/apache/hadoop/hbase/util/TestByteBloomFilter.java diff --git a/CHANGES.txt b/CHANGES.txt index c4b7851eb7a..72d5f3ccce9 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -438,6 +438,7 @@ Release 0.21.0 - Unreleased in the right state (Karthik Ranganathan via JD) HBASE-2727 Splits writing one file only is untenable; need dir of recovered edits ordered by sequenceid + HBASE-2843 Readd bloomfilter test over zealously removed by HBASE-2625 IMPROVEMENTS HBASE-1760 Cleanup TODOs in HTable diff --git a/src/test/java/org/apache/hadoop/hbase/util/TestByteBloomFilter.java b/src/test/java/org/apache/hadoop/hbase/util/TestByteBloomFilter.java new file mode 100644 index 00000000000..bdb3813d6d6 --- /dev/null +++ b/src/test/java/org/apache/hadoop/hbase/util/TestByteBloomFilter.java @@ -0,0 +1,184 @@ +/* + * Copyright 2010 The Apache Software Foundation + * + * 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.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.nio.ByteBuffer; +import java.util.BitSet; + +import junit.framework.TestCase; + +public class TestByteBloomFilter extends TestCase { + + public void testBasicBloom() throws Exception { + ByteBloomFilter bf1 = new ByteBloomFilter(1000, (float)0.01, Hash.MURMUR_HASH, 0); + ByteBloomFilter bf2 = new ByteBloomFilter(1000, (float)0.01, Hash.MURMUR_HASH, 0); + bf1.allocBloom(); + bf2.allocBloom(); + + // test 1: verify no fundamental false negatives or positives + 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); + + assertTrue(bf1.contains(key1)); + assertFalse(bf1.contains(key2)); + assertFalse(bf2.contains(key1)); + assertTrue(bf2.contains(key2)); + + byte [] bkey = {1,2,3,4}; + byte [] bval = "this is a much larger byte array".getBytes(); + + bf1.add(bkey); + bf1.add(bval, 1, bval.length-1); + + assertTrue( bf1.contains(bkey) ); + assertTrue( bf1.contains(bval, 1, bval.length-1) ); + assertFalse( bf1.contains(bval) ); + assertFalse( bf1.contains(bval) ); + + // test 2: serialization & deserialization. + // (convert bloom to byte array & read byte array back in as input) + ByteArrayOutputStream bOut = new ByteArrayOutputStream(); + bf1.writeBloom(new DataOutputStream(bOut)); + ByteBuffer bb = ByteBuffer.wrap(bOut.toByteArray()); + ByteBloomFilter newBf1 = new ByteBloomFilter(1000, (float)0.01, + Hash.MURMUR_HASH, 0); + assertTrue(newBf1.contains(key1, bb)); + assertFalse(newBf1.contains(key2, bb)); + assertTrue( newBf1.contains(bkey, bb) ); + assertTrue( newBf1.contains(bval, 1, bval.length-1, bb) ); + assertFalse( newBf1.contains(bval, bb) ); + assertFalse( newBf1.contains(bval, bb) ); + + System.out.println("Serialized as " + bOut.size() + " bytes"); + assertTrue(bOut.size() - bf1.byteSize < 10); //... allow small padding + } + + public void testBloomFold() throws Exception { + // test: foldFactor < log(max/actual) + ByteBloomFilter b = new ByteBloomFilter(1003, (float)0.01, Hash.MURMUR_HASH, 2); + b.allocBloom(); + int origSize = b.getByteSize(); + assertEquals(1204, origSize); + for (int i = 0; i < 12; ++i) { + b.add(Bytes.toBytes(i)); + } + b.compactBloom(); + assertEquals(origSize>>2, b.getByteSize()); + int falsePositives = 0; + for (int i = 0; i < 25; ++i) { + if (b.contains(Bytes.toBytes(i))) { + if(i >= 12) falsePositives++; + } else { + assertFalse(i < 12); + } + } + assertTrue(falsePositives <= 1); + + // test: foldFactor > log(max/actual) + } + + public void testBloomPerf() throws Exception { + // add + float err = (float)0.01; + ByteBloomFilter b = new ByteBloomFilter(10*1000*1000, (float)err, Hash.MURMUR_HASH, 3); + b.allocBloom(); + long startTime = System.currentTimeMillis(); + int origSize = b.getByteSize(); + for (int i = 0; i < 1*1000*1000; ++i) { + b.add(Bytes.toBytes(i)); + } + long endTime = System.currentTimeMillis(); + System.out.println("Total Add time = " + (endTime - startTime) + "ms"); + + // fold + startTime = System.currentTimeMillis(); + b.compactBloom(); + endTime = System.currentTimeMillis(); + System.out.println("Total Fold time = " + (endTime - startTime) + "ms"); + assertTrue(origSize >= b.getByteSize()<<3); + + // test + startTime = System.currentTimeMillis(); + int falsePositives = 0; + for (int i = 0; i < 2*1000*1000; ++i) { + + if (b.contains(Bytes.toBytes(i))) { + if(i >= 1*1000*1000) falsePositives++; + } else { + assertFalse(i < 1*1000*1000); + } + } + endTime = System.currentTimeMillis(); + System.out.println("Total Contains time = " + (endTime - startTime) + "ms"); + System.out.println("False Positive = " + falsePositives); + assertTrue(falsePositives <= (1*1000*1000)*err); + + // test: foldFactor > log(max/actual) + } + + public void testDynamicBloom() throws Exception { + int keyInterval = 1000; + float err = (float)0.01; + BitSet valid = new BitSet(keyInterval*4); + + DynamicByteBloomFilter bf1 = new DynamicByteBloomFilter(keyInterval, err, + Hash.MURMUR_HASH); + bf1.allocBloom(); + + for (int i = 0; i < keyInterval*4; ++i) { // add + if (Math.random() > 0.5) { + bf1.add(Bytes.toBytes(i)); + valid.set(i); + } + } + assertTrue(2 <= bf1.bloomCount() && bf1.bloomCount() <= 3); + + // test serialization/deserialization + ByteArrayOutputStream metaOut = new ByteArrayOutputStream(); + ByteArrayOutputStream dataOut = new ByteArrayOutputStream(); + bf1.getMetaWriter().write(new DataOutputStream(metaOut)); + bf1.getDataWriter().write(new DataOutputStream(dataOut)); + ByteBuffer bb = ByteBuffer.wrap(dataOut.toByteArray()); + DynamicByteBloomFilter newBf1 = new DynamicByteBloomFilter( + ByteBuffer.wrap(metaOut.toByteArray())); + + int falsePositives = 0; + for (int i = 0; i < keyInterval*4; ++i) { // check + if (newBf1.contains(Bytes.toBytes(i), bb)) { + if (!valid.get(i)) ++falsePositives; + } else { + if (valid.get(i)) { + assert false; + } + } + } + + // note that actualErr = err * bloomCount + // error rate should be roughly: (keyInterval*2)*(err*2), allow some tolerance + System.out.println("False positives: " + falsePositives); + assertTrue(falsePositives <= (keyInterval*5)*err); + } +} \ No newline at end of file