Permalink
Browse files

HBASE-3063 BloomMapFile - fail-fast version of MapFile for sparsely p…

…opulated key space

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@726797 13f79535-47bb-0310-9956-ffa450edef68
  • Loading branch information...
saintstack committed Dec 15, 2008
1 parent dc21226 commit ffd1a365834f4b193afc02fe7fbe500721416d5d
View
@@ -86,6 +86,9 @@ Trunk (unreleased changes)
HADOOP-4826. Introduce admin command saveNamespace. (shv)
+ HADOOP-3063 BloomMapFile - fail-fast version of MapFile for sparsely
+ populated key space (Andrzej Bialecki via stack)
+
IMPROVEMENTS
HADOOP-4749. Added a new counter REDUCE_INPUT_BYTES. (Yongqiang He via
View
@@ -200,3 +200,45 @@
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.
+
+
+APACHE HADOOP SUBCOMPONENTS:
+
+The Apache Hadoop project contains subcomponents with separate copyright
+notices and license terms. Your use of the source code for the these
+subcomponents is subject to the terms and conditions of the following
+licenses.
+
+For the org.apache.hadoop.util.bloom.* classes:
+
+/**
+ *
+ * Copyright (c) 2005, European Commission project OneLab under contract
+ * 034819 (http://www.one-lab.org)
+ * All rights reserved.
+ * Redistribution and use in source and binary forms, with or
+ * without modification, are permitted provided that the following
+ * conditions are met:
+ * - Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ * - Redistributions in binary form must reproduce the above copyright
+ * notice, this list of conditions and the following disclaimer in
+ * the documentation and/or other materials provided with the distribution.
+ * - Neither the name of the University Catholique de Louvain - UCL
+ * nor the names of its contributors may be used to endorse or
+ * promote products derived from this software without specific prior
+ * written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS
+ * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE
+ * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT,
+ * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING,
+ * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+ * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER
+ * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT
+ * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN
+ * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
+ * POSSIBILITY OF SUCH DAMAGE.
+ */
View
@@ -1263,6 +1263,34 @@ creations/deletions), or "all".</description>
</description>
</property>
+<property>
+ <name>io.mapfile.bloom.size</name>
+ <value>1048576</value>
+ <description>The size of BloomFilter-s used in BloomMapFile. Each time this many
+ keys is appended the next BloomFilter will be created (inside a DynamicBloomFilter).
+ Larger values minimize the number of filters, which slightly increases the performance,
+ but may waste too much space if the total number of keys is usually much smaller
+ than this number.
+ </description>
+</property>
+
+<property>
+ <name>io.mapfile.bloom.error.rate</name>
+ <value>0.005</value>
+ <description>The rate of false positives in BloomFilter-s used in BloomMapFile.
+ As this value decreases, the size of BloomFilter-s increases exponentially. This
+ value is the probability of encountering false positives (default is 0.5%).
+ </description>
+</property>
+
+<property>
+ <name>hadoop.util.hash.type</name>
+ <value>murmur</value>
+ <description>The default implementation of Hash. Currently this can take one of the
+ two values: 'murmur' to select MurmurHash and 'jenkins' to select JenkinsHash.
+ </description>
+</property>
+
<property>
<name>map.sort.class</name>
<value>org.apache.hadoop.util.QuickSort</value>
@@ -0,0 +1,259 @@
+/**
+ * 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.io;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.SequenceFile.CompressionType;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.bloom.DynamicBloomFilter;
+import org.apache.hadoop.util.bloom.Filter;
+import org.apache.hadoop.util.bloom.Key;
+import org.apache.hadoop.util.hash.Hash;
+
+/**
+ * This class extends {@link MapFile} and provides very much the same
+ * functionality. However, it uses dynamic Bloom filters to provide
+ * quick membership test for keys, and it offers a fast version of
+ * {@link Reader#get(WritableComparable, Writable)} operation, especially in
+ * case of sparsely populated MapFile-s.
+ */
+public class BloomMapFile {
+ private static final Log LOG = LogFactory.getLog(BloomMapFile.class);
+ public static final String BLOOM_FILE_NAME = "bloom";
+ public static final int HASH_COUNT = 5;
+
+ public static void delete(FileSystem fs, String name) throws IOException {
+ Path dir = new Path(name);
+ Path data = new Path(dir, MapFile.DATA_FILE_NAME);
+ Path index = new Path(dir, MapFile.INDEX_FILE_NAME);
+ Path bloom = new Path(dir, BLOOM_FILE_NAME);
+
+ fs.delete(data, true);
+ fs.delete(index, true);
+ fs.delete(bloom, true);
+ fs.delete(dir, true);
+ }
+
+ public static class Writer extends MapFile.Writer {
+ private DynamicBloomFilter bloomFilter;
+ private int numKeys;
+ private int vectorSize;
+ private Key bloomKey = new Key();
+ private DataOutputBuffer buf = new DataOutputBuffer();
+ private FileSystem fs;
+ private Path dir;
+
+ public Writer(Configuration conf, FileSystem fs, String dirName,
+ Class<? extends WritableComparable> keyClass,
+ Class<? extends Writable> valClass, CompressionType compress,
+ CompressionCodec codec, Progressable progress) throws IOException {
+ super(conf, fs, dirName, keyClass, valClass, compress, codec, progress);
+ this.fs = fs;
+ this.dir = new Path(dirName);
+ initBloomFilter(conf);
+ }
+
+ public Writer(Configuration conf, FileSystem fs, String dirName,
+ Class<? extends WritableComparable> keyClass,
+ Class valClass, CompressionType compress,
+ Progressable progress) throws IOException {
+ super(conf, fs, dirName, keyClass, valClass, compress, progress);
+ this.fs = fs;
+ this.dir = new Path(dirName);
+ initBloomFilter(conf);
+ }
+
+ public Writer(Configuration conf, FileSystem fs, String dirName,
+ Class<? extends WritableComparable> keyClass,
+ Class valClass, CompressionType compress)
+ throws IOException {
+ super(conf, fs, dirName, keyClass, valClass, compress);
+ this.fs = fs;
+ this.dir = new Path(dirName);
+ initBloomFilter(conf);
+ }
+
+ public Writer(Configuration conf, FileSystem fs, String dirName,
+ WritableComparator comparator, Class valClass,
+ CompressionType compress, CompressionCodec codec, Progressable progress)
+ throws IOException {
+ super(conf, fs, dirName, comparator, valClass, compress, codec, progress);
+ this.fs = fs;
+ this.dir = new Path(dirName);
+ initBloomFilter(conf);
+ }
+
+ public Writer(Configuration conf, FileSystem fs, String dirName,
+ WritableComparator comparator, Class valClass,
+ CompressionType compress, Progressable progress) throws IOException {
+ super(conf, fs, dirName, comparator, valClass, compress, progress);
+ this.fs = fs;
+ this.dir = new Path(dirName);
+ initBloomFilter(conf);
+ }
+
+ public Writer(Configuration conf, FileSystem fs, String dirName,
+ WritableComparator comparator, Class valClass, CompressionType compress)
+ throws IOException {
+ super(conf, fs, dirName, comparator, valClass, compress);
+ this.fs = fs;
+ this.dir = new Path(dirName);
+ initBloomFilter(conf);
+ }
+
+ public Writer(Configuration conf, FileSystem fs, String dirName,
+ WritableComparator comparator, Class valClass) throws IOException {
+ super(conf, fs, dirName, comparator, valClass);
+ this.fs = fs;
+ this.dir = new Path(dirName);
+ initBloomFilter(conf);
+ }
+
+ public Writer(Configuration conf, FileSystem fs, String dirName,
+ Class<? extends WritableComparable> keyClass,
+ Class valClass) throws IOException {
+ super(conf, fs, dirName, keyClass, valClass);
+ this.fs = fs;
+ this.dir = new Path(dirName);
+ initBloomFilter(conf);
+ }
+
+ private synchronized void initBloomFilter(Configuration conf) {
+ numKeys = conf.getInt("io.mapfile.bloom.size", 1024 * 1024);
+ // vector size should be <code>-kn / (ln(1 - c^(1/k)))</code> bits for
+ // single key, where <code> is the number of hash functions,
+ // <code>n</code> is the number of keys and <code>c</code> is the desired
+ // max. error rate.
+ // Our desired error rate is by default 0.005, i.e. 0.5%
+ float errorRate = conf.getFloat("io.mapfile.bloom.error.rate", 0.005f);
+ vectorSize = (int)Math.ceil((double)(-HASH_COUNT * numKeys) /
+ Math.log(1.0 - Math.pow(errorRate, 1.0/HASH_COUNT)));
+ bloomFilter = new DynamicBloomFilter(vectorSize, HASH_COUNT,
+ Hash.getHashType(conf), numKeys);
+ }
+
+ @Override
+ public synchronized void append(WritableComparable key, Writable val)
+ throws IOException {
+ super.append(key, val);
+ buf.reset();
+ key.write(buf);
+ bloomKey.set(buf.getData(), 1.0);
+ bloomFilter.add(bloomKey);
+ }
+
+ @Override
+ public synchronized void close() throws IOException {
+ super.close();
+ DataOutputStream out = fs.create(new Path(dir, BLOOM_FILE_NAME), true);
+ bloomFilter.write(out);
+ out.flush();
+ out.close();
+ }
+
+ }
+
+ public static class Reader extends MapFile.Reader {
+ private DynamicBloomFilter bloomFilter;
+ private DataOutputBuffer buf = new DataOutputBuffer();
+ private Key bloomKey = new Key();
+
+ public Reader(FileSystem fs, String dirName, Configuration conf)
+ throws IOException {
+ super(fs, dirName, conf);
+ initBloomFilter(fs, dirName, conf);
+ }
+
+ public Reader(FileSystem fs, String dirName, WritableComparator comparator,
+ Configuration conf, boolean open) throws IOException {
+ super(fs, dirName, comparator, conf, open);
+ initBloomFilter(fs, dirName, conf);
+ }
+
+ public Reader(FileSystem fs, String dirName, WritableComparator comparator,
+ Configuration conf) throws IOException {
+ super(fs, dirName, comparator, conf);
+ initBloomFilter(fs, dirName, conf);
+ }
+
+ private void initBloomFilter(FileSystem fs, String dirName,
+ Configuration conf) {
+ try {
+ DataInputStream in = fs.open(new Path(dirName, BLOOM_FILE_NAME));
+ bloomFilter = new DynamicBloomFilter();
+ bloomFilter.readFields(in);
+ in.close();
+ } catch (IOException ioe) {
+ LOG.warn("Can't open BloomFilter: " + ioe + " - fallback to MapFile.");
+ bloomFilter = null;
+ }
+ }
+
+ /**
+ * Checks if this MapFile has the indicated key. The membership test is
+ * performed using a Bloom filter, so the result has always non-zero
+ * probability of false positives.
+ * @param key key to check
+ * @return false iff key doesn't exist, true if key probably exists.
+ * @throws IOException
+ */
+ public boolean probablyHasKey(WritableComparable key) throws IOException {
+ if (bloomFilter == null) {
+ return true;
+ }
+ buf.reset();
+ key.write(buf);
+ bloomKey.set(buf.getData(), 1.0);
+ return bloomFilter.membershipTest(bloomKey);
+ }
+
+ /**
+ * Fast version of the
+ * {@link MapFile.Reader#get(WritableComparable, Writable)} method. First
+ * it checks the Bloom filter for the existence of the key, and only if
+ * present it performs the real get operation. This yields significant
+ * performance improvements for get operations on sparsely populated files.
+ */
+ @Override
+ public synchronized Writable get(WritableComparable key, Writable val)
+ throws IOException {
+ if (!probablyHasKey(key)) {
+ return null;
+ }
+ return super.get(key, val);
+ }
+
+ /**
+ * Retrieve the Bloom filter used by this instance of the Reader.
+ * @return a Bloom filter (see {@link Filter})
+ */
+ public Filter getBloomFilter() {
+ return bloomFilter;
+ }
+ }
+}
Oops, something went wrong.

0 comments on commit ffd1a36

Please sign in to comment.