HBASE-10501 Improve IncreasingToUpperBoundRegionSplitPolicy to avoid too many regions.
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1569507 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
6b81ac7c93
commit
09e16b32d0
|
@ -29,30 +29,34 @@ import org.apache.hadoop.hbase.HTableDescriptor;
|
|||
|
||||
/**
|
||||
* Split size is the number of regions that are on this server that all are
|
||||
* of the same table, squared, times the region flush size OR the maximum
|
||||
* 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 on first flush we will split which will make two regions
|
||||
* that will split when their size is 2 * 2 * 128M = 512M. If one of these
|
||||
* 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 = 1152M, and so on until we reach the configured
|
||||
* 3^3 * 128M*2 = 6912M, and so on until we reach the configured
|
||||
* maximum filesize and then from there on out, we'll use that.
|
||||
*/
|
||||
public class IncreasingToUpperBoundRegionSplitPolicy
|
||||
extends ConstantSizeRegionSplitPolicy {
|
||||
static final Log LOG =
|
||||
LogFactory.getLog(IncreasingToUpperBoundRegionSplitPolicy.class);
|
||||
private long flushSize;
|
||||
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) {
|
||||
return;
|
||||
}
|
||||
HTableDescriptor desc = region.getTableDesc();
|
||||
if (desc != null) {
|
||||
this.flushSize = desc.getMemStoreFlushSize();
|
||||
this.initialSize = 2*desc.getMemStoreFlushSize();
|
||||
}
|
||||
if (this.flushSize <= 0) {
|
||||
this.flushSize = conf.getLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE,
|
||||
if (this.initialSize <= 0) {
|
||||
this.initialSize = 2*conf.getLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE,
|
||||
HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE);
|
||||
}
|
||||
}
|
||||
|
@ -90,10 +94,11 @@ extends ConstantSizeRegionSplitPolicy {
|
|||
* @return Region max size or <code>count of regions squared * flushsize, which ever is
|
||||
* smaller; guard against there being zero regions on this server.
|
||||
*/
|
||||
long getSizeToCheck(final int tableRegionsCount) {
|
||||
return tableRegionsCount == 0? getDesiredMaxFileSize():
|
||||
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.flushSize * (tableRegionsCount * (long)tableRegionsCount));
|
||||
this.initialSize * tableRegionsCount * tableRegionsCount * tableRegionsCount);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -76,9 +76,9 @@ public class TestRegionSplitPolicy {
|
|||
// Set max size for this 'table'.
|
||||
long maxSplitSize = 1024L;
|
||||
htd.setMaxFileSize(maxSplitSize);
|
||||
// Set flush size to 1/4. IncreasingToUpperBoundRegionSplitPolicy
|
||||
// grows by the square of the number of regions times flushsize each time.
|
||||
long flushSize = maxSplitSize/4;
|
||||
// Set flush size to 1/8. IncreasingToUpperBoundRegionSplitPolicy
|
||||
// grows by the cube of the number of regions times flushsize each time.
|
||||
long flushSize = maxSplitSize/8;
|
||||
conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, flushSize);
|
||||
htd.setMemStoreFlushSize(flushSize);
|
||||
// If RegionServerService with no regions in it -- 'online regions' == 0 --
|
||||
|
@ -101,18 +101,18 @@ public class TestRegionSplitPolicy {
|
|||
// Now test that we increase our split size as online regions for a table
|
||||
// grows. With one region, split size should be flushsize.
|
||||
regions.add(mockRegion);
|
||||
Mockito.doReturn(flushSize/2).when(mockStore).getSize();
|
||||
// Should not split since store is 1/2 flush size.
|
||||
Mockito.doReturn(flushSize).when(mockStore).getSize();
|
||||
// Should not split since store is flush size.
|
||||
assertFalse(policy.shouldSplit());
|
||||
// Set size of store to be > flush size and we should split
|
||||
Mockito.doReturn(flushSize + 1).when(mockStore).getSize();
|
||||
// Set size of store to be > 2*flush size and we should split
|
||||
Mockito.doReturn(flushSize*2 + 1).when(mockStore).getSize();
|
||||
assertTrue(policy.shouldSplit());
|
||||
// Add another region to the 'online regions' on this server and we should
|
||||
// now be no longer be splittable since split size has gone up.
|
||||
regions.add(mockRegion);
|
||||
assertFalse(policy.shouldSplit());
|
||||
// Quadruple (2 squared) the store size and make sure its just over; verify it'll split
|
||||
Mockito.doReturn((flushSize * 2 * 2) + 1).when(mockStore).getSize();
|
||||
Mockito.doReturn((flushSize * 2 * 2 * 2) + 1).when(mockStore).getSize();
|
||||
assertTrue(policy.shouldSplit());
|
||||
|
||||
// Finally assert that even if loads of regions, we'll split at max size
|
||||
|
|
Loading…
Reference in New Issue