HBASE-1039 Compaction fails if bloomfilters are enabled

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@723109 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2008-12-03 22:39:37 +00:00
parent 977f3934d0
commit 58208a275a
4 changed files with 16 additions and 8 deletions

View File

@ -94,6 +94,7 @@ Release 0.19.0 - Unreleased
(Ryan Smith via Jim Kellerman)
HBASE-927 We don't recover if HRS hosting -ROOT-/.META. goes down -
(fix bug in createTable which caused tests to fail)
HBASE-1039 Compaction fails if bloomfilters are enabled
IMPROVEMENTS

View File

@ -75,8 +75,8 @@ public class BloomFilterMapFile extends HBaseMapFile {
throws IOException {
Path filterFile = new Path(dirName, BLOOMFILTER_FILE_NAME);
if(!fs.exists(filterFile)) {
throw new FileNotFoundException("Could not find bloom filter: " +
filterFile);
LOG.warn("FileNotFound: " + filterFile + "; proceeding without");
return null;
}
BloomFilter filter = new BloomFilter();
FSDataInputStream in = fs.open(filterFile);
@ -180,13 +180,19 @@ public class BloomFilterMapFile extends HBaseMapFile {
* If we fix the number of hash functions and know the number of
* entries, then the optimal vector size m = (k * n) / ln(2)
*/
this.bloomFilter = new BloomFilter(
BloomFilter f = null;
try {
f = new BloomFilter(
(int) Math.ceil(
(DEFAULT_NUMBER_OF_HASH_FUNCTIONS * (1.0 * nrows)) /
Math.log(2.0)),
(int) DEFAULT_NUMBER_OF_HASH_FUNCTIONS,
Hash.getHashType(conf)
);
);
} catch (IllegalArgumentException e) {
LOG.warn("Failed creating bloomfilter; proceeding without", e);
}
this.bloomFilter = f;
} else {
this.bloomFilter = null;
}

View File

@ -62,7 +62,8 @@ class MemcacheFlusher extends Thread implements FlushRequester {
* @param conf
* @param server
*/
public MemcacheFlusher(final HBaseConfiguration conf, final HRegionServer server) {
public MemcacheFlusher(final HBaseConfiguration conf,
final HRegionServer server) {
super();
this.server = server;
threadWakeFrequency = conf.getLong(
@ -235,10 +236,10 @@ class MemcacheFlusher extends Thread implements FlushRequester {
HRegion biggestMemcacheRegion = m.remove(m.firstKey());
LOG.info("Forced flushing of " + biggestMemcacheRegion.toString() +
" because global memcache limit of " + this.globalMemcacheLimit +
" exceeded; currenly " + globalMemcacheSize + " and flushing till " +
" exceeded; currently " + globalMemcacheSize + " and flushing till " +
this.globalMemcacheLimitLowMark);
if (!flushRegion(biggestMemcacheRegion, true)) {
// Something bad happened - give up.
LOG.warn("Flush failed");
break;
}
}

View File

@ -98,7 +98,7 @@ public abstract class Filter implements Writable {
* @param nbHash The number of hash functions to consider.
* @param hashType type of the hashing function (see {@link Hash}).
*/
protected Filter(int vectorSize, int nbHash, int hashType){
protected Filter(int vectorSize, int nbHash, int hashType) {
this.vectorSize = vectorSize;
this.nbHash = nbHash;
this.hashType = hashType;