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:06:33 UTC
svn commit: r1181414 - in
/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver:
Store.java StoreFile.java
Author: nspiegelberg
Date: Tue Oct 11 02:06:33 2011
New Revision: 1181414
URL: http://svn.apache.org/viewvc?rev=1181414&view=rev
Log:
Hotfixes
Summary:
1. No major compactions if storefile count is too large
2. Catch bloomfilter write exceptions and keep going without bloom data.
Test Plan:
mvn clean test -Dtest=TestStoreFile
DiffCamp Revision: 173818
Reviewed By: kannan
CC: kannan
Tasks:
#422853: DL cluster: compaction errors on one region...
Revert Plan:
OK
Modified:
hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java?rev=1181414&r1=1181413&r2=1181414&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java Tue Oct 11 02:06:33 2011
@@ -636,6 +636,11 @@ public class Store implements HeapSize {
totalSize += len;
}
+ // never run major compaction if we have too many files to avoid OOM
+ if (countOfFiles > this.maxFilesToCompact) {
+ majorcompaction = false;
+ }
+
if (!majorcompaction && !references) {
// Here we select files for incremental compaction.
// The rule is: if the largest(oldest) one is more than twice the
Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java?rev=1181414&r1=1181413&r2=1181414&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java Tue Oct 11 02:06:33 2011
@@ -693,6 +693,9 @@ public class StoreFile {
this.kvComparator = comparator;
+ BloomFilter bloom = null;
+ BloomType bt = BloomType.NONE;
+
if (bloomType != BloomType.NONE && conf != null) {
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
@@ -704,14 +707,22 @@ public class StoreFile {
}
int maxFold = conf.getInt(IO_STOREFILE_BLOOM_MAX_FOLD, 7);
- this.bloomFilter = new ByteBloomFilter(maxKeys, err,
- Hash.getHashType(conf), maxFold);
- this.bloomFilter.allocBloom();
- this.bloomType = bloomType;
- } else {
- this.bloomFilter = null;
- this.bloomType = BloomType.NONE;
+ try {
+ bloom = new ByteBloomFilter(maxKeys, err,
+ Hash.getHashType(conf), maxFold);
+ bloom.allocBloom();
+ bt = bloomType;
+ } catch (IllegalArgumentException iae) {
+ LOG.error(String.format(
+ "Parse error while creating bloom for %s (%d, %d)",
+ path, maxKeys, err), iae);
+ bloom = null;
+ bt = BloomType.NONE;
+ }
}
+
+ this.bloomFilter = bloom;
+ this.bloomType = bt;
}
/**