HBASE-9921 stripe compaction - findbugs and javadoc issues, some improvements

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1540215 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
sershe 2013-11-08 23:17:51 +00:00
parent 45a337446e
commit ecb4b1017c
6 changed files with 18 additions and 39 deletions

View File

@ -48,7 +48,7 @@ public interface StoreFileManager {
/**
* Adds new files, either for from MemStore flush or bulk insert, into the structure.
* @param sf New store file.
* @param sfs New store files.
*/
void insertNewFiles(Collection<StoreFile> sfs) throws IOException;

View File

@ -22,6 +22,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
/**
* Configuration class for stripe store and compactions.
@ -79,14 +80,17 @@ public class StripeStoreConfig {
private static final double EPSILON = 0.001; // good enough for this, not a real epsilon.
public StripeStoreConfig(Configuration config, StoreConfigInformation sci) {
this.level0CompactMinFiles = config.getInt(MIN_FILES_L0_KEY, 4);
this.stripeCompactMinFiles = config.getInt(MIN_FILES_KEY, 3);
this.stripeCompactMaxFiles = config.getInt(MAX_FILES_KEY, 10);
this.maxRegionSplitImbalance = getFloat(config, MAX_REGION_SPLIT_IMBALANCE_KEY, 1.5f, true);
this.flushIntoL0 = config.getBoolean(FLUSH_TO_L0_KEY, false);
int minMinFiles = flushIntoL0 ? 3 : 4; // make sure not to compact tiny files too often.
int minFiles = config.getInt(CompactionConfiguration.MIN_KEY, -1);
this.stripeCompactMinFiles = config.getInt(MIN_FILES_KEY, Math.max(minMinFiles, minFiles));
this.stripeCompactMaxFiles = config.getInt(MAX_FILES_KEY,
config.getInt(CompactionConfiguration.MAX_KEY, 10));
this.maxRegionSplitImbalance = getFloat(config, MAX_REGION_SPLIT_IMBALANCE_KEY, 1.5f, true);
float splitPartCount = getFloat(config, SPLIT_PARTS_KEY, 2f, true);
if (Math.abs(splitPartCount - 1.0) < EPSILON) {
LOG.error("Split part count cannot be 1 (" + this.splitPartCount + "), using the default");
LOG.error("Split part count cannot be 1 (" + splitPartCount + "), using the default");
splitPartCount = 2f;
}
this.splitPartCount = splitPartCount;

View File

@ -89,7 +89,9 @@ public class StripeStoreFlusher extends StoreFlusher {
}
} finally {
if (!success && (mw != null)) {
result.clear();
if (result != null) {
result.clear();
}
for (Path leftoverFile : mw.abortWriters()) {
try {
store.getFileSystem().delete(leftoverFile, false);

View File

@ -48,6 +48,8 @@ public class CompactionConfiguration {
private static final String CONFIG_PREFIX = "hbase.hstore.compaction.";
public static final String RATIO_KEY = CONFIG_PREFIX + "ratio";
public static final String MIN_KEY = CONFIG_PREFIX + "min";
public static final String MAX_KEY = CONFIG_PREFIX + "max";
Configuration conf;
StoreConfigInformation storeConfigInfo;
@ -70,9 +72,9 @@ public class CompactionConfiguration {
maxCompactSize = conf.getLong(CONFIG_PREFIX + "max.size", Long.MAX_VALUE);
minCompactSize = conf.getLong(CONFIG_PREFIX + "min.size",
storeConfigInfo.getMemstoreFlushSize());
minFilesToCompact = Math.max(2, conf.getInt(CONFIG_PREFIX + "min",
minFilesToCompact = Math.max(2, conf.getInt(MIN_KEY,
/*old name*/ conf.getInt("hbase.hstore.compactionThreshold", 3)));
maxFilesToCompact = conf.getInt(CONFIG_PREFIX + "max", 10);
maxFilesToCompact = conf.getInt(MAX_KEY, 10);
compactionRatio = conf.getFloat(RATIO_KEY, 1.2F);
offPeekCompactionRatio = conf.getFloat(CONFIG_PREFIX + "ratio.offpeak", 5.0F);

View File

@ -114,7 +114,7 @@ public class StripeCompactionPolicy extends CompactionPolicy {
Collection<StoreFile> allFiles = si.getStorefiles();
if (StoreUtils.hasReferences(allFiles)) {
LOG.debug("There are references in the store; compacting all files");
long targetKvs = estimateTargetKvs(allFiles, config.getSplitCount()).getFirst();
long targetKvs = estimateTargetKvs(allFiles, config.getInitialCount()).getFirst();
SplitStripeCompactionRequest request = new SplitStripeCompactionRequest(
allFiles, OPEN_KEY, OPEN_KEY, targetKvs);
request.setMajorRangeFull();
@ -509,35 +509,6 @@ public class StripeCompactionPolicy extends CompactionPolicy {
}
}
/** Helper class used to calculate size related things */
private static class StripeSizes {
public final ArrayList<Long> kvCounts;
public final ArrayList<Long> fileSizes;
public double avgKvCount = 0;
public long minKvCount = Long.MAX_VALUE, maxKvCount = Long.MIN_VALUE;
public int minIndex = -1, maxIndex = -1;
public StripeSizes(List<ImmutableList<StoreFile>> stripes) {
assert !stripes.isEmpty();
kvCounts = new ArrayList<Long>(stripes.size());
fileSizes = new ArrayList<Long>(stripes.size());
for (int i = 0; i < stripes.size(); ++i) {
long kvCount = getTotalKvCount(stripes.get(i));
fileSizes.add(getTotalFileSize(stripes.get(i)));
kvCounts.add(kvCount);
avgKvCount += (double)(kvCount - avgKvCount) / (i + 1);
if (minKvCount > kvCount) {
minIndex = i;
minKvCount = kvCount;
}
if (maxKvCount < kvCount) {
maxIndex = i;
maxKvCount = kvCount;
}
}
}
}
/** The information about stripes that the policy needs to do its stuff */
public static interface StripeInformationProvider {
public Collection<StoreFile> getStorefiles();

View File

@ -329,7 +329,7 @@ public class TestStripeCompactionPolicy {
Configuration conf = HBaseConfiguration.create();
StripeCompactionPolicy policy = createPolicy(conf);
// Verify the deletes can be dropped if there are no L0 files.
Long[][] stripes = new Long[][] { new Long[] { 3L, 2L, 2L }, new Long[] { 6L } };
Long[][] stripes = new Long[][] { new Long[] { 3L, 2L, 2L, 2L }, new Long[] { 6L } };
StripeInformationProvider si = createStripesWithSizes(0, 0, stripes);
verifySingleStripeCompaction(policy, si, 0, true);
// But cannot be dropped if there are.