HBASE-3158 Bloom File Writes Broken if keySize is large
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1027810 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
97f7976755
commit
f9fda11ec0
|
@ -615,6 +615,8 @@ Release 0.21.0 - Unreleased
|
||||||
HBASE-2753 Remove sorted() methods from Result now that Gets are Scans
|
HBASE-2753 Remove sorted() methods from Result now that Gets are Scans
|
||||||
HBASE-3147 Regions stuck in transition after rolling restart, perpetual
|
HBASE-3147 Regions stuck in transition after rolling restart, perpetual
|
||||||
timeout handling but nothing happens
|
timeout handling but nothing happens
|
||||||
|
HBASE-3158 Bloom File Writes Broken if keySize is large
|
||||||
|
(Nicolas Spiegelberg via Stack)
|
||||||
|
|
||||||
IMPROVEMENTS
|
IMPROVEMENTS
|
||||||
HBASE-1760 Cleanup TODOs in HTable
|
HBASE-1760 Cleanup TODOs in HTable
|
||||||
|
|
|
@ -83,6 +83,7 @@ public class StoreFile {
|
||||||
// Config keys.
|
// Config keys.
|
||||||
static final String IO_STOREFILE_BLOOM_ERROR_RATE = "io.storefile.bloom.error.rate";
|
static final String IO_STOREFILE_BLOOM_ERROR_RATE = "io.storefile.bloom.error.rate";
|
||||||
static final String IO_STOREFILE_BLOOM_MAX_FOLD = "io.storefile.bloom.max.fold";
|
static final String IO_STOREFILE_BLOOM_MAX_FOLD = "io.storefile.bloom.max.fold";
|
||||||
|
static final String IO_STOREFILE_BLOOM_MAX_KEYS = "io.storefile.bloom.max.keys";
|
||||||
static final String IO_STOREFILE_BLOOM_ENABLED = "io.storefile.bloom.enabled";
|
static final String IO_STOREFILE_BLOOM_ENABLED = "io.storefile.bloom.enabled";
|
||||||
static final String HFILE_BLOCK_CACHE_SIZE_KEY = "hfile.block.cache.size";
|
static final String HFILE_BLOCK_CACHE_SIZE_KEY = "hfile.block.cache.size";
|
||||||
|
|
||||||
|
@ -691,6 +692,9 @@ public class StoreFile {
|
||||||
|
|
||||||
this.kvComparator = comparator;
|
this.kvComparator = comparator;
|
||||||
|
|
||||||
|
BloomFilter bloom = null;
|
||||||
|
BloomType bt = BloomType.NONE;
|
||||||
|
|
||||||
if (bloomType != BloomType.NONE && conf != null) {
|
if (bloomType != BloomType.NONE && conf != null) {
|
||||||
float err = conf.getFloat(IO_STOREFILE_BLOOM_ERROR_RATE, (float)0.01);
|
float err = conf.getFloat(IO_STOREFILE_BLOOM_ERROR_RATE, (float)0.01);
|
||||||
// Since in row+col blooms we have 2 calls to shouldSeek() instead of 1
|
// Since in row+col blooms we have 2 calls to shouldSeek() instead of 1
|
||||||
|
@ -701,15 +705,31 @@ public class StoreFile {
|
||||||
err /= 2;
|
err /= 2;
|
||||||
}
|
}
|
||||||
int maxFold = conf.getInt(IO_STOREFILE_BLOOM_MAX_FOLD, 7);
|
int maxFold = conf.getInt(IO_STOREFILE_BLOOM_MAX_FOLD, 7);
|
||||||
|
int tooBig = conf.getInt(IO_STOREFILE_BLOOM_MAX_KEYS, 128*1000*1000);
|
||||||
this.bloomFilter = new ByteBloomFilter(maxKeys, err,
|
|
||||||
Hash.getHashType(conf), maxFold);
|
if (maxKeys < tooBig) {
|
||||||
this.bloomFilter.allocBloom();
|
try {
|
||||||
this.bloomType = bloomType;
|
bloom = new ByteBloomFilter(maxKeys, err,
|
||||||
} else {
|
Hash.getHashType(conf), maxFold);
|
||||||
this.bloomFilter = null;
|
bloom.allocBloom();
|
||||||
this.bloomType = BloomType.NONE;
|
bt = bloomType;
|
||||||
|
} catch (IllegalArgumentException iae) {
|
||||||
|
LOG.warn(String.format(
|
||||||
|
"Parse error while creating bloom for %s (%d, %f)",
|
||||||
|
path, maxKeys, err), iae);
|
||||||
|
bloom = null;
|
||||||
|
bt = BloomType.NONE;
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("Skipping bloom filter because max keysize too large: "
|
||||||
|
+ maxKeys);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
this.bloomFilter = bloom;
|
||||||
|
this.bloomType = bt;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -820,6 +840,10 @@ public class StoreFile {
|
||||||
public Path getPath() {
|
public Path getPath() {
|
||||||
return this.writer.getPath();
|
return this.writer.getPath();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
boolean hasBloom() {
|
||||||
|
return this.bloomFilter != null;
|
||||||
|
}
|
||||||
|
|
||||||
public void append(final byte [] key, final byte [] value) throws IOException {
|
public void append(final byte [] key, final byte [] value) throws IOException {
|
||||||
if (this.bloomFilter != null) {
|
if (this.bloomFilter != null) {
|
||||||
|
|
|
@ -52,7 +52,7 @@ public class ByteBloomFilter implements BloomFilter {
|
||||||
public static final int VERSION = 1;
|
public static final int VERSION = 1;
|
||||||
|
|
||||||
/** Bytes (B) in the array */
|
/** Bytes (B) in the array */
|
||||||
protected int byteSize;
|
protected long byteSize;
|
||||||
/** Number of hash functions */
|
/** Number of hash functions */
|
||||||
protected final int hashCount;
|
protected final int hashCount;
|
||||||
/** Hash type */
|
/** Hash type */
|
||||||
|
@ -134,11 +134,11 @@ public class ByteBloomFilter implements BloomFilter {
|
||||||
*
|
*
|
||||||
* The probability of false positives is minimized when k = m/n ln(2).
|
* The probability of false positives is minimized when k = m/n ln(2).
|
||||||
*/
|
*/
|
||||||
int bitSize = (int)Math.ceil(maxKeys * (Math.log(errorRate) / Math.log(0.6185)));
|
long bitSize = (long)Math.ceil(maxKeys * (Math.log(errorRate) / Math.log(0.6185)));
|
||||||
int functionCount = (int)Math.ceil(Math.log(2) * (bitSize / maxKeys));
|
int functionCount = (int)Math.ceil(Math.log(2) * (bitSize / maxKeys));
|
||||||
|
|
||||||
// increase byteSize so folding is possible
|
// increase byteSize so folding is possible
|
||||||
int byteSize = (bitSize + 7) / 8;
|
long byteSize = (bitSize + 7) / 8;
|
||||||
int mask = (1 << foldFactor) - 1;
|
int mask = (1 << foldFactor) - 1;
|
||||||
if ( (mask & byteSize) != 0) {
|
if ( (mask & byteSize) != 0) {
|
||||||
byteSize >>= foldFactor;
|
byteSize >>= foldFactor;
|
||||||
|
@ -161,13 +161,13 @@ public class ByteBloomFilter implements BloomFilter {
|
||||||
if (this.bloom != null) {
|
if (this.bloom != null) {
|
||||||
throw new IllegalArgumentException("can only create bloom once.");
|
throw new IllegalArgumentException("can only create bloom once.");
|
||||||
}
|
}
|
||||||
this.bloom = ByteBuffer.allocate(this.byteSize);
|
this.bloom = ByteBuffer.allocate((int)this.byteSize);
|
||||||
assert this.bloom.hasArray();
|
assert this.bloom.hasArray();
|
||||||
}
|
}
|
||||||
|
|
||||||
void sanityCheck() throws IllegalArgumentException {
|
void sanityCheck() throws IllegalArgumentException {
|
||||||
if(this.byteSize <= 0) {
|
if(0 >= this.byteSize || this.byteSize > Integer.MAX_VALUE) {
|
||||||
throw new IllegalArgumentException("maxValue must be > 0");
|
throw new IllegalArgumentException("Invalid byteSize: " + this.byteSize);
|
||||||
}
|
}
|
||||||
|
|
||||||
if(this.hashCount <= 0) {
|
if(this.hashCount <= 0) {
|
||||||
|
@ -205,7 +205,7 @@ public class ByteBloomFilter implements BloomFilter {
|
||||||
int hash2 = this.hash.hash(buf, offset, len, hash1);
|
int hash2 = this.hash.hash(buf, offset, len, hash1);
|
||||||
|
|
||||||
for (int i = 0; i < this.hashCount; i++) {
|
for (int i = 0; i < this.hashCount; i++) {
|
||||||
int hashLoc = Math.abs((hash1 + i * hash2) % (this.byteSize * 8));
|
long hashLoc = Math.abs((hash1 + i * hash2) % (this.byteSize * 8));
|
||||||
set(hashLoc);
|
set(hashLoc);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -243,7 +243,7 @@ public class ByteBloomFilter implements BloomFilter {
|
||||||
int hash2 = this.hash.hash(buf, offset, length, hash1);
|
int hash2 = this.hash.hash(buf, offset, length, hash1);
|
||||||
|
|
||||||
for (int i = 0; i < this.hashCount; i++) {
|
for (int i = 0; i < this.hashCount; i++) {
|
||||||
int hashLoc = Math.abs((hash1 + i * hash2) % (this.byteSize * 8));
|
long hashLoc = Math.abs((hash1 + i * hash2) % (this.byteSize * 8));
|
||||||
if (!get(hashLoc, theBloom) ) {
|
if (!get(hashLoc, theBloom) ) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
@ -259,9 +259,9 @@ public class ByteBloomFilter implements BloomFilter {
|
||||||
*
|
*
|
||||||
* @param pos index of bit
|
* @param pos index of bit
|
||||||
*/
|
*/
|
||||||
void set(int pos) {
|
void set(long pos) {
|
||||||
int bytePos = pos / 8;
|
int bytePos = (int)(pos / 8);
|
||||||
int bitPos = pos % 8;
|
int bitPos = (int)(pos % 8);
|
||||||
byte curByte = bloom.get(bytePos);
|
byte curByte = bloom.get(bytePos);
|
||||||
curByte |= bitvals[bitPos];
|
curByte |= bitvals[bitPos];
|
||||||
bloom.put(bytePos, curByte);
|
bloom.put(bytePos, curByte);
|
||||||
|
@ -273,9 +273,9 @@ public class ByteBloomFilter implements BloomFilter {
|
||||||
* @param pos index of bit
|
* @param pos index of bit
|
||||||
* @return true if bit at specified index is 1, false if 0.
|
* @return true if bit at specified index is 1, false if 0.
|
||||||
*/
|
*/
|
||||||
static boolean get(int pos, ByteBuffer theBloom) {
|
static boolean get(long pos, ByteBuffer theBloom) {
|
||||||
int bytePos = pos / 8;
|
int bytePos = (int)(pos / 8);
|
||||||
int bitPos = pos % 8;
|
int bitPos = (int)(pos % 8);
|
||||||
byte curByte = theBloom.get(bytePos);
|
byte curByte = theBloom.get(bytePos);
|
||||||
curByte &= bitvals[bitPos];
|
curByte &= bitvals[bitPos];
|
||||||
return (curByte != 0);
|
return (curByte != 0);
|
||||||
|
@ -293,7 +293,7 @@ public class ByteBloomFilter implements BloomFilter {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int getByteSize() {
|
public int getByteSize() {
|
||||||
return this.byteSize;
|
return (int)this.byteSize;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -301,7 +301,7 @@ public class ByteBloomFilter implements BloomFilter {
|
||||||
// see if the actual size is exponentially smaller than expected.
|
// see if the actual size is exponentially smaller than expected.
|
||||||
if (this.keyCount > 0 && this.bloom.hasArray()) {
|
if (this.keyCount > 0 && this.bloom.hasArray()) {
|
||||||
int pieces = 1;
|
int pieces = 1;
|
||||||
int newByteSize = this.byteSize;
|
int newByteSize = (int)this.byteSize;
|
||||||
int newMaxKeys = this.maxKeys;
|
int newMaxKeys = this.maxKeys;
|
||||||
|
|
||||||
// while exponentially smaller & folding is lossless
|
// while exponentially smaller & folding is lossless
|
||||||
|
@ -367,7 +367,7 @@ public class ByteBloomFilter implements BloomFilter {
|
||||||
@Override
|
@Override
|
||||||
public void write(DataOutput out) throws IOException {
|
public void write(DataOutput out) throws IOException {
|
||||||
out.writeInt(VERSION);
|
out.writeInt(VERSION);
|
||||||
out.writeInt(byteSize);
|
out.writeInt((int)byteSize);
|
||||||
out.writeInt(hashCount);
|
out.writeInt(hashCount);
|
||||||
out.writeInt(hashType);
|
out.writeInt(hashType);
|
||||||
out.writeInt(keyCount);
|
out.writeInt(keyCount);
|
||||||
|
|
|
@ -40,7 +40,9 @@ import org.apache.hadoop.hbase.client.Scan;
|
||||||
import org.apache.hadoop.hbase.io.Reference.Range;
|
import org.apache.hadoop.hbase.io.Reference.Range;
|
||||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||||
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
|
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
|
||||||
|
import org.apache.hadoop.hbase.util.ByteBloomFilter;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
import org.apache.hadoop.hbase.util.Hash;
|
||||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||||
import org.mockito.Mockito;
|
import org.mockito.Mockito;
|
||||||
|
|
||||||
|
@ -322,18 +324,11 @@ public class TestStoreFile extends HBaseTestCase {
|
||||||
private static String ROOT_DIR =
|
private static String ROOT_DIR =
|
||||||
HBaseTestingUtility.getTestDir("TestStoreFile").toString();
|
HBaseTestingUtility.getTestDir("TestStoreFile").toString();
|
||||||
private static String localFormatter = "%010d";
|
private static String localFormatter = "%010d";
|
||||||
|
|
||||||
public void testBloomFilter() throws Exception {
|
private void bloomWriteRead(StoreFile.Writer writer, FileSystem fs)
|
||||||
FileSystem fs = FileSystem.getLocal(conf);
|
throws Exception {
|
||||||
conf.setFloat("io.hfile.bloom.error.rate", (float)0.01);
|
float err = conf.getFloat(StoreFile.IO_STOREFILE_BLOOM_ERROR_RATE, 0);
|
||||||
conf.setBoolean("io.hfile.bloom.enabled", true);
|
Path f = writer.getPath();
|
||||||
|
|
||||||
// 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);
|
|
||||||
|
|
||||||
long now = System.currentTimeMillis();
|
long now = System.currentTimeMillis();
|
||||||
for (int i = 0; i < 2000; i += 2) {
|
for (int i = 0; i < 2000; i += 2) {
|
||||||
String row = String.format(localFormatter, i);
|
String row = String.format(localFormatter, i);
|
||||||
|
@ -370,14 +365,31 @@ public class TestStoreFile extends HBaseTestCase {
|
||||||
System.out.println("False negatives: " + falseNeg);
|
System.out.println("False negatives: " + falseNeg);
|
||||||
assertEquals(0, falseNeg);
|
assertEquals(0, falseNeg);
|
||||||
System.out.println("False positives: " + falsePos);
|
System.out.println("False positives: " + falsePos);
|
||||||
assertTrue(falsePos < 2);
|
if (!(falsePos <= 2* 2000 * err)) {
|
||||||
|
System.out.println("WTFBBQ! " + falsePos + ", " + (2* 2000 * err) );
|
||||||
|
}
|
||||||
|
assertTrue(falsePos <= 2* 2000 * err);
|
||||||
|
}
|
||||||
|
|
||||||
|
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);
|
||||||
|
|
||||||
|
// 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);
|
||||||
|
|
||||||
|
bloomWriteRead(writer, fs);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testBloomTypes() throws Exception {
|
public void testBloomTypes() throws Exception {
|
||||||
float err = (float) 0.01;
|
float err = (float) 0.01;
|
||||||
FileSystem fs = FileSystem.getLocal(conf);
|
FileSystem fs = FileSystem.getLocal(conf);
|
||||||
conf.setFloat("io.hfile.bloom.error.rate", err);
|
conf.setFloat(StoreFile.IO_STOREFILE_BLOOM_ERROR_RATE, err);
|
||||||
conf.setBoolean("io.hfile.bloom.enabled", true);
|
conf.setBoolean(StoreFile.IO_STOREFILE_BLOOM_ENABLED, true);
|
||||||
|
|
||||||
int rowCount = 50;
|
int rowCount = 50;
|
||||||
int colCount = 10;
|
int colCount = 10;
|
||||||
|
@ -455,6 +467,45 @@ public class TestStoreFile extends HBaseTestCase {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void testBloomEdgeCases() throws Exception {
|
||||||
|
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);
|
||||||
|
|
||||||
|
// 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);
|
||||||
|
assertFalse(writer.hasBloom());
|
||||||
|
writer.close();
|
||||||
|
fs.delete(f, true);
|
||||||
|
|
||||||
|
conf.setInt(StoreFile.IO_STOREFILE_BLOOM_MAX_KEYS, Integer.MAX_VALUE);
|
||||||
|
|
||||||
|
// TODO: commented out because we run out of java heap space on trunk
|
||||||
|
/*
|
||||||
|
// the below config caused IllegalArgumentException in our production cluster
|
||||||
|
// however, the resulting byteSize is < MAX_INT, so this should work properly
|
||||||
|
writer = new StoreFile.Writer(fs, f,
|
||||||
|
StoreFile.DEFAULT_BLOCKSIZE_SMALL, HFile.DEFAULT_COMPRESSION_ALGORITHM,
|
||||||
|
conf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW, 272446963);
|
||||||
|
assertTrue(writer.hasBloom());
|
||||||
|
bloomWriteRead(writer, fs);
|
||||||
|
*/
|
||||||
|
|
||||||
|
// this, however, is too large and should not create a bloom
|
||||||
|
// 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);
|
||||||
|
assertFalse(writer.hasBloom());
|
||||||
|
writer.close();
|
||||||
|
fs.delete(f, true);
|
||||||
|
}
|
||||||
|
|
||||||
public void testFlushTimeComparator() {
|
public void testFlushTimeComparator() {
|
||||||
assertOrdering(StoreFile.Comparators.FLUSH_TIME,
|
assertOrdering(StoreFile.Comparators.FLUSH_TIME,
|
||||||
mockStoreFile(true, 1000, -1, "/foo/123"),
|
mockStoreFile(true, 1000, -1, "/foo/123"),
|
||||||
|
|
Loading…
Reference in New Issue