HBASE-3290 (Addendum) : Fixes & new unit tests per Kannan's peer review
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1042288 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
d60262f2a9
commit
c83a86ec85
|
@ -92,7 +92,7 @@ public class Store implements HeapSize {
|
|||
final Configuration conf;
|
||||
// ttl in milliseconds.
|
||||
protected long ttl;
|
||||
private long majorCompactionTime;
|
||||
long majorCompactionTime;
|
||||
private final int minFilesToCompact;
|
||||
private final int maxFilesToCompact;
|
||||
private final long minCompactSize;
|
||||
|
@ -204,7 +204,7 @@ public class Store implements HeapSize {
|
|||
this.minCompactSize = conf.getLong("hbase.hstore.compaction.min.size",
|
||||
this.region.memstoreFlushSize);
|
||||
this.maxCompactSize
|
||||
= conf.getLong("hbase.hstore.compaction.max.size", 0);
|
||||
= conf.getLong("hbase.hstore.compaction.max.size", Long.MAX_VALUE);
|
||||
this.compactRatio = conf.getFloat("hbase.hstore.compaction.ratio", 1.2F);
|
||||
|
||||
if (Store.closeCheckInterval == 0) {
|
||||
|
@ -840,23 +840,19 @@ public class Store implements HeapSize {
|
|||
*/
|
||||
List<StoreFile> filesToCompact = new ArrayList<StoreFile>(candidates);
|
||||
|
||||
// Do not compact files above a configurable max filesize unless they are
|
||||
// references. We MUST compact these
|
||||
if (this.maxCompactSize > 0) {
|
||||
final long msize = this.maxCompactSize;
|
||||
filesToCompact.removeAll(Collections2.filter(filesToCompact,
|
||||
new Predicate<StoreFile>() {
|
||||
public boolean apply(StoreFile sf) {
|
||||
// NOTE: keep all references. we must compact them
|
||||
return sf.getReader().length() > msize && !sf.isReference();
|
||||
}
|
||||
}));
|
||||
if (!forcemajor) {
|
||||
// do not compact old files above a configurable threshold
|
||||
// save all references. we MUST compact them
|
||||
int pos = 0;
|
||||
while (pos < filesToCompact.size() &&
|
||||
filesToCompact.get(pos).getReader().length() > maxCompactSize &&
|
||||
!filesToCompact.get(pos).isReference()) ++pos;
|
||||
filesToCompact.subList(0, pos).clear();
|
||||
}
|
||||
|
||||
|
||||
// major compact on user action or age (caveat: we have too many files)
|
||||
boolean majorcompaction = forcemajor ||
|
||||
(isMajorCompaction(filesToCompact) &&
|
||||
filesToCompact.size() > this.maxFilesToCompact);
|
||||
boolean majorcompaction = (forcemajor || isMajorCompaction(filesToCompact))
|
||||
&& filesToCompact.size() < this.maxFilesToCompact;
|
||||
|
||||
if (filesToCompact.isEmpty()) {
|
||||
LOG.debug(this.storeNameStr + ": no store files to compact");
|
||||
|
@ -868,26 +864,10 @@ public class Store implements HeapSize {
|
|||
int start = 0;
|
||||
double r = this.compactRatio;
|
||||
|
||||
/* TODO: add sorting + unit test back in when HBASE-2856 is fixed
|
||||
// Sort files by size to correct when normal skew is altered by bulk load.
|
||||
//
|
||||
// So, technically, order is important for optimizations like the TimeStamp
|
||||
// filter. However, realistically this isn't a problem because our normal
|
||||
// skew always decreases in filesize over time. The only place where our
|
||||
// skew doesn't decrease is for files that have been recently flushed.
|
||||
// However, all those will be unconditionally compacted because they will
|
||||
// be lower than "hbase.hstore.compaction.min.size".
|
||||
//
|
||||
// The sorting is to handle an interesting issue that popped up for us
|
||||
// during migration: we're bulk loading StoreFiles of extremely variable
|
||||
// size (are we migrating 1k users or 10M?) and they will all appear at
|
||||
// the end of the StoreFile list. How do we determine when it is
|
||||
// efficient to compact them? The easiest option was to sort the compact
|
||||
// list and handle bulk files by relative size instead of making some
|
||||
// custom compaction selection algorithm just for bulk inclusion. It
|
||||
// seems like any other companies that will incrementally migrate data
|
||||
// into HBase would hit the same issue. Nicolas.
|
||||
//
|
||||
Collections.sort(filesToCompact, StoreFile.Comparators.FILE_SIZE);
|
||||
*/
|
||||
|
||||
// get store file sizes for incremental compacting selection.
|
||||
int countOfFiles = filesToCompact.size();
|
||||
|
|
|
@ -139,20 +139,24 @@ public class TestCompactSelection extends TestCase {
|
|||
return ret;
|
||||
}
|
||||
|
||||
void compactEquals(List<StoreFile> actual, long ... expected)
|
||||
long[] getSizes(List<StoreFile> sfList) {
|
||||
long[] aNums = new long[sfList.size()];
|
||||
for (int i=0; i <sfList.size(); ++i) {
|
||||
aNums[i] = sfList.get(i).getReader().length();
|
||||
}
|
||||
return aNums;
|
||||
}
|
||||
|
||||
void compactEquals(List<StoreFile> candidates, long ... expected)
|
||||
throws IOException {
|
||||
compactEquals(actual, false, expected);
|
||||
compactEquals(candidates, false, expected);
|
||||
}
|
||||
|
||||
void compactEquals(List<StoreFile> actual, boolean forcemajor,
|
||||
void compactEquals(List<StoreFile> candidates, boolean forcemajor,
|
||||
long ... expected)
|
||||
throws IOException {
|
||||
List<StoreFile> result = store.compactSelection(actual, forcemajor);
|
||||
long[] aNums = new long[result.size()];
|
||||
for (int i=0; i <result.size(); ++i) {
|
||||
aNums[i] = result.get(i).getReader().length();
|
||||
}
|
||||
assertEquals(Arrays.toString(expected), Arrays.toString(aNums));
|
||||
List<StoreFile> actual = store.compactSelection(candidates, forcemajor);
|
||||
assertEquals(Arrays.toString(expected), Arrays.toString(getSizes(actual)));
|
||||
}
|
||||
|
||||
public void testCompactionRatio() throws IOException {
|
||||
|
@ -173,32 +177,44 @@ public class TestCompactSelection extends TestCase {
|
|||
compactEquals(sfCreate(tooBig, tooBig, 700,700) /* empty */);
|
||||
// small files = don't care about ratio
|
||||
compactEquals(sfCreate(8,3,1), 8,3,1);
|
||||
/* TODO: add sorting + unit test back in when HBASE-2856 is fixed
|
||||
// sort first so you don't include huge file the tail end
|
||||
// happens with HFileOutputFormat bulk migration
|
||||
compactEquals(sfCreate(100,50,23,12,12, 500), 23, 12, 12);
|
||||
*/
|
||||
// don't exceed max file compact threshold
|
||||
assertEquals(maxFiles,
|
||||
store.compactSelection(sfCreate(7,6,5,4,3,2,1), false).size());
|
||||
|
||||
/* MAJOR COMPACTION */
|
||||
// if a major compaction has been forced, then compact everything
|
||||
compactEquals(sfCreate(100,50,25,12,12), true, 100, 50, 25, 12, 12);
|
||||
compactEquals(sfCreate(50,25,12,12), true, 50, 25, 12, 12);
|
||||
// also choose files < threshold on major compaction
|
||||
compactEquals(sfCreate(12,12), true, 12, 12);
|
||||
// unless one of those files is too big
|
||||
compactEquals(sfCreate(tooBig, 12,12), true, 12, 12);
|
||||
// even if one of those files is too big
|
||||
compactEquals(sfCreate(tooBig, 12,12), true, tooBig, 12, 12);
|
||||
// don't exceed max file compact threshold, even with major compaction
|
||||
assertEquals(maxFiles,
|
||||
store.compactSelection(sfCreate(7,6,5,4,3,2,1), true).size());
|
||||
// if we exceed maxCompactSize, downgrade to minor
|
||||
// if not, it creates a 'snowball effect' when files >> maxCompactSize:
|
||||
// the last file in compaction is the aggregate of all previous compactions
|
||||
compactEquals(sfCreate(100,50,23,12,12), true, 23, 12, 12);
|
||||
// trigger an aged major compaction
|
||||
store.majorCompactionTime = 1;
|
||||
compactEquals(sfCreate(50,25,12,12), 50, 25, 12, 12);
|
||||
// major sure exceeding maxCompactSize also downgrades aged minors
|
||||
store.majorCompactionTime = 1;
|
||||
compactEquals(sfCreate(100,50,23,12,12), 23, 12, 12);
|
||||
|
||||
/* REFERENCES == file is from a region that was split */
|
||||
// treat storefiles that have references like a major compaction
|
||||
compactEquals(sfCreate(true, 100,50,25,12,12), true, 100, 50, 25, 12, 12);
|
||||
compactEquals(sfCreate(true, 100,50,25,12,12), 100, 50, 25, 12, 12);
|
||||
// reference files shouldn't obey max threshold
|
||||
compactEquals(sfCreate(true, tooBig, 12,12), true, tooBig, 12, 12);
|
||||
compactEquals(sfCreate(true, tooBig, 12,12), tooBig, 12, 12);
|
||||
// reference files should obey max file compact to avoid OOM
|
||||
assertEquals(maxFiles,
|
||||
store.compactSelection(sfCreate(true, 7,6,5,4,3,2,1), true).size());
|
||||
store.compactSelection(sfCreate(true, 7,6,5,4,3,2,1), false).size());
|
||||
|
||||
// empty case
|
||||
compactEquals(new ArrayList<StoreFile>() /* empty */);
|
||||
|
|
Loading…
Reference in New Issue