HBASE-25518 Support separate child regions to different region servers (#3001)
Signed-off-by: stack <stack@apache.org>
This commit is contained in:
parent
d93035a131
commit
585aca1f05
|
@ -156,6 +156,21 @@ public final class HConstants {
|
|||
/** Default value for the balancer period */
|
||||
public static final int DEFAULT_HBASE_BALANCER_PERIOD = 300000;
|
||||
|
||||
/**
|
||||
* Config key for enable/disable automatically separate child regions to different region servers
|
||||
* in the procedure of split regions. One child will be kept to the server where parent
|
||||
* region is on, and the other child will be assigned to a random server.
|
||||
* See HBASE-25518.
|
||||
*/
|
||||
public static final String HBASE_ENABLE_SEPARATE_CHILD_REGIONS =
|
||||
"hbase.master.auto.separate.child.regions.after.split.enabled";
|
||||
|
||||
/**
|
||||
* Default value for automatically separate child regions to different region servers
|
||||
* (set to "false" to keep all child regions to the server where parent region is on)
|
||||
*/
|
||||
public static final boolean DEFAULT_HBASE_ENABLE_SEPARATE_CHILD_REGIONS = false;
|
||||
|
||||
/** The name of the ensemble table */
|
||||
public static final TableName ENSEMBLE_TABLE_NAME = TableName.valueOf("hbase:ensemble");
|
||||
|
||||
|
|
|
@ -27,6 +27,7 @@ import java.util.stream.IntStream;
|
|||
import java.util.stream.Stream;
|
||||
import org.apache.commons.lang3.ArrayUtils;
|
||||
import org.apache.hadoop.hbase.HBaseIOException;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.client.AsyncRegionServerAdmin;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
|
@ -40,6 +41,7 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
|
||||
import static org.apache.hadoop.hbase.HConstants.DEFAULT_HBASE_ENABLE_SEPARATE_CHILD_REGIONS;
|
||||
|
||||
/**
|
||||
* Utility for this assignment package only.
|
||||
|
@ -189,6 +191,67 @@ final class AssignmentManagerUtil {
|
|||
return ArrayUtils.addAll(primaryRegionProcs, replicaRegionAssignProcs);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create round robin assign procedures for the given regions,
|
||||
* according to the {@code regionReplication}.
|
||||
* <p/>
|
||||
* For rolling back, we will submit procedures directly to the {@code ProcedureExecutor}, so it is
|
||||
* possible that we persist the newly scheduled procedures, and then crash before persisting the
|
||||
* rollback state, so when we arrive here the second time, it is possible that some regions have
|
||||
* already been associated with a TRSP.
|
||||
* @param ignoreIfInTransition if true, will skip creating TRSP for the given region if it is
|
||||
* already in transition, otherwise we will add an assert that it should not in
|
||||
* transition.
|
||||
*/
|
||||
private static TransitRegionStateProcedure[] createRoundRobinAssignProcedures(
|
||||
MasterProcedureEnv env, List<RegionInfo> regions, int regionReplication,
|
||||
List<ServerName> serversToExclude, boolean ignoreIfInTransition) {
|
||||
List<RegionInfo> regionsAndReplicas = new ArrayList<>(regions);
|
||||
if (regionReplication != DEFAULT_REGION_REPLICA) {
|
||||
|
||||
// collect the replica region infos
|
||||
List<RegionInfo> replicaRegionInfos =
|
||||
new ArrayList<RegionInfo>(regions.size() * (regionReplication - 1));
|
||||
for (RegionInfo hri : regions) {
|
||||
// start the index from 1
|
||||
for (int i = 1; i < regionReplication; i++) {
|
||||
replicaRegionInfos.add(RegionReplicaUtil.getRegionInfoForReplica(hri, i));
|
||||
}
|
||||
}
|
||||
regionsAndReplicas.addAll(replicaRegionInfos);
|
||||
}
|
||||
if (ignoreIfInTransition) {
|
||||
for (RegionInfo region : regionsAndReplicas) {
|
||||
if (env.getAssignmentManager().getRegionStates().getOrCreateRegionStateNode(region)
|
||||
.isInTransition()) {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
||||
// create round robin procs. Note that we exclude the primary region's target server
|
||||
return env.getAssignmentManager()
|
||||
.createRoundRobinAssignProcedures(regionsAndReplicas, serversToExclude);
|
||||
}
|
||||
|
||||
static TransitRegionStateProcedure[] createAssignProceduresForSplitDaughters(
|
||||
MasterProcedureEnv env, List<RegionInfo> daughters, int regionReplication,
|
||||
ServerName parentServer) {
|
||||
if(env.getMasterConfiguration().getBoolean(HConstants.HBASE_ENABLE_SEPARATE_CHILD_REGIONS,
|
||||
DEFAULT_HBASE_ENABLE_SEPARATE_CHILD_REGIONS)){
|
||||
// keep one daughter on the parent region server
|
||||
TransitRegionStateProcedure[] daughterOne =
|
||||
createAssignProcedures(env, Collections.singletonList(daughters.get(0)),
|
||||
regionReplication, parentServer, false);
|
||||
// round robin assign the other daughter
|
||||
TransitRegionStateProcedure[] daughterTwo =
|
||||
createRoundRobinAssignProcedures(env, Collections.singletonList(daughters.get(1)),
|
||||
regionReplication, Collections.singletonList(parentServer), false);
|
||||
return ArrayUtils.addAll(daughterOne, daughterTwo);
|
||||
}
|
||||
return createAssignProceduresForOpeningNewRegions(env, daughters, regionReplication,
|
||||
parentServer);
|
||||
}
|
||||
|
||||
static TransitRegionStateProcedure[] createAssignProceduresForOpeningNewRegions(
|
||||
MasterProcedureEnv env, List<RegionInfo> regions, int regionReplication,
|
||||
ServerName targetServer) {
|
||||
|
|
|
@ -869,7 +869,7 @@ public class SplitTableRegionProcedure
|
|||
List<RegionInfo> hris = new ArrayList<RegionInfo>(2);
|
||||
hris.add(daughterOneRI);
|
||||
hris.add(daughterTwoRI);
|
||||
return AssignmentManagerUtil.createAssignProceduresForOpeningNewRegions(env, hris,
|
||||
return AssignmentManagerUtil.createAssignProceduresForSplitDaughters(env, hris,
|
||||
getRegionReplication(env), getParentRegionServerName(env));
|
||||
}
|
||||
|
||||
|
|
|
@ -24,12 +24,16 @@ import java.io.IOException;
|
|||
import java.util.Set;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.Waiter;
|
||||
import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.master.RegionState.State;
|
||||
import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
|
@ -152,4 +156,25 @@ public final class AssignmentTestingUtil {
|
|||
proc, 5L * 60 * 1000);
|
||||
return true;
|
||||
}
|
||||
|
||||
public static void insertData(final HBaseTestingUtility UTIL, final TableName tableName,
|
||||
int rowCount, int startRowNum, String... cfs) throws IOException {
|
||||
Table t = UTIL.getConnection().getTable(tableName);
|
||||
Put p;
|
||||
for (int i = 0; i < rowCount / 2; i++) {
|
||||
p = new Put(Bytes.toBytes("" + (startRowNum + i)));
|
||||
for (String cf : cfs) {
|
||||
p.addColumn(Bytes.toBytes(cf), Bytes.toBytes("q"), Bytes.toBytes(i));
|
||||
}
|
||||
t.put(p);
|
||||
p = new Put(Bytes.toBytes("" + (startRowNum + rowCount - i - 1)));
|
||||
for (String cf : cfs) {
|
||||
p.addColumn(Bytes.toBytes(cf), Bytes.toBytes("q"), Bytes.toBytes(i));
|
||||
}
|
||||
t.put(p);
|
||||
if (i % 5 == 0) {
|
||||
UTIL.getAdmin().flush(tableName);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,22 +17,24 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.master.assignment;
|
||||
|
||||
import static org.apache.hadoop.hbase.master.assignment.AssignmentTestingUtil.insertData;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.StartMiniClusterOption;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
|
||||
import org.apache.hadoop.hbase.master.procedure.MasterProcedureTestingUtility;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -60,7 +62,7 @@ public class TestRegionSplit {
|
|||
|
||||
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
||||
|
||||
private static String ColumnFamilyName = "cf";
|
||||
private static String columnFamilyName = "cf";
|
||||
|
||||
private static final int startRowNum = 11;
|
||||
private static final int rowCount = 60;
|
||||
|
@ -94,7 +96,8 @@ public class TestRegionSplit {
|
|||
UTIL.getHBaseCluster().getMaster().setCatalogJanitorEnabled(false);
|
||||
// Disable compaction.
|
||||
for (int i = 0; i < UTIL.getHBaseCluster().getLiveRegionServerThreads().size(); i++) {
|
||||
UTIL.getHBaseCluster().getRegionServer(i).getCompactSplitThread().switchCompaction(false);
|
||||
UTIL.getHBaseCluster().getRegionServer(i).getCompactSplitThread().switchCompaction(
|
||||
false);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -111,8 +114,8 @@ public class TestRegionSplit {
|
|||
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
||||
|
||||
RegionInfo[] regions =
|
||||
MasterProcedureTestingUtility.createTable(procExec, tableName, null, ColumnFamilyName);
|
||||
insertData(tableName);
|
||||
MasterProcedureTestingUtility.createTable(procExec, tableName, null, columnFamilyName);
|
||||
insertData(UTIL, tableName, rowCount, startRowNum, columnFamilyName);
|
||||
int splitRowNum = startRowNum + rowCount / 2;
|
||||
byte[] splitKey = Bytes.toBytes("" + splitRowNum);
|
||||
|
||||
|
@ -121,7 +124,7 @@ public class TestRegionSplit {
|
|||
|
||||
// Split region of the table
|
||||
long procId = procExec.submitProcedure(
|
||||
new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], splitKey));
|
||||
new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], splitKey));
|
||||
// Wait the completion
|
||||
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
||||
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
|
||||
|
@ -146,22 +149,12 @@ public class TestRegionSplit {
|
|||
UTIL.getAdmin().enableTable(tableName);
|
||||
Thread.sleep(500);
|
||||
|
||||
assertEquals("Table region not correct.", 2,
|
||||
UTIL.getHBaseCluster().getRegions(tableName).size());
|
||||
}
|
||||
|
||||
private void insertData(final TableName tableName) throws IOException {
|
||||
Table t = UTIL.getConnection().getTable(tableName);
|
||||
Put p;
|
||||
for (int i = 0; i < rowCount / 2; i++) {
|
||||
p = new Put(Bytes.toBytes("" + (startRowNum + i)));
|
||||
p.addColumn(Bytes.toBytes(ColumnFamilyName), Bytes.toBytes("q1"), Bytes.toBytes(i));
|
||||
t.put(p);
|
||||
p = new Put(Bytes.toBytes("" + (startRowNum + rowCount - i - 1)));
|
||||
p.addColumn(Bytes.toBytes(ColumnFamilyName), Bytes.toBytes("q1"), Bytes.toBytes(i));
|
||||
t.put(p);
|
||||
}
|
||||
UTIL.getAdmin().flush(tableName);
|
||||
List<HRegion> tableRegions = UTIL.getHBaseCluster().getRegions(tableName);
|
||||
assertEquals("Table region not correct.", 2, tableRegions.size());
|
||||
Map<RegionInfo, ServerName> regionInfoMap = UTIL.getHBaseCluster().getMaster()
|
||||
.getAssignmentManager().getRegionStates().getRegionAssignments();
|
||||
assertEquals(regionInfoMap.get(tableRegions.get(0).getRegionInfo()),
|
||||
regionInfoMap.get(tableRegions.get(1).getRegionInfo()));
|
||||
}
|
||||
|
||||
private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
|
||||
|
|
|
@ -0,0 +1,169 @@
|
|||
/**
|
||||
* 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.master.assignment;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.StartMiniClusterOption;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
|
||||
import org.apache.hadoop.hbase.master.procedure.MasterProcedureTestingUtility;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.JVMClusterUtil;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import static org.apache.hadoop.hbase.master.assignment.AssignmentTestingUtil.insertData;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
@Category({ MasterTests.class, MediumTests.class})
|
||||
public class TestRegionSplitAndSeparateChildren {
|
||||
|
||||
@ClassRule
|
||||
public static final HBaseClassTestRule CLASS_RULE =
|
||||
HBaseClassTestRule.forClass(TestRegionSplitAndSeparateChildren.class);
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(
|
||||
TestRegionSplitAndSeparateChildren.class);
|
||||
|
||||
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
||||
|
||||
private static String columnFamilyName = "cf";
|
||||
|
||||
private static final int startRowNum = 11;
|
||||
private static final int rowCount = 60;
|
||||
|
||||
@Rule
|
||||
public TestName name = new TestName();
|
||||
|
||||
private static void setupConf(Configuration conf) {
|
||||
// enable automatically separate child regions
|
||||
conf.setBoolean(HConstants.HBASE_ENABLE_SEPARATE_CHILD_REGIONS, true);
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
public static void setupCluster() throws Exception {
|
||||
setupConf(UTIL.getConfiguration());
|
||||
StartMiniClusterOption option =
|
||||
StartMiniClusterOption.builder().numMasters(1).numRegionServers(3).numDataNodes(3).build();
|
||||
UTIL.startMiniCluster(option);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void cleanupTest() throws Exception {
|
||||
try {
|
||||
UTIL.shutdownMiniCluster();
|
||||
} catch (Exception e) {
|
||||
LOG.warn("failure shutting down cluster", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
// Turn off the meta scanner so it don't remove parent on us.
|
||||
UTIL.getHBaseCluster().getMaster().setCatalogJanitorEnabled(false);
|
||||
// Disable compaction.
|
||||
for (int i = 0; i < UTIL.getHBaseCluster().getLiveRegionServerThreads().size(); i++) {
|
||||
UTIL.getHBaseCluster().getRegionServer(i).getCompactSplitThread().switchCompaction(
|
||||
false);
|
||||
}
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
for (TableDescriptor htd : UTIL.getAdmin().listTableDescriptors()) {
|
||||
UTIL.deleteTable(htd.getTableName());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSplitTableRegionAndSeparateChildRegions() throws Exception {
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
||||
|
||||
RegionInfo[] regions =
|
||||
MasterProcedureTestingUtility.createTable(procExec, tableName, null, columnFamilyName);
|
||||
insertData(UTIL, tableName, rowCount, startRowNum, columnFamilyName);
|
||||
int splitRowNum = startRowNum + rowCount / 2;
|
||||
byte[] splitKey = Bytes.toBytes("" + splitRowNum);
|
||||
|
||||
assertTrue("not able to find a splittable region", regions != null);
|
||||
assertTrue("not able to find a splittable region", regions.length == 1);
|
||||
|
||||
// Split region of the table
|
||||
long procId = procExec.submitProcedure(
|
||||
new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], splitKey));
|
||||
// Wait the completion
|
||||
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
||||
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
|
||||
|
||||
assertTrue("not able to split table",
|
||||
UTIL.getHBaseCluster().getRegions(tableName).size() == 2);
|
||||
|
||||
//disable table
|
||||
UTIL.getAdmin().disableTable(tableName);
|
||||
Thread.sleep(500);
|
||||
|
||||
//stop master
|
||||
UTIL.getHBaseCluster().stopMaster(0);
|
||||
UTIL.getHBaseCluster().waitOnMaster(0);
|
||||
Thread.sleep(500);
|
||||
|
||||
//restart master
|
||||
JVMClusterUtil.MasterThread t = UTIL.getHBaseCluster().startMaster();
|
||||
Thread.sleep(500);
|
||||
|
||||
UTIL.invalidateConnection();
|
||||
// enable table
|
||||
UTIL.getAdmin().enableTable(tableName);
|
||||
Thread.sleep(500);
|
||||
|
||||
List<HRegion> tableRegions = UTIL.getHBaseCluster().getRegions(tableName);
|
||||
assertEquals("Table region not correct.", 2, tableRegions.size());
|
||||
Map<RegionInfo, ServerName> regionInfoMap = UTIL.getHBaseCluster().getMaster()
|
||||
.getAssignmentManager().getRegionStates().getRegionAssignments();
|
||||
assertNotEquals(regionInfoMap.get(tableRegions.get(0).getRegionInfo()),
|
||||
regionInfoMap.get(tableRegions.get(1).getRegionInfo()));
|
||||
}
|
||||
|
||||
private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
|
||||
return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
|
||||
}
|
||||
}
|
|
@ -17,6 +17,7 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.master.assignment;
|
||||
|
||||
import static org.apache.hadoop.hbase.master.assignment.AssignmentTestingUtil.insertData;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
@ -35,7 +36,6 @@ import org.apache.hadoop.hbase.Waiter;
|
|||
import org.apache.hadoop.hbase.client.CompactionState;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
|
@ -73,8 +73,8 @@ public class TestSplitTableRegionProcedure {
|
|||
|
||||
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
||||
|
||||
private static String ColumnFamilyName1 = "cf1";
|
||||
private static String ColumnFamilyName2 = "cf2";
|
||||
private static String columnFamilyName1 = "cf1";
|
||||
private static String columnFamilyName2 = "cf2";
|
||||
|
||||
private static final int startRowNum = 11;
|
||||
private static final int rowCount = 60;
|
||||
|
@ -143,8 +143,8 @@ public class TestSplitTableRegionProcedure {
|
|||
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
||||
|
||||
RegionInfo [] regions = MasterProcedureTestingUtility.createTable(
|
||||
procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
|
||||
insertData(tableName);
|
||||
procExec, tableName, null, columnFamilyName1, columnFamilyName2);
|
||||
insertData(UTIL, tableName, rowCount, startRowNum, columnFamilyName1, columnFamilyName2);
|
||||
int splitRowNum = startRowNum + rowCount / 2;
|
||||
byte[] splitKey = Bytes.toBytes("" + splitRowNum);
|
||||
|
||||
|
@ -177,7 +177,7 @@ public class TestSplitTableRegionProcedure {
|
|||
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
||||
|
||||
RegionInfo [] regions = MasterProcedureTestingUtility.createTable(
|
||||
procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
|
||||
procExec, tableName, null, columnFamilyName1, columnFamilyName2);
|
||||
int splitRowNum = startRowNum + rowCount / 2;
|
||||
byte[] splitKey = Bytes.toBytes("" + splitRowNum);
|
||||
|
||||
|
@ -207,8 +207,8 @@ public class TestSplitTableRegionProcedure {
|
|||
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
||||
|
||||
RegionInfo [] regions = MasterProcedureTestingUtility.createTable(
|
||||
procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
|
||||
insertData(tableName);
|
||||
procExec, tableName, null, columnFamilyName1, columnFamilyName2);
|
||||
insertData(UTIL, tableName, rowCount, startRowNum, columnFamilyName1, columnFamilyName2);
|
||||
// Split to two daughters with one of them only has 1 row
|
||||
int splitRowNum = startRowNum + rowCount / 4;
|
||||
byte[] splitKey = Bytes.toBytes("" + splitRowNum);
|
||||
|
@ -238,8 +238,8 @@ public class TestSplitTableRegionProcedure {
|
|||
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
||||
|
||||
RegionInfo [] regions = MasterProcedureTestingUtility.createTable(
|
||||
procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
|
||||
insertData(tableName);
|
||||
procExec, tableName, null, columnFamilyName1, columnFamilyName2);
|
||||
insertData(UTIL, tableName, rowCount, startRowNum, columnFamilyName1, columnFamilyName2);
|
||||
// Split to two daughters with one of them only has 1 row
|
||||
int splitRowNum = startRowNum + rowCount;
|
||||
byte[] splitKey = Bytes.toBytes("" + splitRowNum);
|
||||
|
@ -274,8 +274,8 @@ public class TestSplitTableRegionProcedure {
|
|||
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
||||
|
||||
RegionInfo [] regions = MasterProcedureTestingUtility.createTable(
|
||||
procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
|
||||
insertData(tableName);
|
||||
procExec, tableName, null, columnFamilyName1, columnFamilyName2);
|
||||
insertData(UTIL, tableName, rowCount, startRowNum, columnFamilyName1, columnFamilyName2);
|
||||
// Split to two daughters with one of them only has 1 row
|
||||
int splitRowNum = rowCount;
|
||||
deleteData(tableName, splitRowNum);
|
||||
|
@ -320,8 +320,8 @@ public class TestSplitTableRegionProcedure {
|
|||
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
||||
|
||||
RegionInfo [] regions = MasterProcedureTestingUtility.createTable(
|
||||
procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
|
||||
insertData(tableName);
|
||||
procExec, tableName, null, columnFamilyName1, columnFamilyName2);
|
||||
insertData(UTIL, tableName, rowCount, startRowNum, columnFamilyName1, columnFamilyName2);
|
||||
|
||||
assertTrue("not able to find a splittable region", regions != null);
|
||||
assertTrue("not able to find a splittable region", regions.length == 1);
|
||||
|
@ -349,8 +349,8 @@ public class TestSplitTableRegionProcedure {
|
|||
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
||||
|
||||
RegionInfo[] regions = MasterProcedureTestingUtility.createTable(
|
||||
procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
|
||||
insertData(tableName);
|
||||
procExec, tableName, null, columnFamilyName1, columnFamilyName2);
|
||||
insertData(UTIL, tableName, rowCount, startRowNum, columnFamilyName1, columnFamilyName2);
|
||||
int splitRowNum = startRowNum + rowCount / 2;
|
||||
byte[] splitKey = Bytes.toBytes("" + splitRowNum);
|
||||
|
||||
|
@ -379,7 +379,7 @@ public class TestSplitTableRegionProcedure {
|
|||
List<HRegion> newRegions = UTIL.getMiniHBaseCluster().getRegions(tableName);
|
||||
assertEquals(1, newRegions.size());
|
||||
verifyData(newRegions.get(0), startRowNum, rowCount,
|
||||
Bytes.toBytes(ColumnFamilyName1), Bytes.toBytes(ColumnFamilyName2));
|
||||
Bytes.toBytes(columnFamilyName1), Bytes.toBytes(columnFamilyName2));
|
||||
|
||||
assertEquals(splitSubmittedCount + 1, splitProcMetrics.getSubmittedCounter().getCount());
|
||||
assertEquals(splitFailedCount + 1, splitProcMetrics.getFailedCounter().getCount());
|
||||
|
@ -391,8 +391,8 @@ public class TestSplitTableRegionProcedure {
|
|||
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
||||
|
||||
RegionInfo [] regions = MasterProcedureTestingUtility.createTable(
|
||||
procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
|
||||
insertData(tableName);
|
||||
procExec, tableName, null, columnFamilyName1, columnFamilyName2);
|
||||
insertData(UTIL, tableName, rowCount, startRowNum, columnFamilyName1, columnFamilyName2);
|
||||
int splitRowNum = startRowNum + rowCount / 2;
|
||||
byte[] splitKey = Bytes.toBytes("" + splitRowNum);
|
||||
|
||||
|
@ -425,8 +425,8 @@ public class TestSplitTableRegionProcedure {
|
|||
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
||||
|
||||
RegionInfo [] regions = MasterProcedureTestingUtility.createTable(
|
||||
procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
|
||||
insertData(tableName);
|
||||
procExec, tableName, null, columnFamilyName1, columnFamilyName2);
|
||||
insertData(UTIL, tableName, rowCount, startRowNum, columnFamilyName1, columnFamilyName2);
|
||||
int splitRowNum = startRowNum + rowCount / 2;
|
||||
byte[] splitKey = Bytes.toBytes("" + splitRowNum);
|
||||
|
||||
|
@ -452,24 +452,6 @@ public class TestSplitTableRegionProcedure {
|
|||
verify(tableName, splitRowNum);
|
||||
}
|
||||
|
||||
private void insertData(final TableName tableName) throws IOException, InterruptedException {
|
||||
Table t = UTIL.getConnection().getTable(tableName);
|
||||
Put p;
|
||||
for (int i= 0; i < rowCount / 2; i++) {
|
||||
p = new Put(Bytes.toBytes("" + (startRowNum + i)));
|
||||
p.addColumn(Bytes.toBytes(ColumnFamilyName1), Bytes.toBytes("q1"), Bytes.toBytes(i));
|
||||
p.addColumn(Bytes.toBytes(ColumnFamilyName2), Bytes.toBytes("q2"), Bytes.toBytes(i));
|
||||
t.put(p);
|
||||
p = new Put(Bytes.toBytes("" + (startRowNum + rowCount - i - 1)));
|
||||
p.addColumn(Bytes.toBytes(ColumnFamilyName1), Bytes.toBytes("q1"), Bytes.toBytes(i));
|
||||
p.addColumn(Bytes.toBytes(ColumnFamilyName2), Bytes.toBytes("q2"), Bytes.toBytes(i));
|
||||
t.put(p);
|
||||
if (i % 5 == 0) {
|
||||
UTIL.getAdmin().flush(tableName);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void deleteData(
|
||||
final TableName tableName,
|
||||
final int startDeleteRowNum) throws IOException, InterruptedException {
|
||||
|
@ -505,8 +487,8 @@ public class TestSplitTableRegionProcedure {
|
|||
daughters.get(i),
|
||||
startRow,
|
||||
numRows,
|
||||
Bytes.toBytes(ColumnFamilyName1),
|
||||
Bytes.toBytes(ColumnFamilyName2));
|
||||
Bytes.toBytes(columnFamilyName1),
|
||||
Bytes.toBytes(columnFamilyName2));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue