HBASE-14461 Cleanup IncreasingToUpperBoundRegionSplitPolicy (Lars Francke)

This commit is contained in:
stack 2015-09-22 09:40:32 -07:00
parent 4b5dd8ee5a
commit da7003ade2
1 changed files with 45 additions and 38 deletions

View File

@ -22,50 +22,54 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
/**
* Split size is the number of regions that are on this server that all are
* of the same table, cubed, times 2x the region flush size OR the maximum
* region split size, whichever is smaller. For example, if the flush size
* is 128M, then after two flushes (256MB) we will split which will make two regions
* that will split when their size is 2^3 * 128M*2 = 2048M. If one of these
* regions splits, then there are three regions and now the split size is
* 3^3 * 128M*2 = 6912M, and so on until we reach the configured
* region split size, whichever is smaller.
* <p>
* For example, if the flush size is 128MB, then after two flushes (256MB) we
* will split which will make two regions that will split when their size is
* {@code 2^3 * 128MB*2 = 2048MB}.
* <p>
* If one of these regions splits, then there are three regions and now the
* split size is {@code 3^3 * 128MB*2 = 6912MB}, and so on until we reach the configured
* maximum file size and then from there on out, we'll use that.
*/
@InterfaceAudience.Private
public class IncreasingToUpperBoundRegionSplitPolicy
extends ConstantSizeRegionSplitPolicy {
private static final Log LOG =
LogFactory.getLog(IncreasingToUpperBoundRegionSplitPolicy.class);
public class IncreasingToUpperBoundRegionSplitPolicy extends ConstantSizeRegionSplitPolicy {
private static final Log LOG = LogFactory.getLog(IncreasingToUpperBoundRegionSplitPolicy.class);
private long initialSize;
@Override
protected void configureForRegion(HRegion region) {
super.configureForRegion(region);
Configuration conf = getConf();
this.initialSize = conf.getLong("hbase.increasing.policy.initial.size", -1);
if (this.initialSize > 0) {
initialSize = conf.getLong("hbase.increasing.policy.initial.size", -1);
if (initialSize > 0) {
return;
}
HTableDescriptor desc = region.getTableDesc();
if (desc != null) {
this.initialSize = 2*desc.getMemStoreFlushSize();
initialSize = 2 * desc.getMemStoreFlushSize();
}
if (this.initialSize <= 0) {
this.initialSize = 2*conf.getLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE,
if (initialSize <= 0) {
initialSize = 2 * conf.getLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE,
HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE);
}
}
@Override
protected boolean shouldSplit() {
if (region.shouldForceSplit()) return true;
if (region.shouldForceSplit()) {
return true;
}
boolean foundABigStore = false;
// Get count of regions that have the same common table as this.region
int tableRegionsCount = getCountOfCommonTableRegions();
@ -75,16 +79,16 @@ extends ConstantSizeRegionSplitPolicy {
for (Store store : region.getStores()) {
// If any of the stores is unable to split (eg they contain reference files)
// then don't split
if ((!store.canSplit())) {
if (!store.canSplit()) {
return false;
}
// Mark if any store is big enough
long size = store.getSize();
if (size > sizeToCheck) {
LOG.debug("ShouldSplit because " + store.getColumnFamilyName() +
" size=" + size + ", sizeToCheck=" + sizeToCheck +
", regionsWithCommonTable=" + tableRegionsCount);
LOG.debug("ShouldSplit because " + store.getColumnFamilyName() + " size=" + size
+ ", sizeToCheck=" + sizeToCheck + ", regionsWithCommonTable="
+ tableRegionsCount);
foundABigStore = true;
}
}
@ -92,26 +96,17 @@ extends ConstantSizeRegionSplitPolicy {
return foundABigStore;
}
/**
* @return Region max size or <code>count of regions squared * flushsize</code>,
* which ever is smaller; guard against there being zero regions on this server.
*/
protected long getSizeToCheck(final int tableRegionsCount) {
// safety check for 100 to avoid numerical overflow in extreme cases
return tableRegionsCount == 0 || tableRegionsCount > 100 ? getDesiredMaxFileSize():
Math.min(getDesiredMaxFileSize(),
this.initialSize * tableRegionsCount * tableRegionsCount * tableRegionsCount);
}
/**
* @return Count of regions on this server that share the table this.region
* belongs to
*/
private int getCountOfCommonTableRegions() {
RegionServerServices rss = this.region.getRegionServerServices();
RegionServerServices rss = region.getRegionServerServices();
// Can be null in tests
if (rss == null) return 0;
TableName tablename = this.region.getTableDesc().getTableName();
if (rss == null) {
return 0;
}
TableName tablename = region.getTableDesc().getTableName();
int tableRegionsCount = 0;
try {
List<Region> hri = rss.getOnlineRegions(tablename);
@ -121,4 +116,16 @@ extends ConstantSizeRegionSplitPolicy {
}
return tableRegionsCount;
}
/**
* @return Region max size or {@code count of regions cubed * flushsize},
* which ever is smaller; guard against there being zero regions on this server.
*/
protected long getSizeToCheck(final int tableRegionsCount) {
// safety check for 100 to avoid numerical overflow in extreme cases
return tableRegionsCount == 0 || tableRegionsCount > 100
? getDesiredMaxFileSize()
: Math.min(getDesiredMaxFileSize(),
initialSize * tableRegionsCount * tableRegionsCount * tableRegionsCount);
}
}