HBASE-25518 Support separate child regions to different region servers (#3001)

Signed-off-by: stack <stack@apache.org>
This commit is contained in:
haxiaolin 2021-03-19 03:38:17 +08:00 committed by GitHub
parent d93035a131
commit 585aca1f05
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 313 additions and 66 deletions

View File

@ -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");

View File

@ -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) {

View File

@ -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));
}

View File

@ -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);
}
}
}
}

View File

@ -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() {

View File

@ -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();
}
}

View File

@ -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));
}
}