HBASE-4381 Refactor split decisions into a split policy class.
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1170468 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
1df9e04262
commit
4a448e4d74
|
@ -479,6 +479,7 @@ Release 0.91.0 - Unreleased
|
|||
(Harsh J)
|
||||
HBASE-4287 If region opening fails, change region in transition into
|
||||
a FAILED_OPEN state so that it can be retried quickly. (todd)
|
||||
HBASE-4381 Refactor split decisions into a split policy class. (todd)
|
||||
|
||||
TASKS
|
||||
HBASE-3559 Move report of split to master OFF the heartbeat channel
|
||||
|
|
|
@ -488,7 +488,10 @@ public final class HConstants {
|
|||
public static final String HBASE_MASTER_LOGCLEANER_PLUGINS =
|
||||
"hbase.master.logcleaner.plugins";
|
||||
|
||||
/*
|
||||
public static final String HBASE_REGION_SPLIT_POLICY_KEY =
|
||||
"hbase.regionserver.region.split.policy";
|
||||
|
||||
/*
|
||||
* Minimum percentage of free heap necessary for a successful cluster startup.
|
||||
*/
|
||||
public static final float HBASE_CLUSTER_MINIMUM_MEMORY_THRESHOLD = 0.2f;
|
||||
|
|
|
@ -63,6 +63,12 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
|
|||
public static final String MAX_FILESIZE = "MAX_FILESIZE";
|
||||
public static final ImmutableBytesWritable MAX_FILESIZE_KEY =
|
||||
new ImmutableBytesWritable(Bytes.toBytes(MAX_FILESIZE));
|
||||
|
||||
// The class which determines when a region should split
|
||||
public static final String SPLIT_POLICY = "SPLIT_POLICY";
|
||||
public static final ImmutableBytesWritable SPLIT_POLICY_KEY =
|
||||
new ImmutableBytesWritable(Bytes.toBytes(SPLIT_POLICY));
|
||||
|
||||
public static final String READONLY = "READONLY";
|
||||
public static final ImmutableBytesWritable READONLY_KEY =
|
||||
new ImmutableBytesWritable(Bytes.toBytes(READONLY));
|
||||
|
@ -422,6 +428,15 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
|
|||
return Long.valueOf(Bytes.toString(value)).longValue();
|
||||
return HConstants.DEFAULT_MAX_FILE_SIZE;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the class name of the region split policy for this table.
|
||||
* If this returns null, the default constant size based split policy
|
||||
* is used.
|
||||
*/
|
||||
public String getRegionSplitPolicyClassName() {
|
||||
return getValue(SPLIT_POLICY);
|
||||
}
|
||||
|
||||
/** @param name name of table */
|
||||
public void setName(byte[] name) {
|
||||
|
|
|
@ -0,0 +1,67 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
|
||||
/**
|
||||
* A {@link RegionSplitPolicy} implementation which splits a region
|
||||
* as soon as any of its store files exceeds a maximum configurable
|
||||
* size.
|
||||
*/
|
||||
class ConstantSizeRegionSplitPolicy extends RegionSplitPolicy {
|
||||
private long desiredMaxFileSize;
|
||||
|
||||
@Override
|
||||
void configureForRegion(HRegion region) {
|
||||
super.configureForRegion(region);
|
||||
long maxFileSize = region.getTableDesc().getMaxFileSize();
|
||||
|
||||
// By default we split region if a file > HConstants.DEFAULT_MAX_FILE_SIZE.
|
||||
if (maxFileSize == HConstants.DEFAULT_MAX_FILE_SIZE) {
|
||||
maxFileSize = getConf().getLong("hbase.hregion.max.filesize",
|
||||
HConstants.DEFAULT_MAX_FILE_SIZE);
|
||||
}
|
||||
this.desiredMaxFileSize = maxFileSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
boolean shouldSplit() {
|
||||
boolean force = region.shouldForceSplit();
|
||||
boolean foundABigStore = false;
|
||||
|
||||
for (Store store : region.getStores().values()) {
|
||||
// If any of the stores are unable to split (eg they contain reference files)
|
||||
// then don't split
|
||||
if ((!store.canSplit())) {
|
||||
return false;
|
||||
}
|
||||
|
||||
// Mark if any store is big enough
|
||||
if (store.getSize() > desiredMaxFileSize) {
|
||||
foundABigStore = true;
|
||||
}
|
||||
}
|
||||
|
||||
return foundABigStore || force;
|
||||
}
|
||||
|
||||
long getDesiredMaxFileSize() {
|
||||
return desiredMaxFileSize;
|
||||
}
|
||||
}
|
|
@ -257,7 +257,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
private final ReentrantReadWriteLock updatesLock =
|
||||
new ReentrantReadWriteLock();
|
||||
private boolean splitRequest;
|
||||
private byte[] splitPoint = null;
|
||||
private byte[] explicitSplitPoint = null;
|
||||
|
||||
private final ReadWriteConsistencyControl rwcc =
|
||||
new ReadWriteConsistencyControl();
|
||||
|
@ -270,6 +270,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
*/
|
||||
public final static String REGIONINFO_FILE = ".regioninfo";
|
||||
private HTableDescriptor htableDescriptor = null;
|
||||
private RegionSplitPolicy splitPolicy;
|
||||
|
||||
/**
|
||||
* Should only be used for testing purposes
|
||||
|
@ -419,6 +420,10 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
this.writestate.setReadOnly(this.htableDescriptor.isReadOnly());
|
||||
|
||||
this.writestate.compacting = 0;
|
||||
|
||||
// Initialize split policy
|
||||
this.splitPolicy = RegionSplitPolicy.create(this, conf);
|
||||
|
||||
this.lastFlushTime = EnvironmentEdgeManager.currentTimeMillis();
|
||||
// Use maximum of log sequenceid or that which was found in stores
|
||||
// (particularly if no recovered edits, seqid will be -1).
|
||||
|
@ -1285,7 +1290,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
}
|
||||
// look across all the HStores for this region and determine what the
|
||||
// closest key is across all column families, since the data may be sparse
|
||||
checkRow(row);
|
||||
checkRow(row, "getClosestRowBefore");
|
||||
startRegionOperation();
|
||||
this.readRequestsCount.increment();
|
||||
try {
|
||||
|
@ -2403,10 +2408,10 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
//////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
/** Make sure this is a valid row for the HRegion */
|
||||
private void checkRow(final byte [] row) throws IOException {
|
||||
private void checkRow(final byte [] row, String op) throws IOException {
|
||||
if(!rowIsInRange(regionInfo, row)) {
|
||||
throw new WrongRegionException("Requested row out of range for " +
|
||||
"HRegion " + this + ", startKey='" +
|
||||
op + "on HRegion " + this + ", startKey='" +
|
||||
Bytes.toStringBinary(regionInfo.getStartKey()) + "', getEndKey()='" +
|
||||
Bytes.toStringBinary(regionInfo.getEndKey()) + "', row='" +
|
||||
Bytes.toStringBinary(row) + "'");
|
||||
|
@ -2454,7 +2459,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
*/
|
||||
private Integer internalObtainRowLock(final byte[] row, boolean waitForLock)
|
||||
throws IOException {
|
||||
checkRow(row);
|
||||
checkRow(row, "row lock");
|
||||
startRegionOperation();
|
||||
try {
|
||||
HashedBytes rowKey = new HashedBytes(row);
|
||||
|
@ -3490,7 +3495,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
throws IOException {
|
||||
// TODO: Use RWCC to make this set of increments atomic to reads
|
||||
byte [] row = increment.getRow();
|
||||
checkRow(row);
|
||||
checkRow(row, "increment");
|
||||
TimeRange tr = increment.getTimeRange();
|
||||
boolean flush = false;
|
||||
WALEdit walEdits = null;
|
||||
|
@ -3591,7 +3596,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
public long incrementColumnValue(byte [] row, byte [] family,
|
||||
byte [] qualifier, long amount, boolean writeToWAL)
|
||||
throws IOException {
|
||||
checkRow(row);
|
||||
checkRow(row, "increment");
|
||||
boolean flush = false;
|
||||
boolean wrongLength = false;
|
||||
// Lock row
|
||||
|
@ -3877,8 +3882,8 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
return this.splitRequest;
|
||||
}
|
||||
|
||||
byte[] getSplitPoint() {
|
||||
return this.splitPoint;
|
||||
byte[] getExplicitSplitPoint() {
|
||||
return this.explicitSplitPoint;
|
||||
}
|
||||
|
||||
void forceSplit(byte[] sp) {
|
||||
|
@ -3886,7 +3891,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
// therefore, no reason to clear this value
|
||||
this.splitRequest = true;
|
||||
if (sp != null) {
|
||||
this.splitPoint = sp;
|
||||
this.explicitSplitPoint = sp;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -3908,20 +3913,33 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
* is based on the size of the store.
|
||||
*/
|
||||
public byte[] checkSplit() {
|
||||
if (this.splitPoint != null) {
|
||||
return this.splitPoint;
|
||||
}
|
||||
byte[] splitPointFromLargestStore = null;
|
||||
long largestStoreSize = 0;
|
||||
for (Store s : stores.values()) {
|
||||
byte[] splitPoint = s.checkSplit();
|
||||
long storeSize = s.getSize();
|
||||
if (splitPoint != null && largestStoreSize < storeSize) {
|
||||
splitPointFromLargestStore = splitPoint;
|
||||
largestStoreSize = storeSize;
|
||||
// Can't split META
|
||||
if (getRegionInfo().isMetaRegion()) {
|
||||
if (shouldForceSplit()) {
|
||||
LOG.warn("Cannot split meta regions in HBase 0.20 and above");
|
||||
}
|
||||
return null;
|
||||
}
|
||||
return splitPointFromLargestStore;
|
||||
|
||||
if (this.explicitSplitPoint != null) {
|
||||
return this.explicitSplitPoint;
|
||||
}
|
||||
|
||||
if (!splitPolicy.shouldSplit()) {
|
||||
return null;
|
||||
}
|
||||
|
||||
byte[] ret = splitPolicy.getSplitPoint();
|
||||
|
||||
if (ret != null) {
|
||||
try {
|
||||
checkRow(ret, "calculated split");
|
||||
} catch (IOException e) {
|
||||
LOG.error("Ignoring invalid split", e);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -0,0 +1,122 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.conf.Configured;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.util.ReflectionUtils;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* A split policy determines when a region should be split.
|
||||
* {@see ConstantSizeRegionSplitPolicy}
|
||||
*/
|
||||
abstract class RegionSplitPolicy extends Configured {
|
||||
private static final Class<ConstantSizeRegionSplitPolicy>
|
||||
DEFAULT_SPLIT_POLICY_CLASS = ConstantSizeRegionSplitPolicy.class;
|
||||
|
||||
/**
|
||||
* The region configured for this split policy.
|
||||
*/
|
||||
protected HRegion region;
|
||||
|
||||
/**
|
||||
* Upon construction, this method will be called with the region
|
||||
* to be governed. It will be called once and only once.
|
||||
*/
|
||||
void configureForRegion(HRegion region) {
|
||||
Preconditions.checkState(
|
||||
this.region == null,
|
||||
"Policy already configured for region {}",
|
||||
this.region);
|
||||
|
||||
this.region = region;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return true if the specified region should be split.
|
||||
*/
|
||||
abstract boolean shouldSplit();
|
||||
|
||||
/**
|
||||
* @return the key at which the region should be split, or null
|
||||
* if it cannot be split. This will only be called if shouldSplit
|
||||
* previously returned true.
|
||||
*/
|
||||
byte[] getSplitPoint() {
|
||||
Map<byte[], Store> stores = region.getStores();
|
||||
|
||||
byte[] splitPointFromLargestStore = null;
|
||||
long largestStoreSize = 0;
|
||||
for (Store s : stores.values()) {
|
||||
byte[] splitPoint = s.getSplitPoint();
|
||||
long storeSize = s.getSize();
|
||||
if (splitPoint != null && largestStoreSize < storeSize) {
|
||||
splitPointFromLargestStore = splitPoint;
|
||||
largestStoreSize = storeSize;
|
||||
}
|
||||
}
|
||||
|
||||
return splitPointFromLargestStore;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create the RegionSplitPolicy configured for the given table.
|
||||
* Each
|
||||
* @param htd
|
||||
* @param conf
|
||||
* @return
|
||||
* @throws IOException
|
||||
*/
|
||||
public static RegionSplitPolicy create(HRegion region,
|
||||
Configuration conf) throws IOException {
|
||||
|
||||
Class<? extends RegionSplitPolicy> clazz = getSplitPolicyClass(
|
||||
region.getTableDesc(), conf);
|
||||
RegionSplitPolicy policy = ReflectionUtils.newInstance(clazz, conf);
|
||||
policy.configureForRegion(region);
|
||||
return policy;
|
||||
}
|
||||
|
||||
static Class<? extends RegionSplitPolicy> getSplitPolicyClass(
|
||||
HTableDescriptor htd, Configuration conf) throws IOException {
|
||||
String className = htd.getRegionSplitPolicyClassName();
|
||||
if (className == null) {
|
||||
className = conf.get(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
|
||||
DEFAULT_SPLIT_POLICY_CLASS.getName());
|
||||
}
|
||||
|
||||
try {
|
||||
Class<? extends RegionSplitPolicy> clazz =
|
||||
Class.forName(className).asSubclass(RegionSplitPolicy.class);
|
||||
return clazz;
|
||||
} catch (Exception e) {
|
||||
throw new IOException(
|
||||
"Unable to load configured region split policy '" +
|
||||
className + "' for table '" + htd.getNameAsString() + "'",
|
||||
e);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -110,7 +110,6 @@ public class Store implements HeapSize {
|
|||
volatile boolean forceMajor = false;
|
||||
/* how many bytes to write between status checks */
|
||||
static int closeCheckInterval = 0;
|
||||
private final long desiredMaxFileSize;
|
||||
private final int blockingStoreFileCount;
|
||||
private volatile long storeSize = 0L;
|
||||
private volatile long totalUncompressedBytes = 0L;
|
||||
|
@ -200,18 +199,6 @@ public class Store implements HeapSize {
|
|||
this.inMemory = family.isInMemory();
|
||||
long maxFileSize = 0L;
|
||||
HTableDescriptor hTableDescriptor = region.getTableDesc();
|
||||
if (hTableDescriptor != null) {
|
||||
maxFileSize = hTableDescriptor.getMaxFileSize();
|
||||
} else {
|
||||
maxFileSize = HConstants.DEFAULT_MAX_FILE_SIZE;
|
||||
}
|
||||
|
||||
// By default we split region if a file > HConstants.DEFAULT_MAX_FILE_SIZE.
|
||||
if (maxFileSize == HConstants.DEFAULT_MAX_FILE_SIZE) {
|
||||
maxFileSize = conf.getLong("hbase.hregion.max.filesize",
|
||||
HConstants.DEFAULT_MAX_FILE_SIZE);
|
||||
}
|
||||
this.desiredMaxFileSize = maxFileSize;
|
||||
this.blockingStoreFileCount =
|
||||
conf.getInt("hbase.hstore.blockingStoreFiles", 7);
|
||||
|
||||
|
@ -1471,49 +1458,54 @@ public class Store implements HeapSize {
|
|||
return foundCandidate;
|
||||
}
|
||||
|
||||
public boolean canSplit() {
|
||||
this.lock.readLock().lock();
|
||||
try {
|
||||
// Not splitable if we find a reference store file present in the store.
|
||||
for (StoreFile sf : storefiles) {
|
||||
if (sf.isReference()) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(sf + " is not splittable");
|
||||
}
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
} finally {
|
||||
this.lock.readLock().unlock();
|
||||
}
|
||||
}
|
||||
/**
|
||||
* Determines if Store should be split
|
||||
* @return byte[] if store should be split, null otherwise.
|
||||
*/
|
||||
public byte[] checkSplit() {
|
||||
public byte[] getSplitPoint() {
|
||||
this.lock.readLock().lock();
|
||||
try {
|
||||
boolean force = this.region.shouldForceSplit();
|
||||
// sanity checks
|
||||
if (this.storefiles.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
if (!force && storeSize < this.desiredMaxFileSize) {
|
||||
return null;
|
||||
}
|
||||
|
||||
if (this.region.getRegionInfo().isMetaRegion()) {
|
||||
if (force) {
|
||||
LOG.warn("Cannot split meta regions in HBase 0.20");
|
||||
}
|
||||
return null;
|
||||
}
|
||||
// Should already be enforced by the split policy!
|
||||
assert !this.region.getRegionInfo().isMetaRegion();
|
||||
|
||||
// Not splitable if we find a reference store file present in the store.
|
||||
boolean splitable = true;
|
||||
long maxSize = 0L;
|
||||
StoreFile largestSf = null;
|
||||
for (StoreFile sf : storefiles) {
|
||||
if (splitable) {
|
||||
splitable = !sf.isReference();
|
||||
if (!splitable) {
|
||||
// RETURN IN MIDDLE OF FUNCTION!!! If not splitable, just return.
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(sf + " is not splittable");
|
||||
}
|
||||
return null;
|
||||
}
|
||||
if (sf.isReference()) {
|
||||
// Should already be enforced since we return false in this case
|
||||
assert false : "getSplitPoint() called on a region that can't split!";
|
||||
return null;
|
||||
}
|
||||
|
||||
StoreFile.Reader r = sf.getReader();
|
||||
if (r == null) {
|
||||
LOG.warn("Storefile " + sf + " Reader is null");
|
||||
continue;
|
||||
}
|
||||
|
||||
long size = r.length();
|
||||
if (size > maxSize) {
|
||||
// This is the largest one so far
|
||||
|
@ -1521,10 +1513,7 @@ public class Store implements HeapSize {
|
|||
largestSf = sf;
|
||||
}
|
||||
}
|
||||
// if the user explicit set a split point, use that
|
||||
if (this.region.getSplitPoint() != null) {
|
||||
return this.region.getSplitPoint();
|
||||
}
|
||||
|
||||
StoreFile.Reader r = largestSf.getReader();
|
||||
if (r == null) {
|
||||
LOG.warn("Storefile " + largestSf + " Reader is null");
|
||||
|
|
|
@ -0,0 +1,141 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
public class TestRegionSplitPolicy {
|
||||
|
||||
private Configuration conf;
|
||||
private HTableDescriptor htd;
|
||||
private HRegion mockRegion;
|
||||
private TreeMap<byte[], Store> stores;
|
||||
|
||||
@Before
|
||||
public void setupMocks() {
|
||||
conf = HBaseConfiguration.create();
|
||||
|
||||
HRegionInfo hri = new HRegionInfo(Bytes.toBytes("testtable"));
|
||||
|
||||
htd = new HTableDescriptor();
|
||||
mockRegion = Mockito.mock(HRegion.class);
|
||||
Mockito.doReturn(htd).when(mockRegion).getTableDesc();
|
||||
Mockito.doReturn(hri).when(mockRegion).getRegionInfo();
|
||||
|
||||
stores = new TreeMap<byte[], Store>(Bytes.BYTES_COMPARATOR);
|
||||
Mockito.doReturn(stores).when(mockRegion).getStores();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCreateDefault() throws IOException {
|
||||
conf.setLong("hbase.hregion.max.filesize", 1234L);
|
||||
|
||||
// Using a default HTD, should pick up the file size from
|
||||
// configuration.
|
||||
ConstantSizeRegionSplitPolicy policy =
|
||||
(ConstantSizeRegionSplitPolicy)RegionSplitPolicy.create(
|
||||
mockRegion, conf);
|
||||
assertEquals(1234L, policy.getDesiredMaxFileSize());
|
||||
|
||||
// If specified in HTD, should use that
|
||||
htd.setMaxFileSize(9999L);
|
||||
policy = (ConstantSizeRegionSplitPolicy)RegionSplitPolicy.create(
|
||||
mockRegion, conf);
|
||||
assertEquals(9999L, policy.getDesiredMaxFileSize());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConstantSizePolicy() throws IOException {
|
||||
htd.setMaxFileSize(1024L);
|
||||
|
||||
ConstantSizeRegionSplitPolicy policy =
|
||||
(ConstantSizeRegionSplitPolicy)RegionSplitPolicy.create(mockRegion, conf);
|
||||
|
||||
// For no stores, should not split
|
||||
assertFalse(policy.shouldSplit());
|
||||
|
||||
// Add a store above the requisite size. Should split.
|
||||
Store mockStore = Mockito.mock(Store.class);
|
||||
Mockito.doReturn(2000L).when(mockStore).getSize();
|
||||
Mockito.doReturn(true).when(mockStore).canSplit();
|
||||
stores.put(new byte[]{1}, mockStore);
|
||||
|
||||
assertTrue(policy.shouldSplit());
|
||||
|
||||
// Act as if there's a reference file or some other reason it can't split.
|
||||
// This should prevent splitting even though it's big enough.
|
||||
Mockito.doReturn(false).when(mockStore).canSplit();
|
||||
assertFalse(policy.shouldSplit());
|
||||
|
||||
// Reset splittability after above
|
||||
Mockito.doReturn(true).when(mockStore).canSplit();
|
||||
|
||||
// Set to a small size but turn on forceSplit. Should result in a split.
|
||||
Mockito.doReturn(true).when(mockRegion).shouldForceSplit();
|
||||
Mockito.doReturn(100L).when(mockStore).getSize();
|
||||
assertTrue(policy.shouldSplit());
|
||||
|
||||
// Turn off forceSplit, should not split
|
||||
Mockito.doReturn(false).when(mockRegion).shouldForceSplit();
|
||||
assertFalse(policy.shouldSplit());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetSplitPoint() throws IOException {
|
||||
ConstantSizeRegionSplitPolicy policy =
|
||||
(ConstantSizeRegionSplitPolicy)RegionSplitPolicy.create(mockRegion, conf);
|
||||
|
||||
// For no stores, should not split
|
||||
assertFalse(policy.shouldSplit());
|
||||
assertNull(policy.getSplitPoint());
|
||||
|
||||
// Add a store above the requisite size. Should split.
|
||||
Store mockStore = Mockito.mock(Store.class);
|
||||
Mockito.doReturn(2000L).when(mockStore).getSize();
|
||||
Mockito.doReturn(true).when(mockStore).canSplit();
|
||||
Mockito.doReturn(Bytes.toBytes("store 1 split"))
|
||||
.when(mockStore).getSplitPoint();
|
||||
stores.put(new byte[]{1}, mockStore);
|
||||
|
||||
assertEquals("store 1 split",
|
||||
Bytes.toString(policy.getSplitPoint()));
|
||||
|
||||
// Add a bigger store. The split point should come from that one
|
||||
Store mockStore2 = Mockito.mock(Store.class);
|
||||
Mockito.doReturn(4000L).when(mockStore2).getSize();
|
||||
Mockito.doReturn(true).when(mockStore2).canSplit();
|
||||
Mockito.doReturn(Bytes.toBytes("store 2 split"))
|
||||
.when(mockStore2).getSplitPoint();
|
||||
stores.put(new byte[]{2}, mockStore2);
|
||||
|
||||
assertEquals("store 2 split",
|
||||
Bytes.toString(policy.getSplitPoint()));
|
||||
}
|
||||
}
|
|
@ -700,9 +700,9 @@ public class TestStore extends TestCase {
|
|||
*/
|
||||
public void testSplitWithEmptyColFam() throws IOException {
|
||||
init(this.getName());
|
||||
assertNull(store.checkSplit());
|
||||
assertNull(store.getSplitPoint());
|
||||
store.getHRegion().forceSplit(null);
|
||||
assertNull(store.checkSplit());
|
||||
assertNull(store.getSplitPoint());
|
||||
store.getHRegion().clearSplit_TESTS_ONLY();
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue