You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ns...@apache.org on 2011/10/11 04:19:55 UTC
svn commit: r1181557 [2/2] - in /hbase/branches/0.89/src:
main/java/org/apache/hadoop/hbase/regionserver/
main/java/org/apache/hadoop/hbase/util/ main/resources/
test/java/org/apache/hadoop/hbase/regionserver/
test/java/org/apache/hadoop/hbase/util/
Added: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterWriter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterWriter.java?rev=1181557&view=auto
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterWriter.java (added)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterWriter.java Tue Oct 11 02:19:54 2011
@@ -0,0 +1,281 @@
+/*
+ * Copyright 2011 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.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.LinkedList;
+import java.util.Queue;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.io.hfile.BlockType;
+import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex;
+import org.apache.hadoop.hbase.io.hfile.InlineBlockWriter;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Adds methods required for writing a compound Bloom filter to the data
+ * section of an {@link org.apache.hadoop.hbase.io.hfile.HFile} to the
+ * {@link CompoundBloomFilter} class.
+ */
+public class CompoundBloomFilterWriter extends CompoundBloomFilterBase
+ implements BloomFilterWriter, InlineBlockWriter {
+
+ protected static final Log LOG =
+ LogFactory.getLog(CompoundBloomFilterWriter.class);
+
+ /** The current chunk being written to */
+ private ByteBloomFilter chunk;
+
+ /** Previous chunk, so that we can create another similar chunk */
+ private ByteBloomFilter prevChunk;
+
+ /** Maximum fold factor */
+ private int maxFold;
+
+ /** The size of individual Bloom filter chunks to create */
+ private int chunkByteSize;
+
+ /** A Bloom filter chunk enqueued for writing */
+ private static class ReadyChunk {
+ int chunkId;
+ byte[] firstKey;
+ ByteBloomFilter chunk;
+ }
+
+ private Queue<ReadyChunk> readyChunks = new LinkedList<ReadyChunk>();
+
+ /** The first key in the current Bloom filter chunk. */
+ private byte[] firstKeyInChunk = null;
+
+ private HFileBlockIndex.BlockIndexWriter bloomBlockIndexWriter =
+ new HFileBlockIndex.BlockIndexWriter();
+
+ /** Whether to cache-on-write compound Bloom filter chunks */
+ private boolean cacheOnWrite;
+
+ /**
+ * @param chunkByteSizeHint
+ * each chunk's size in bytes. The real chunk size might be different
+ * as required by the fold factor.
+ * @param errorRate
+ * target false positive rate
+ * @param hashType
+ * hash function type to use
+ * @param maxFold
+ * maximum degree of folding allowed
+ */
+ public CompoundBloomFilterWriter(int chunkByteSizeHint, float errorRate,
+ int hashType, int maxFold, boolean cacheOnWrite,
+ RawComparator<byte[]> comparator) {
+ chunkByteSize = ByteBloomFilter.computeFoldableByteSize(
+ chunkByteSizeHint * 8, maxFold);
+
+ this.errorRate = errorRate;
+ this.hashType = hashType;
+ this.maxFold = maxFold;
+ this.cacheOnWrite = cacheOnWrite;
+ this.comparator = comparator;
+ }
+
+ @Override
+ public boolean shouldWriteBlock(boolean closing) {
+ enqueueReadyChunk(closing);
+ return !readyChunks.isEmpty();
+ }
+
+ /**
+ * Enqueue the current chunk if it is ready to be written out.
+ *
+ * @param closing true if we are closing the file, so we do not expect new
+ * keys to show up
+ */
+ private void enqueueReadyChunk(boolean closing) {
+ if (chunk == null ||
+ (chunk.getKeyCount() < chunk.getMaxKeys() && !closing)) {
+ return;
+ }
+
+ if (firstKeyInChunk == null) {
+ throw new NullPointerException("Trying to enqueue a chunk, " +
+ "but first key is null: closing=" + closing + ", keyCount=" +
+ chunk.getKeyCount() + ", maxKeys=" + chunk.getMaxKeys());
+ }
+
+ ReadyChunk readyChunk = new ReadyChunk();
+ readyChunk.chunkId = numChunks - 1;
+ readyChunk.chunk = chunk;
+ readyChunk.firstKey = firstKeyInChunk;
+ readyChunks.add(readyChunk);
+
+ long prevMaxKeys = chunk.getMaxKeys();
+ long prevByteSize = chunk.getByteSize();
+
+ chunk.compactBloom();
+
+ if (LOG.isDebugEnabled() && prevByteSize != chunk.getByteSize()) {
+ LOG.debug("Compacted Bloom chunk #" + readyChunk.chunkId + " from ["
+ + prevMaxKeys + " max keys, " + prevByteSize + " bytes] to ["
+ + chunk.getMaxKeys() + " max keys, " + chunk.getByteSize()
+ + " bytes]");
+ }
+
+ totalMaxKeys += chunk.getMaxKeys();
+ totalByteSize += chunk.getByteSize();
+
+ firstKeyInChunk = null;
+ prevChunk = chunk;
+ chunk = null;
+ }
+
+ /**
+ * Adds a Bloom filter key. This key must be greater than the previous key,
+ * as defined by the comparator this compound Bloom filter is configured
+ * with. For efficiency, key monotonicity is not checked here. See
+ * {@link org.apache.hadoop.hbase.regionserver.StoreFile.Writer#append(
+ * org.apache.hadoop.hbase.KeyValue)} for the details of deduplication.
+ */
+ @Override
+ public void add(byte[] bloomKey, int keyOffset, int keyLength) {
+ if (bloomKey == null)
+ throw new NullPointerException();
+
+ enqueueReadyChunk(false);
+
+ if (chunk == null) {
+ if (firstKeyInChunk != null) {
+ throw new IllegalStateException("First key in chunk already set: "
+ + Bytes.toStringBinary(firstKeyInChunk));
+ }
+ firstKeyInChunk = Arrays.copyOfRange(bloomKey, keyOffset, keyOffset
+ + keyLength);
+
+ if (prevChunk == null) {
+ // First chunk
+ chunk = ByteBloomFilter.createBySize(chunkByteSize, errorRate,
+ hashType, maxFold);
+ } else {
+ // Use the same parameters as the last chunk, but a new array and
+ // a zero key count.
+ chunk = prevChunk.createAnother();
+ }
+
+ if (chunk.getKeyCount() != 0) {
+ throw new IllegalStateException("keyCount=" + chunk.getKeyCount()
+ + " > 0");
+ }
+
+ chunk.allocBloom();
+ ++numChunks;
+ }
+
+ chunk.add(bloomKey, keyOffset, keyLength);
+ ++totalKeyCount;
+ }
+
+ @Override
+ public void writeInlineBlock(DataOutput out) throws IOException {
+ // We don't remove the chunk from the queue here, because we might need it
+ // again for cache-on-write.
+ ReadyChunk readyChunk = readyChunks.peek();
+
+ ByteBloomFilter readyChunkBloom = readyChunk.chunk;
+ readyChunkBloom.getDataWriter().write(out);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Wrote Bloom chunk #" + readyChunk.chunkId + ": "
+ + readyChunkBloom + "; total Bloom " + "keys: " + totalKeyCount);
+ }
+ }
+
+ @Override
+ public void blockWritten(long offset, int onDiskSize, int uncompressedSize) {
+ ReadyChunk readyChunk = readyChunks.remove();
+ bloomBlockIndexWriter.addEntry(readyChunk.firstKey, offset, onDiskSize);
+ }
+
+ @Override
+ public BlockType getInlineBlockType() {
+ return BlockType.BLOOM_CHUNK;
+ }
+
+ private class MetaWriter implements Writable {
+ protected MetaWriter() {}
+
+ @Override
+ public void readFields(DataInput in) throws IOException {
+ throw new IOException("Cant read with this class.");
+ }
+
+ /**
+ * This is modeled after {@link ByteBloomFilter.MetaWriter} for simplicity,
+ * although the two metadata formats do not have to be consistent. This
+ * does have to be consistent with how {@link
+ * CompoundBloomFilter#CompoundBloomFilter(DataInput,
+ * org.apache.hadoop.hbase.io.hfile.HFile.Reader)} reads fields.
+ */
+ @Override
+ public void write(DataOutput out) throws IOException {
+ out.writeInt(VERSION);
+
+ out.writeLong(getByteSize());
+ out.writeInt(prevChunk.getHashCount());
+ out.writeInt(prevChunk.getHashType());
+ out.writeLong(getKeyCount());
+ out.writeLong(getMaxKeys());
+
+ // Fields that don't have equivalents in ByteBloomFilter.
+ out.writeInt(numChunks);
+ Bytes.writeByteArray(out,
+ Bytes.toBytes(comparator.getClass().getName()));
+
+ // Write a single-level index without compression or block header.
+ bloomBlockIndexWriter.writeSingleLevelIndex(out, "Bloom filter");
+ }
+ }
+
+ @Override
+ public Writable getMetaWriter() {
+ return new MetaWriter();
+ }
+
+ @Override
+ public void compactBloom() {
+ }
+
+ @Override
+ public void allocBloom() {
+ // Nothing happens here. All allocation happens on demand.
+ }
+
+ @Override
+ public Writable getDataWriter() {
+ return null;
+ }
+
+ @Override
+ public boolean cacheOnWrite() {
+ return cacheOnWrite;
+ }
+
+}
Modified: hbase/branches/0.89/src/main/resources/hbase-default.xml
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/resources/hbase-default.xml?rev=1181557&r1=1181556&r2=1181557&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/resources/hbase-default.xml (original)
+++ hbase/branches/0.89/src/main/resources/hbase-default.xml Tue Oct 11 02:19:54 2011
@@ -422,6 +422,49 @@
</description>
</property>
<property>
+ <name>hfile.block.index.cacheonwrite</name>
+ <value>false</value>
+ <description>
+ This allows to put non-root multi-level index blocks into the block
+ cache at the time the index is being written.
+ </description>
+ </property>
+ <property>
+ <name>hfile.index.block.max.size</name>
+ <value>131072</value>
+ <description>
+ When the size of a leaf-level, intermediate-level, or root-level
+ index block in a multi-level block index grows to this size, the
+ block is written out and a new block is started.
+ </description>
+ </property>
+ <property>
+ <name>hfile.format.version</name>
+ <value>2</value>
+ <description>
+ The HFile format version to use for new files. Set this to 1 to test
+ backwards-compatibility. The default value of this option should be
+ consistent with FixedFileTrailer.MAX_VERSION.
+ </description>
+ </property>
+ <property>
+ <name>io.storefile.bloom.block.size</name>
+ <value>131072</value>
+ <description>
+ The size in bytes of a single block ("chunk") of a compound Bloom
+ filter. This size is approximate, because Bloom blocks can only be
+ inserted at data block boundaries, and the number of keys per data
+ block varies.
+ </description>
+ </property>
+ <property>
+ <name>io.storefile.bloom.cacheonwrite</name>
+ <value>false</value>
+ <description>
+ Enables cache-on-write for inline blocks of a compound Bloom filter.
+ </description>
+ </property>
+ <property>
<name>hbase.rs.cacheblocksonwrite</name>
<value>false</value>
<description>
Added: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java?rev=1181557&view=auto
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java (added)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java Tue Oct 11 02:19:54 2011
@@ -0,0 +1,353 @@
+/*
+ * Copyright 2009 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.regionserver;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+import java.util.TreeSet;
+
+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.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.io.hfile.BlockCache;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.TestHFileWriterV2;
+import org.apache.hadoop.hbase.regionserver.StoreFile.BloomType;
+import org.apache.hadoop.hbase.util.BloomFilterFactory;
+import org.apache.hadoop.hbase.util.ByteBloomFilter;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CompoundBloomFilter;
+import org.apache.hadoop.hbase.util.CompoundBloomFilterBase;
+import org.apache.hadoop.hbase.util.CompoundBloomFilterWriter;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Tests writing Bloom filter blocks in the same part of the file as data
+ * blocks.
+ */
+public class TestCompoundBloomFilter {
+
+ private static final HBaseTestingUtility TEST_UTIL =
+ new HBaseTestingUtility();
+
+ private static final Log LOG = LogFactory.getLog(
+ TestCompoundBloomFilter.class);
+
+ private static final int NUM_TESTS = 9;
+ private static final BloomType BLOOM_TYPES[] = { BloomType.ROW,
+ BloomType.ROW, BloomType.ROWCOL, BloomType.ROWCOL, BloomType.ROW,
+ BloomType.ROWCOL, BloomType.ROWCOL, BloomType.ROWCOL, BloomType.ROW };
+
+ private static final int NUM_KV[];
+ static {
+ final int N = 10000; // Only used in initialization.
+ NUM_KV = new int[] { 21870, N, N, N, N, 1000, N, 7500, 7500};
+ assert NUM_KV.length == NUM_TESTS;
+ }
+
+ private static final int BLOCK_SIZES[];
+ static {
+ final int blkSize = 65536;
+ BLOCK_SIZES = new int[] { 512, 1000, blkSize, blkSize, blkSize, 128, 300,
+ blkSize, blkSize };
+ assert BLOCK_SIZES.length == NUM_TESTS;
+ }
+
+ /**
+ * Be careful not to specify too high a Bloom filter block size, otherwise
+ * there will only be one oversized chunk and the observed false positive
+ * rate will be too low.
+ */
+ private static final int BLOOM_BLOCK_SIZES[] = { 1000, 4096, 4096, 4096,
+ 8192, 128, 1024, 600, 600 };
+ static { assert BLOOM_BLOCK_SIZES.length == NUM_TESTS; }
+
+ private static final double TARGET_ERROR_RATES[] = { 0.025, 0.01, 0.015,
+ 0.01, 0.03, 0.01, 0.01, 0.07, 0.07 };
+ static { assert TARGET_ERROR_RATES.length == NUM_TESTS; }
+
+ /** A false positive rate that is obviously too high. */
+ private static final double TOO_HIGH_ERROR_RATE;
+ static {
+ double m = 0;
+ for (double errorRate : TARGET_ERROR_RATES)
+ m = Math.max(m, errorRate);
+ TOO_HIGH_ERROR_RATE = m + 0.03;
+ }
+
+ private static Configuration conf;
+ private FileSystem fs;
+ private BlockCache blockCache;
+
+ /** A message of the form "in test#<number>:" to include in logging. */
+ private String testIdMsg;
+
+ private static final int GENERATION_SEED = 2319;
+ private static final int EVALUATION_SEED = 135;
+
+ @Before
+ public void setUp() throws IOException {
+ conf = TEST_UTIL.getConfiguration();
+
+ // This test requires the most recent HFile format (i.e. v2).
+ conf.setInt(HFile.FORMAT_VERSION_KEY, HFile.MAX_FORMAT_VERSION);
+
+ fs = FileSystem.get(conf);
+
+ blockCache = StoreFile.getBlockCache(conf);
+ assertNotNull(blockCache);
+ }
+
+ private List<KeyValue> createSortedKeyValues(Random rand, int n) {
+ List<KeyValue> kvList = new ArrayList<KeyValue>(n);
+ for (int i = 0; i < n; ++i)
+ kvList.add(TestHFileWriterV2.randomKeyValue(rand));
+ Collections.sort(kvList, KeyValue.COMPARATOR);
+ return kvList;
+ }
+
+ @Test
+ public void testCompoundBloomFilter() throws IOException {
+ conf.setBoolean(BloomFilterFactory.IO_STOREFILE_BLOOM_ENABLED, true);
+ for (int t = 0; t < NUM_TESTS; ++t) {
+ conf.setFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE,
+ (float) TARGET_ERROR_RATES[t]);
+
+ testIdMsg = "in test #" + t + ":";
+ Random generationRand = new Random(GENERATION_SEED);
+ List<KeyValue> kvs = createSortedKeyValues(generationRand, NUM_KV[t]);
+ BloomType bt = BLOOM_TYPES[t];
+ Path sfPath = writeStoreFile(t, bt, kvs);
+ readStoreFile(t, bt, kvs, sfPath);
+ }
+ }
+
+ /**
+ * Validates the false positive ratio by computing its z-value and comparing
+ * it to the provided threshold.
+ *
+ * @param falsePosRate experimental positive rate
+ * @param nTrials the number of calls to
+ * {@link StoreFile.Reader#shouldSeek(Scan, java.util.SortedSet)}.
+ * @param zValueBoundary z-value boundary, positive for an upper bound and
+ * negative for a lower bound
+ * @param cbf the compound Bloom filter we are using
+ * @param additionalMsg additional message to include in log output and
+ * assertion failures
+ */
+ private void validateFalsePosRate(double falsePosRate, int nTrials,
+ double zValueBoundary, CompoundBloomFilter cbf, String additionalMsg) {
+ double p = BloomFilterFactory.getErrorRate(conf);
+ double zValue = (falsePosRate - p) / Math.sqrt(p * (1 - p) / nTrials);
+
+ String assortedStatsStr = " (targetErrorRate=" + p + ", falsePosRate="
+ + falsePosRate + ", nTrials=" + nTrials + ")";
+ LOG.info("z-value is " + zValue + assortedStatsStr);
+
+ boolean isUpperBound = zValueBoundary > 0;
+
+ if (isUpperBound && zValue > zValueBoundary ||
+ !isUpperBound && zValue < zValueBoundary) {
+ String errorMsg = "False positive rate z-value " + zValue + " is "
+ + (isUpperBound ? "higher" : "lower") + " than " + zValueBoundary
+ + assortedStatsStr + ". Per-chunk stats:\n"
+ + cbf.formatTestingStats();
+ fail(errorMsg + additionalMsg);
+ }
+ }
+
+ private void readStoreFile(int t, BloomType bt, List<KeyValue> kvs,
+ Path sfPath) throws IOException {
+ StoreFile sf = new StoreFile(fs, sfPath, true, conf, bt, false);
+ StoreFile.Reader r = sf.createReader();
+ final boolean pread = true; // does not really matter
+ StoreFileScanner scanner = r.getStoreFileScanner(true, pread);
+
+ {
+ // Test for false negatives (not allowed).
+ int numChecked = 0;
+ for (KeyValue kv : kvs) {
+ byte[] row = kv.getRow();
+ boolean present = isInBloom(scanner, row, kv.getQualifier());
+ assertTrue(testIdMsg + " Bloom filter false negative on row "
+ + Bytes.toStringBinary(row) + " after " + numChecked
+ + " successful checks", present);
+ ++numChecked;
+ }
+ }
+
+ // Test for false positives (some percentage allowed). We test in two modes:
+ // "fake lookup" which ignores the key distribution, and production mode.
+ for (boolean fakeLookupEnabled : new boolean[] { true, false }) {
+ ByteBloomFilter.setFakeLookupMode(fakeLookupEnabled);
+ try {
+ String fakeLookupModeStr = ", fake lookup is " + (fakeLookupEnabled ?
+ "enabled" : "disabled");
+ CompoundBloomFilter cbf = (CompoundBloomFilter) r.getBloomFilter();
+ cbf.enableTestingStats();
+ int numFalsePos = 0;
+ Random rand = new Random(EVALUATION_SEED);
+ int nTrials = NUM_KV[t] * 10;
+ for (int i = 0; i < nTrials; ++i) {
+ byte[] query = TestHFileWriterV2.randomRowOrQualifier(rand);
+ if (isInBloom(scanner, query, bt, rand)) {
+ numFalsePos += 1;
+ }
+ }
+ double falsePosRate = numFalsePos * 1.0 / nTrials;
+ LOG.debug(String.format(testIdMsg
+ + " False positives: %d out of %d (%f)",
+ numFalsePos, nTrials, falsePosRate) + fakeLookupModeStr);
+
+ // Check for obvious Bloom filter crashes.
+ assertTrue("False positive is too high: " + falsePosRate + " (greater "
+ + "than " + TOO_HIGH_ERROR_RATE + ")" + fakeLookupModeStr,
+ falsePosRate < TOO_HIGH_ERROR_RATE);
+
+ // Now a more precise check to see if the false positive rate is not
+ // too high. The reason we use a relaxed restriction for the real-world
+ // case as opposed to the "fake lookup" case is that our hash functions
+ // are not completely independent.
+
+ double maxZValue = fakeLookupEnabled ? 1.96 : 2.5;
+ validateFalsePosRate(falsePosRate, nTrials, maxZValue, cbf,
+ fakeLookupModeStr);
+
+ // For checking the lower bound we need to eliminate the last chunk,
+ // because it is frequently smaller and the false positive rate in it
+ // is too low. This does not help if there is only one under-sized
+ // chunk, though.
+ int nChunks = cbf.getNumChunks();
+ if (nChunks > 1) {
+ numFalsePos -= cbf.getNumPositivesForTesting(nChunks - 1);
+ nTrials -= cbf.getNumQueriesForTesting(nChunks - 1);
+ falsePosRate = numFalsePos * 1.0 / nTrials;
+ LOG.info(testIdMsg + " False positive rate without last chunk is " +
+ falsePosRate + fakeLookupModeStr);
+ }
+
+ validateFalsePosRate(falsePosRate, nTrials, -2.58, cbf,
+ fakeLookupModeStr);
+ } finally {
+ ByteBloomFilter.setFakeLookupMode(false);
+ }
+ }
+
+ r.close();
+ }
+
+ private boolean isInBloom(StoreFileScanner scanner, byte[] row, BloomType bt,
+ Random rand) {
+ return isInBloom(scanner, row,
+ TestHFileWriterV2.randomRowOrQualifier(rand));
+ }
+
+ private boolean isInBloom(StoreFileScanner scanner, byte[] row,
+ byte[] qualifier) {
+ Scan scan = new Scan(row, row);
+ TreeSet<byte[]> columns = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
+ columns.add(qualifier);
+ return scanner.shouldSeek(scan, columns);
+ }
+
+ private Path writeStoreFile(int t, BloomType bt, List<KeyValue> kvs)
+ throws IOException {
+ conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_BLOCK_SIZE,
+ BLOOM_BLOCK_SIZES[t]);
+ conf.setBoolean(HFile.CACHE_BLOCKS_ON_WRITE_KEY, true);
+
+ StoreFile.Writer w = StoreFile.createWriter(fs,
+ HBaseTestingUtility.getTestDir(), BLOCK_SIZES[t], null, null, conf,
+ bt, 0);
+
+ assertTrue(w.hasBloom());
+ assertTrue(w.getBloomWriter() instanceof CompoundBloomFilterWriter);
+ CompoundBloomFilterWriter cbbf =
+ (CompoundBloomFilterWriter) w.getBloomWriter();
+
+ int keyCount = 0;
+ KeyValue prev = null;
+ LOG.debug("Total keys/values to insert: " + kvs.size());
+ for (KeyValue kv : kvs) {
+ w.append(kv);
+
+ // Validate the key count in the Bloom filter.
+ boolean newKey = true;
+ if (prev != null) {
+ newKey = !(bt == BloomType.ROW ? KeyValue.COMPARATOR.matchingRows(kv,
+ prev) : KeyValue.COMPARATOR.matchingRowColumn(kv, prev));
+ }
+ if (newKey)
+ ++keyCount;
+ assertEquals(keyCount, cbbf.getKeyCount());
+
+ prev = kv;
+ }
+ w.close();
+
+ return w.getPath();
+ }
+
+ @Test
+ public void testCompoundBloomSizing() {
+ int bloomBlockByteSize = 4096;
+ int bloomBlockBitSize = bloomBlockByteSize * 8;
+ double targetErrorRate = 0.01;
+ long maxKeysPerChunk = ByteBloomFilter.idealMaxKeys(bloomBlockBitSize,
+ targetErrorRate);
+
+ long bloomSize1 = bloomBlockByteSize * 8;
+ long bloomSize2 = ByteBloomFilter.computeBitSize(maxKeysPerChunk,
+ targetErrorRate);
+
+ double bloomSizeRatio = (bloomSize2 * 1.0 / bloomSize1);
+ assertTrue(Math.abs(bloomSizeRatio - 0.9999) < 0.0001);
+ }
+
+ @Test
+ public void testCreateKey() {
+ CompoundBloomFilterBase cbfb = new CompoundBloomFilterBase();
+ byte[] row = "myRow".getBytes();
+ byte[] qualifier = "myQualifier".getBytes();
+ byte[] rowKey = cbfb.createBloomKey(row, 0, row.length,
+ row, 0, 0);
+ byte[] rowColKey = cbfb.createBloomKey(row, 0, row.length,
+ qualifier, 0, qualifier.length);
+ KeyValue rowKV = KeyValue.createKeyValueFromKey(rowKey);
+ KeyValue rowColKV = KeyValue.createKeyValueFromKey(rowColKey);
+ assertEquals(rowKV.getTimestamp(), rowColKV.getTimestamp());
+ assertEquals(Bytes.toStringBinary(rowKV.getRow()),
+ Bytes.toStringBinary(rowColKV.getRow()));
+ assertEquals(0, rowKV.getQualifier().length);
+ }
+
+}
Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java?rev=1181557&r1=1181556&r2=1181557&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java Tue Oct 11 02:19:54 2011
@@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.io.hfile.
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
import org.apache.hadoop.hbase.io.hfile.LruBlockCache.CacheStats;
+import org.apache.hadoop.hbase.util.BloomFilterFactory;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.mockito.Mockito;
@@ -87,7 +88,8 @@ public class TestStoreFile extends HBase
public void testBasicHalfMapFile() throws Exception {
// Make up a directory hierarchy that has a regiondir and familyname.
StoreFile.Writer writer = StoreFile.createWriter(this.fs,
- new Path(new Path(this.testDir, "regionname"), "familyname"), 2 * 1024);
+ new Path(new Path(this.testDir, "regionname"), "familyname"), 2 * 1024,
+ conf);
writeStoreFile(writer);
checkHalfHFile(new StoreFile(this.fs, writer.getPath(), true, conf,
StoreFile.BloomType.NONE, false));
@@ -127,7 +129,8 @@ public class TestStoreFile extends HBase
Path storedir = new Path(new Path(this.testDir, "regionname"), "familyname");
Path dir = new Path(storedir, "1234567890");
// Make a store file and write data to it.
- StoreFile.Writer writer = StoreFile.createWriter(this.fs, dir, 8 * 1024);
+ StoreFile.Writer writer = StoreFile.createWriter(this.fs, dir, 8 * 1024,
+ conf);
writeStoreFile(writer);
StoreFile hsf = new StoreFile(this.fs, writer.getPath(), true, conf,
StoreFile.BloomType.NONE, false);
@@ -197,8 +200,11 @@ public class TestStoreFile extends HBase
(topScanner.isSeeked() && topScanner.next())) {
key = topScanner.getKey();
- assertTrue(topScanner.getReader().getComparator().compare(key.array(),
- key.arrayOffset(), key.limit(), midkey, 0, midkey.length) >= 0);
+ if (topScanner.getReader().getComparator().compare(key.array(),
+ key.arrayOffset(), key.limit(), midkey, 0, midkey.length) < 0) {
+ fail("key=" + Bytes.toStringBinary(key) + " < midkey=" +
+ Bytes.toStringBinary(midkey));
+ }
if (first) {
first = false;
LOG.info("First in top: " + Bytes.toString(Bytes.toBytes(key)));
@@ -327,7 +333,8 @@ public class TestStoreFile extends HBase
private void bloomWriteRead(StoreFile.Writer writer, FileSystem fs)
throws Exception {
- float err = conf.getFloat(StoreFile.IO_STOREFILE_BLOOM_ERROR_RATE, 0);
+ float err = conf.getFloat(
+ BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE, 0);
Path f = writer.getPath();
long now = System.currentTimeMillis();
for (int i = 0; i < 2000; i += 2) {
@@ -362,25 +369,23 @@ public class TestStoreFile extends HBase
}
reader.close();
fs.delete(f, true);
- System.out.println("False negatives: " + falseNeg);
- assertEquals(0, falseNeg);
- System.out.println("False positives: " + falsePos);
- if (!(falsePos <= 2* 2000 * err)) {
- System.out.println("WTFBBQ! " + falsePos + ", " + (2* 2000 * err) );
- }
- assertTrue(falsePos <= 2* 2000 * err);
+ assertEquals("False negatives: " + falseNeg, 0, falseNeg);
+ int maxFalsePos = (int) (2 * 2000 * err);
+ assertTrue("Too many false positives: " + falsePos + " (err=" + err
+ + ", expected no more than " + maxFalsePos, falsePos <= maxFalsePos);
}
public void testBloomFilter() throws Exception {
FileSystem fs = FileSystem.getLocal(conf);
- conf.setFloat(StoreFile.IO_STOREFILE_BLOOM_ERROR_RATE, (float)0.01);
- conf.setBoolean(StoreFile.IO_STOREFILE_BLOOM_ENABLED, true);
+ conf.setFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE,
+ (float) 0.01);
+ conf.setBoolean(BloomFilterFactory.IO_STOREFILE_BLOOM_ENABLED, true);
// write the file
Path f = new Path(ROOT_DIR, getName());
StoreFile.Writer writer = new StoreFile.Writer(fs, f,
StoreFile.DEFAULT_BLOCKSIZE_SMALL, HFile.DEFAULT_COMPRESSION_ALGORITHM,
- conf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW, 2000, false);
+ conf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW, 2000);
bloomWriteRead(writer, fs);
}
@@ -388,8 +393,8 @@ public class TestStoreFile extends HBase
public void testBloomTypes() throws Exception {
float err = (float) 0.01;
FileSystem fs = FileSystem.getLocal(conf);
- conf.setFloat(StoreFile.IO_STOREFILE_BLOOM_ERROR_RATE, err);
- conf.setBoolean(StoreFile.IO_STOREFILE_BLOOM_ENABLED, true);
+ conf.setFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE, err);
+ conf.setBoolean(BloomFilterFactory.IO_STOREFILE_BLOOM_ENABLED, true);
int rowCount = 50;
int colCount = 10;
@@ -411,7 +416,7 @@ public class TestStoreFile extends HBase
StoreFile.Writer writer = new StoreFile.Writer(fs, f,
StoreFile.DEFAULT_BLOCKSIZE_SMALL,
HFile.DEFAULT_COMPRESSION_ALGORITHM,
- conf, KeyValue.COMPARATOR, bt[x], expKeys[x], false);
+ conf, KeyValue.COMPARATOR, bt[x], expKeys[x]);
long now = System.currentTimeMillis();
for (int i = 0; i < rowCount*2; i += 2) { // rows
@@ -471,19 +476,23 @@ public class TestStoreFile extends HBase
float err = (float)0.005;
FileSystem fs = FileSystem.getLocal(conf);
Path f = new Path(ROOT_DIR, getName());
- conf.setFloat(StoreFile.IO_STOREFILE_BLOOM_ERROR_RATE, err);
- conf.setBoolean(StoreFile.IO_STOREFILE_BLOOM_ENABLED, true);
- conf.setInt(StoreFile.IO_STOREFILE_BLOOM_MAX_KEYS, 1000);
+ conf.setFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE, err);
+ conf.setBoolean(BloomFilterFactory.IO_STOREFILE_BLOOM_ENABLED, true);
+ conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_MAX_KEYS, 1000);
+
+ // This test only runs for HFile format version 1.
+ conf.setInt(HFile.FORMAT_VERSION_KEY, 1);
// this should not create a bloom because the max keys is too small
StoreFile.Writer writer = new StoreFile.Writer(fs, f,
StoreFile.DEFAULT_BLOCKSIZE_SMALL, HFile.DEFAULT_COMPRESSION_ALGORITHM,
- conf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW, 2000, false);
+ conf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW, 2000);
assertFalse(writer.hasBloom());
writer.close();
fs.delete(f, true);
- conf.setInt(StoreFile.IO_STOREFILE_BLOOM_MAX_KEYS, Integer.MAX_VALUE);
+ conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_MAX_KEYS,
+ Integer.MAX_VALUE);
// TODO: commented out because we ran out of java heap space on trunk
/*
@@ -500,8 +509,7 @@ public class TestStoreFile extends HBase
// because Java can't create a contiguous array > MAX_INT
writer = new StoreFile.Writer(fs, f,
StoreFile.DEFAULT_BLOCKSIZE_SMALL, HFile.DEFAULT_COMPRESSION_ALGORITHM,
- conf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW, Integer.MAX_VALUE,
- false);
+ conf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW, Integer.MAX_VALUE);
assertFalse(writer.hasBloom());
writer.close();
fs.delete(f, true);
@@ -556,7 +564,7 @@ public class TestStoreFile extends HBase
}
/**
- *Generate a list of KeyValues for testing based on given parameters
+ * Generate a list of KeyValues for testing based on given parameters
* @param timestamps
* @param numRows
* @param qualifier
@@ -592,7 +600,8 @@ public class TestStoreFile extends HBase
Path storedir = new Path(new Path(this.testDir, "regionname"),
"familyname");
Path dir = new Path(storedir, "1234567890");
- StoreFile.Writer writer = StoreFile.createWriter(this.fs, dir, 8 * 1024);
+ StoreFile.Writer writer = StoreFile.createWriter(this.fs, dir, 8 * 1024,
+ conf);
List<KeyValue> kvList = getKeyValueSet(timestamps,numRows,
family, qualifier);
@@ -645,7 +654,7 @@ public class TestStoreFile extends HBase
long startEvicted = cs.getEvictedCount();
// Let's write a StoreFile with three blocks, with cache on write off
- conf.setBoolean("hbase.rs.cacheblocksonwrite", false);
+ conf.setBoolean(HFile.CACHE_BLOCKS_ON_WRITE_KEY, false);
Path pathCowOff = new Path(baseDir, "123456789");
StoreFile.Writer writer = writeStoreFile(conf, pathCowOff, 3);
StoreFile hsf = new StoreFile(this.fs, writer.getPath(), true, conf,
@@ -666,7 +675,7 @@ public class TestStoreFile extends HBase
reader.close();
// Now write a StoreFile with three blocks, with cache on write on
- conf.setBoolean("hbase.rs.cacheblocksonwrite", true);
+ conf.setBoolean(HFile.CACHE_BLOCKS_ON_WRITE_KEY, true);
Path pathCowOn = new Path(baseDir, "123456788");
writer = writeStoreFile(conf, pathCowOn, 3);
hsf = new StoreFile(this.fs, writer.getPath(), true, conf,
@@ -704,7 +713,12 @@ public class TestStoreFile extends HBase
while ((kv1 = scannerOne.next()) != null) {
kv2 = scannerTwo.next();
assertTrue(kv1.equals(kv2));
- assertTrue(Bytes.equals(kv1.getBuffer(), kv2.getBuffer()));
+ assertTrue(Bytes.compareTo(
+ kv1.getBuffer(), kv1.getKeyOffset(), kv1.getKeyLength(),
+ kv2.getBuffer(), kv2.getKeyOffset(), kv2.getKeyLength()) == 0);
+ assertTrue(Bytes.compareTo(
+ kv1.getBuffer(), kv1.getValueOffset(), kv1.getValueLength(),
+ kv2.getBuffer(), kv2.getValueOffset(), kv2.getValueLength()) == 0);
}
assertNull(scannerTwo.next());
assertEquals(startHit + 6, cs.getHitCount());
@@ -758,8 +772,7 @@ public class TestStoreFile extends HBase
int blockSize = totalSize / numBlocks;
StoreFile.Writer writer = new StoreFile.Writer(fs, path, blockSize,
HFile.DEFAULT_COMPRESSION_ALGORITHM,
- conf, KeyValue.COMPARATOR, StoreFile.BloomType.NONE, 2000,
- conf.getBoolean("hbase.rs.cacheblocksonwrite", false));
+ conf, KeyValue.COMPARATOR, StoreFile.BloomType.NONE, 2000);
// We'll write N-1 KVs to ensure we don't write an extra block
kvs.remove(kvs.size()-1);
for (KeyValue kv : kvs) {
Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/util/TestByteBloomFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/util/TestByteBloomFilter.java?rev=1181557&r1=1181556&r2=1181557&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/util/TestByteBloomFilter.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/util/TestByteBloomFilter.java Tue Oct 11 02:19:54 2011
@@ -23,7 +23,6 @@ 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;
@@ -78,9 +77,10 @@ public class TestByteBloomFilter extends
public void testBloomFold() throws Exception {
// test: foldFactor < log(max/actual)
- ByteBloomFilter b = new ByteBloomFilter(1003, (float)0.01, Hash.MURMUR_HASH, 2);
+ ByteBloomFilter b = new ByteBloomFilter(1003, (float) 0.01,
+ Hash.MURMUR_HASH, 2);
b.allocBloom();
- int origSize = b.getByteSize();
+ long origSize = b.getByteSize();
assertEquals(1204, origSize);
for (int i = 0; i < 12; ++i) {
b.add(Bytes.toBytes(i));
@@ -106,7 +106,7 @@ public class TestByteBloomFilter extends
ByteBloomFilter b = new ByteBloomFilter(10*1000*1000, (float)err, Hash.MURMUR_HASH, 3);
b.allocBloom();
long startTime = System.currentTimeMillis();
- int origSize = b.getByteSize();
+ long origSize = b.getByteSize();
for (int i = 0; i < 1*1000*1000; ++i) {
b.add(Bytes.toBytes(i));
}
@@ -138,4 +138,27 @@ public class TestByteBloomFilter extends
// test: foldFactor > log(max/actual)
}
+
+ public void testSizing() {
+ int bitSize = 8 * 128 * 1024; // 128 KB
+ double errorRate = 0.025; // target false positive rate
+
+ // How many keys can we store in a Bloom filter of this size maintaining
+ // the given false positive rate, not taking into account that the n
+ long maxKeys = ByteBloomFilter.idealMaxKeys(bitSize, errorRate);
+ assertEquals(136570, maxKeys);
+
+ // A reverse operation: how many bits would we need to store this many keys
+ // and keep the same low false positive rate?
+ long bitSize2 = ByteBloomFilter.computeBitSize(maxKeys, errorRate);
+
+ // The bit size comes out a little different due to rounding.
+ assertTrue(Math.abs(bitSize2 - bitSize) * 1.0 / bitSize < 1e-5);
+ }
+
+ public void testFoldableByteSize() {
+ assertEquals(128, ByteBloomFilter.computeFoldableByteSize(1000, 5));
+ assertEquals(640, ByteBloomFilter.computeFoldableByteSize(5001, 4));
+ }
+
}