HBASE-7842 Add compaction policy that explores more storefile groups
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1465133 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
b5559ab1d7
commit
0f46618bdd
|
@ -19,7 +19,6 @@
|
|||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
@ -27,11 +26,10 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionPolicy;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.Compactor;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactionPolicy;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
|
||||
import org.apache.hadoop.hbase.util.ReflectionUtils;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
|
||||
|
||||
/**
|
||||
* Default StoreEngine creates the default compactor, policy, and store file manager, or
|
||||
|
@ -39,7 +37,7 @@ import org.apache.hadoop.hbase.util.ReflectionUtils;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class DefaultStoreEngine extends StoreEngine<
|
||||
DefaultCompactionPolicy, DefaultCompactor, DefaultStoreFileManager> {
|
||||
RatioBasedCompactionPolicy, DefaultCompactor, DefaultStoreFileManager> {
|
||||
|
||||
public static final String DEFAULT_COMPACTOR_CLASS_KEY =
|
||||
"hbase.hstore.defaultengine.compactor.class";
|
||||
|
@ -48,8 +46,8 @@ public class DefaultStoreEngine extends StoreEngine<
|
|||
|
||||
private static final Class<? extends DefaultCompactor>
|
||||
DEFAULT_COMPACTOR_CLASS = DefaultCompactor.class;
|
||||
private static final Class<? extends DefaultCompactionPolicy>
|
||||
DEFAULT_COMPACTION_POLICY_CLASS = DefaultCompactionPolicy.class;
|
||||
private static final Class<? extends RatioBasedCompactionPolicy>
|
||||
DEFAULT_COMPACTION_POLICY_CLASS = ExploringCompactionPolicy.class;
|
||||
|
||||
@Override
|
||||
protected void createComponents(
|
||||
|
@ -98,4 +96,5 @@ public class DefaultStoreEngine extends StoreEngine<
|
|||
storeFileManager.getStorefiles(), filesCompacting);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -27,8 +27,6 @@ import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
|||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionPolicy;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.Compactor;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactionPolicy;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
|
||||
import org.apache.hadoop.hbase.util.ReflectionUtils;
|
||||
|
||||
/**
|
||||
|
|
|
@ -0,0 +1,89 @@
|
|||
package org.apache.hadoop.hbase.regionserver.compactions;
|
||||
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class ExploringCompactionPolicy extends RatioBasedCompactionPolicy {
|
||||
|
||||
public ExploringCompactionPolicy(Configuration conf,
|
||||
StoreConfigInformation storeConfigInfo) {
|
||||
super(conf, storeConfigInfo);
|
||||
}
|
||||
|
||||
@Override
|
||||
ArrayList<StoreFile> applyCompactionPolicy(ArrayList<StoreFile> candidates,
|
||||
boolean mayUseOffPeak) throws IOException {
|
||||
// Start off choosing nothing.
|
||||
List<StoreFile> bestSelection = new ArrayList<StoreFile>(0);
|
||||
long bestSize = 0;
|
||||
|
||||
// Consider every starting place.
|
||||
for (int start = 0; start < candidates.size(); start++) {
|
||||
// Consider every different sub list permutation in between start and end with min files.
|
||||
for(int currentEnd = start + comConf.getMinFilesToCompact() - 1;
|
||||
currentEnd < candidates.size(); currentEnd++) {
|
||||
List<StoreFile> potentialMatchFiles = candidates.subList(start, currentEnd+1);
|
||||
|
||||
// Sanity checks
|
||||
if (potentialMatchFiles.size() < comConf.getMinFilesToCompact()) continue;
|
||||
if (potentialMatchFiles.size() > comConf.getMaxFilesToCompact()) continue;
|
||||
if (!filesInRatio(potentialMatchFiles, mayUseOffPeak)) continue;
|
||||
|
||||
// Compute the total size of files that will
|
||||
// have to be read if this set of files is compacted.
|
||||
long size = 0;
|
||||
|
||||
for (StoreFile s:potentialMatchFiles) {
|
||||
size += s.getReader().length();
|
||||
}
|
||||
|
||||
// Keep if this gets rid of more files. Or the same number of files for less io.
|
||||
if (potentialMatchFiles.size() > bestSelection.size() ||
|
||||
(potentialMatchFiles.size() == bestSelection.size() && size < bestSize)) {
|
||||
bestSelection = potentialMatchFiles;
|
||||
bestSize = size;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return new ArrayList<StoreFile>(bestSelection);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check that all files satisfy the r
|
||||
* @param files
|
||||
* @return
|
||||
*/
|
||||
private boolean filesInRatio(List<StoreFile> files, boolean isOffPeak) {
|
||||
if (files.size() < 2) {
|
||||
return true;
|
||||
}
|
||||
double currentRatio = isOffPeak ?
|
||||
comConf.getCompactionRatioOffPeak() : comConf.getCompactionRatio();
|
||||
|
||||
long totalFileSize = 0;
|
||||
for (int i = 0; i < files.size(); i++) {
|
||||
totalFileSize += files.get(i).getReader().length();
|
||||
}
|
||||
for (int i = 0; i < files.size(); i++) {
|
||||
long singleFileSize = files.get(i).getReader().length();
|
||||
long sumAllOtherFilesize = totalFileSize - singleFileSize;
|
||||
|
||||
if (( singleFileSize > sumAllOtherFilesize * currentRatio)
|
||||
&& (sumAllOtherFilesize >= comConf.getMinCompactSize())){
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
|
||||
}
|
||||
}
|
|
@ -21,9 +21,7 @@ package org.apache.hadoop.hbase.regionserver.compactions;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Calendar;
|
||||
import java.util.Collection;
|
||||
import java.util.GregorianCalendar;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
|
||||
|
@ -32,11 +30,8 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine;
|
||||
import org.apache.hadoop.hbase.regionserver.HStore;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFileManager;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreUtils;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
|
||||
|
@ -50,10 +45,11 @@ import com.google.common.collect.Collections2;
|
|||
* it's given to produce the list of suitable candidates for compaction.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class DefaultCompactionPolicy extends CompactionPolicy {
|
||||
private static final Log LOG = LogFactory.getLog(DefaultCompactionPolicy.class);
|
||||
public class RatioBasedCompactionPolicy extends CompactionPolicy {
|
||||
private static final Log LOG = LogFactory.getLog(RatioBasedCompactionPolicy.class);
|
||||
|
||||
public DefaultCompactionPolicy(Configuration conf, StoreConfigInformation storeConfigInfo) {
|
||||
public RatioBasedCompactionPolicy(Configuration conf,
|
||||
StoreConfigInformation storeConfigInfo) {
|
||||
super(conf, storeConfigInfo);
|
||||
}
|
||||
|
|
@ -55,7 +55,7 @@ import org.apache.hadoop.hbase.io.hfile.HFileScanner;
|
|||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.Compactor;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactionPolicy;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
@ -321,7 +321,8 @@ public class TestCompaction extends HBaseTestCase {
|
|||
assertEquals(2, s.getStorefilesCount());
|
||||
|
||||
// ensure that major compaction time is deterministic
|
||||
DefaultCompactionPolicy c = (DefaultCompactionPolicy)s.storeEngine.getCompactionPolicy();
|
||||
RatioBasedCompactionPolicy
|
||||
c = (RatioBasedCompactionPolicy)s.storeEngine.getCompactionPolicy();
|
||||
Collection<StoreFile> storeFiles = s.getStorefiles();
|
||||
long mcTime = c.getNextMajorCompactTime(storeFiles);
|
||||
for (int i = 0; i < 10; ++i) {
|
||||
|
|
|
@ -20,8 +20,6 @@ package org.apache.hadoop.hbase.regionserver;
|
|||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Calendar;
|
||||
import java.util.GregorianCalendar;
|
||||
import java.util.List;
|
||||
|
||||
import junit.framework.TestCase;
|
||||
|
@ -40,7 +38,7 @@ import org.apache.hadoop.hbase.SmallTests;
|
|||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.NoOpDataBlockEncoder;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactionPolicy;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -171,6 +169,16 @@ public class TestDefaultCompactSelection extends TestCase {
|
|||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "MockStoreFile{" +
|
||||
"length=" + length +
|
||||
", isRef=" + isRef +
|
||||
", ageInDisk=" + ageInDisk +
|
||||
", sequenceid=" + sequenceid +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
||||
ArrayList<Long> toArrayList(long... numbers) {
|
||||
|
@ -234,7 +242,7 @@ public class TestDefaultCompactSelection extends TestCase {
|
|||
throws IOException {
|
||||
store.forceMajor = forcemajor;
|
||||
//Test Default compactions
|
||||
CompactionRequest result = ((DefaultCompactionPolicy)store.storeEngine.getCompactionPolicy())
|
||||
CompactionRequest result = ((RatioBasedCompactionPolicy)store.storeEngine.getCompactionPolicy())
|
||||
.selectCompaction(candidates, new ArrayList<StoreFile>(), false, isOffPeak, forcemajor);
|
||||
List<StoreFile> actual = new ArrayList<StoreFile>(result.getFiles());
|
||||
if (isOffPeak && !forcemajor) {
|
||||
|
@ -269,7 +277,13 @@ public class TestDefaultCompactSelection extends TestCase {
|
|||
*/
|
||||
// don't exceed max file compact threshold
|
||||
// note: file selection starts with largest to smallest.
|
||||
compactEquals(sfCreate(7, 6, 5, 4, 3, 2, 1), 7, 6, 5, 4, 3);
|
||||
compactEquals(sfCreate(7, 6, 5, 4, 3, 2, 1), 5, 4, 3, 2, 1);
|
||||
|
||||
compactEquals(sfCreate(50, 10, 10 ,10, 10), 10, 10, 10, 10);
|
||||
|
||||
compactEquals(sfCreate(10, 10, 10, 10, 50), 10, 10, 10, 10);
|
||||
|
||||
compactEquals(sfCreate(251, 253, 251, maxSize -1), 251, 253, 251);
|
||||
|
||||
/* MAJOR COMPACTION */
|
||||
// if a major compaction has been forced, then compact everything
|
||||
|
@ -280,7 +294,7 @@ public class TestDefaultCompactSelection extends TestCase {
|
|||
compactEquals(sfCreate(tooBig, 12,12), true, tooBig, 12, 12);
|
||||
// don't exceed max file compact threshold, even with major compaction
|
||||
store.forceMajor = true;
|
||||
compactEquals(sfCreate(7, 6, 5, 4, 3, 2, 1), 7, 6, 5, 4, 3);
|
||||
compactEquals(sfCreate(7, 6, 5, 4, 3, 2, 1), 5, 4, 3, 2, 1);
|
||||
store.forceMajor = false;
|
||||
// if we exceed maxCompactSize, downgrade to minor
|
||||
// if not, it creates a 'snowball effect' when files >> maxCompactSize:
|
||||
|
|
|
@ -23,7 +23,7 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.SmallTests;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactionPolicy;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
@ -38,7 +38,7 @@ public class TestDefaultStoreEngine {
|
|||
}
|
||||
}
|
||||
|
||||
public static class DummyCompactionPolicy extends DefaultCompactionPolicy {
|
||||
public static class DummyCompactionPolicy extends RatioBasedCompactionPolicy {
|
||||
public DummyCompactionPolicy(Configuration conf, StoreConfigInformation storeConfigInfo) {
|
||||
super(conf, storeConfigInfo);
|
||||
}
|
||||
|
|
|
@ -26,7 +26,6 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.SmallTests;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.regionserver.HStore;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
|
@ -53,7 +52,7 @@ public class PerfTestCompactionPolicies {
|
|||
|
||||
static final Log LOG = LogFactory.getLog(PerfTestCompactionPolicies.class);
|
||||
|
||||
private final DefaultCompactionPolicy cp;
|
||||
private final RatioBasedCompactionPolicy cp;
|
||||
private final int max;
|
||||
private final int min;
|
||||
private final float ratio;
|
||||
|
@ -64,11 +63,16 @@ public class PerfTestCompactionPolicies {
|
|||
@Parameterized.Parameters
|
||||
public static Collection<Object[]> data() {
|
||||
return Arrays.asList(new Object[][] {
|
||||
{DefaultCompactionPolicy.class, 3, 2, 1.2f},
|
||||
{DefaultCompactionPolicy.class, 4, 2, 1.2f},
|
||||
{DefaultCompactionPolicy.class, 5, 2, 1.2f},
|
||||
{DefaultCompactionPolicy.class, 4, 2, 1.3f},
|
||||
{DefaultCompactionPolicy.class, 4, 2, 1.4f},
|
||||
{RatioBasedCompactionPolicy.class, 3, 2, 1.2f},
|
||||
{ExploringCompactionPolicy.class, 3, 2, 1.2f},
|
||||
{RatioBasedCompactionPolicy.class, 4, 2, 1.2f},
|
||||
{ExploringCompactionPolicy.class, 4, 2, 1.2f},
|
||||
{RatioBasedCompactionPolicy.class, 5, 2, 1.2f},
|
||||
{ExploringCompactionPolicy.class, 5, 2, 1.2f},
|
||||
{RatioBasedCompactionPolicy.class, 4, 2, 1.3f},
|
||||
{ExploringCompactionPolicy.class, 4, 2, 1.3f},
|
||||
{RatioBasedCompactionPolicy.class, 4, 2, 1.4f},
|
||||
{ExploringCompactionPolicy.class, 4, 2, 1.4f},
|
||||
|
||||
});
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue