diff --git a/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto index 59a1d68fda4..8d8b9af009c 100644 --- a/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto +++ b/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto @@ -72,6 +72,8 @@ enum ModifyTableState { MODIFY_TABLE_DELETE_FS_LAYOUT = 5; MODIFY_TABLE_POST_OPERATION = 6; MODIFY_TABLE_REOPEN_ALL_REGIONS = 7; + MODIFY_TABLE_CLOSE_EXCESS_REPLICAS = 8; + MODIFY_TABLE_ASSIGN_NEW_REPLICAS = 9; } message ModifyTableStateData { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java index d2e1eb3e9d4..fb64514a337 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java @@ -870,32 +870,47 @@ public class AssignmentManager { .sorted(AssignmentManager::compare).toArray(TransitRegionStateProcedure[]::new); } + // for creating unassign TRSP when disabling a table or closing excess region replicas + private TransitRegionStateProcedure forceCreateUnssignProcedure(RegionStateNode regionNode) { + regionNode.lock(); + try { + if (!regionStates.include(regionNode, false) || + regionStates.isRegionOffline(regionNode.getRegionInfo())) { + return null; + } + // As in DisableTableProcedure or ModifyTableProcedure, we will hold the xlock for table, so + // we can make sure that this procedure has not been executed yet, as TRSP will hold the + // shared lock for table all the time. So here we will unset it and when it is actually + // executed, it will find that the attach procedure is not itself and quit immediately. + if (regionNode.getProcedure() != null) { + regionNode.unsetProcedure(regionNode.getProcedure()); + } + return regionNode.setProcedure(TransitRegionStateProcedure.unassign(getProcedureEnvironment(), + regionNode.getRegionInfo())); + } finally { + regionNode.unlock(); + } + } + /** * Called by DisableTableProcedure to unassign all the regions for a table. */ public TransitRegionStateProcedure[] createUnassignProceduresForDisabling(TableName tableName) { - return regionStates.getTableRegionStateNodes(tableName).stream().map(regionNode -> { - regionNode.lock(); - try { - if (!regionStates.include(regionNode, false) || - regionStates.isRegionOffline(regionNode.getRegionInfo())) { - return null; - } - // As in DisableTableProcedure, we will hold the xlock for table, so we can make sure that - // this procedure has not been executed yet, as TRSP will hold the shared lock for table all - // the time. So here we will unset it and when it is actually executed, it will find that - // the attach procedure is not itself and quit immediately. - if (regionNode.getProcedure() != null) { - regionNode.unsetProcedure(regionNode.getProcedure()); - } - TransitRegionStateProcedure proc = TransitRegionStateProcedure - .unassign(getProcedureEnvironment(), regionNode.getRegionInfo()); - regionNode.setProcedure(proc); - return proc; - } finally { - regionNode.unlock(); - } - }).filter(p -> p != null).toArray(TransitRegionStateProcedure[]::new); + return regionStates.getTableRegionStateNodes(tableName).stream() + .map(this::forceCreateUnssignProcedure).filter(p -> p != null) + .toArray(TransitRegionStateProcedure[]::new); + } + + /** + * Called by ModifyTableProcedures to unassign all the excess region replicas + * for a table. + */ + public TransitRegionStateProcedure[] createUnassignProceduresForClosingExcessRegionReplicas( + TableName tableName, int newReplicaCount) { + return regionStates.getTableRegionStateNodes(tableName).stream() + .filter(regionNode -> regionNode.getRegionInfo().getReplicaId() >= newReplicaCount) + .map(this::forceCreateUnssignProcedure).filter(p -> p != null) + .toArray(TransitRegionStateProcedure[]::new); } public SplitTableRegionProcedure createSplitProcedure(final RegionInfo regionToSplit, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java index 500e5ec79da..78f2bb75fe8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java @@ -547,25 +547,12 @@ public class RegionStateStore { LOG.debug("Overwritten regions: {} ", regionInfos); } - /** - * Update region replicas if necessary by adding new replica locations or removing unused region - * replicas - */ - public void updateRegionReplicas(TableName tableName, int oldReplicaCount, int newReplicaCount) - throws IOException { - if (newReplicaCount < oldReplicaCount) { - removeRegionReplicas(tableName, oldReplicaCount, newReplicaCount); - } else if (newReplicaCount > oldReplicaCount) { - addRegionReplicas(tableName, oldReplicaCount, newReplicaCount); - } - } - private Scan getScanForUpdateRegionReplicas(TableName tableName) { return MetaTableAccessor.getScanForTableName(master.getConfiguration(), tableName) .addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); } - private void removeRegionReplicas(TableName tableName, int oldReplicaCount, int newReplicaCount) + public void removeRegionReplicas(TableName tableName, int oldReplicaCount, int newReplicaCount) throws IOException { Scan scan = getScanForUpdateRegionReplicas(tableName); List deletes = new ArrayList<>(); @@ -598,32 +585,6 @@ public class RegionStateStore { } } - private void addRegionReplicas(TableName tableName, int oldReplicaCount, int newReplicaCount) - throws IOException { - Scan scan = getScanForUpdateRegionReplicas(tableName); - List puts = new ArrayList<>(); - long now = EnvironmentEdgeManager.currentTime(); - try (Table metaTable = getMetaTable(); ResultScanner scanner = metaTable.getScanner(scan)) { - for (;;) { - Result result = scanner.next(); - if (result == null) { - break; - } - RegionInfo primaryRegionInfo = CatalogFamilyFormat.getRegionInfo(result); - if (primaryRegionInfo == null || primaryRegionInfo.isSplitParent()) { - continue; - } - Put put = new Put(result.getRow(), now); - for (int i = oldReplicaCount; i < newReplicaCount; i++) { - MetaTableAccessor.addEmptyLocation(put, i); - } - puts.add(put); - } - debugLogMutations(puts); - metaTable.put(puts); - } - } - // ========================================================================== // Table Descriptors helpers // ========================================================================== diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/EnableTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/EnableTableProcedure.java index 6ca83fe01ef..1e48981e417 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/EnableTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/EnableTableProcedure.java @@ -20,17 +20,11 @@ package org.apache.hadoop.hbase.master.procedure; import java.io.IOException; import java.util.ArrayList; import java.util.List; -import org.apache.hadoop.hbase.CatalogFamilyFormat; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotFoundException; -import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionReplicaUtil; -import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; @@ -100,7 +94,6 @@ public class EnableTableProcedure case ENABLE_TABLE_MARK_REGIONS_ONLINE: // Get the region replica count. If changed since disable, need to do // more work assigning. - Connection connection = env.getMasterServices().getConnection(); TableDescriptor tableDescriptor = env.getMasterServices().getTableDescriptors().get(tableName); int configuredReplicaCount = tableDescriptor.getRegionReplication(); @@ -111,25 +104,16 @@ public class EnableTableProcedure // How many replicas do we currently have? Check regions returned from // in-memory state. int currentMaxReplica = getMaxReplicaId(regionsOfTable); - - // Read the META table to know the number of replicas the table currently has. - // If there was a table modification on region replica count then need to - // adjust replica counts here. - int replicasFound = TableName.isMetaTableName(this.tableName)? - 0: // TODO: Figure better what to do here for hbase:meta replica. - getReplicaCountInMeta(connection, configuredReplicaCount, regionsOfTable); - LOG.info("replicasFound={} (configuredReplicaCount={} for {}", replicasFound, - configuredReplicaCount, tableName.getNameAsString()); - if (currentMaxReplica == (configuredReplicaCount - 1)) { - if (LOG.isDebugEnabled()) { - LOG.debug("No change in number of region replicas (configuredReplicaCount={});" - + " assigning.", configuredReplicaCount); - } + if (currentMaxReplica == configuredReplicaCount - 1) { + LOG.debug("No change in number of region replicas (configuredReplicaCount={});" + + " assigning.", configuredReplicaCount); } else if (currentMaxReplica > (configuredReplicaCount - 1)) { // We have additional regions as the replica count has been decreased. Delete // those regions because already the table is in the unassigned state - LOG.info("The number of replicas " + (currentMaxReplica + 1) - + " is more than the region replica count " + configuredReplicaCount); + LOG.warn( + "The number of replicas {} is more than the region replica count {}" + + ", usually this should not happen as we will delete them in ModifyTableProcedure", + currentMaxReplica + 1, configuredReplicaCount); List copyOfRegions = new ArrayList(regionsOfTable); for (RegionInfo regionInfo : copyOfRegions) { if (regionInfo.getReplicaId() > (configuredReplicaCount - 1)) { @@ -140,11 +124,11 @@ public class EnableTableProcedure regionsOfTable.remove(regionInfo); } } - } else { + } else if (currentMaxReplica < configuredReplicaCount - 1) { // the replicasFound is less than the regionReplication LOG.info("Number of replicas has increased for {}. Assigning new region replicas." + "The previous replica count was {}. The current replica count is {}.", - this.tableName, (currentMaxReplica + 1), configuredReplicaCount); + this.tableName, currentMaxReplica + 1, configuredReplicaCount); regionsOfTable = RegionReplicaUtil.addReplicas(regionsOfTable, currentMaxReplica + 1, configuredReplicaCount); } @@ -174,25 +158,6 @@ public class EnableTableProcedure return Flow.HAS_MORE_STATE; } - /** - * @return Count of replicas found reading hbase:meta Region row or zk if - * asking about the hbase:meta table itself.. - */ - private int getReplicaCountInMeta(Connection connection, int regionReplicaCount, - List regionsOfTable) throws IOException { - Result r = MetaTableAccessor.getCatalogFamilyRow(connection, regionsOfTable.get(0)); - int replicasFound = 0; - for (int i = 1; i < regionReplicaCount; i++) { - // Since we have already added the entries to the META we will be getting only that here - List columnCells = - r.getColumnCells(HConstants.CATALOG_FAMILY, CatalogFamilyFormat.getServerColumn(i)); - if (!columnCells.isEmpty()) { - replicasFound++; - } - } - return replicasFound; - } - @Override protected void rollbackState(final MasterProcedureEnv env, final EnableTableState state) throws IOException { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java index 9b29d30b9b3..beb129b6f52 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java @@ -24,16 +24,17 @@ import java.util.Collections; import java.util.List; import java.util.Set; import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.IntStream; import org.apache.hadoop.hbase.ConcurrentTableModificationException; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.TableDescriptor; -import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; import org.apache.hadoop.hbase.replication.ReplicationException; @@ -128,6 +129,12 @@ public class ModifyTableProcedure break; case MODIFY_TABLE_PRE_OPERATION: preModify(env, state); + setNextState(ModifyTableState.MODIFY_TABLE_CLOSE_EXCESS_REPLICAS); + break; + case MODIFY_TABLE_CLOSE_EXCESS_REPLICAS: + if (isTableEnabled(env)) { + closeExcessReplicasIfNeeded(env); + } setNextState(ModifyTableState.MODIFY_TABLE_UPDATE_TABLE_DESCRIPTOR); break; case MODIFY_TABLE_UPDATE_TABLE_DESCRIPTOR: @@ -135,7 +142,7 @@ public class ModifyTableProcedure setNextState(ModifyTableState.MODIFY_TABLE_REMOVE_REPLICA_COLUMN); break; case MODIFY_TABLE_REMOVE_REPLICA_COLUMN: - updateReplicaColumnsIfNeeded(env, unmodifiedTableDescriptor, modifiedTableDescriptor); + removeReplicaColumnsIfNeeded(env); setNextState(ModifyTableState.MODIFY_TABLE_POST_OPERATION); break; case MODIFY_TABLE_POST_OPERATION: @@ -146,6 +153,10 @@ public class ModifyTableProcedure if (isTableEnabled(env)) { addChildProcedure(new ReopenTableRegionsProcedure(getTableName())); } + setNextState(ModifyTableState.MODIFY_TABLE_ASSIGN_NEW_REPLICAS); + break; + case MODIFY_TABLE_ASSIGN_NEW_REPLICAS: + assignNewReplicasIfNeeded(env); if (deleteColumnFamilyInModify) { setNextState(ModifyTableState.MODIFY_TABLE_DELETE_FS_LAYOUT); } else { @@ -297,14 +308,6 @@ public class ModifyTableProcedure env.getMasterServices().getTableDescriptors().get(getTableName()); } - if (env.getMasterServices().getTableStateManager() - .isTableState(getTableName(), TableState.State.ENABLED)) { - if (modifiedTableDescriptor.getRegionReplication() != unmodifiedTableDescriptor - .getRegionReplication()) { - throw new TableNotDisabledException( - "REGION_REPLICATION change is not supported for enabled tables"); - } - } this.deleteColumnFamilyInModify = isDeleteColumnFamily(unmodifiedTableDescriptor, modifiedTableDescriptor); if (!unmodifiedTableDescriptor.getRegionServerGroup() @@ -375,17 +378,36 @@ public class ModifyTableProcedure } /** - * update replica column families if necessary. + * remove replica columns if necessary. */ - private void updateReplicaColumnsIfNeeded(MasterProcedureEnv env, - TableDescriptor oldTableDescriptor, TableDescriptor newTableDescriptor) throws IOException { - final int oldReplicaCount = oldTableDescriptor.getRegionReplication(); - final int newReplicaCount = newTableDescriptor.getRegionReplication(); - env.getAssignmentManager().getRegionStateStore().updateRegionReplicas(getTableName(), + private void removeReplicaColumnsIfNeeded(MasterProcedureEnv env) throws IOException { + final int oldReplicaCount = unmodifiedTableDescriptor.getRegionReplication(); + final int newReplicaCount = modifiedTableDescriptor.getRegionReplication(); + if (newReplicaCount >= oldReplicaCount) { + return; + } + env.getAssignmentManager().getRegionStateStore().removeRegionReplicas(getTableName(), oldReplicaCount, newReplicaCount); - if (newReplicaCount > oldReplicaCount && oldReplicaCount <= 1) { - // The table has been newly enabled for replica. So check if we need to setup - // region replication + env.getAssignmentManager().getRegionStates().getRegionsOfTable(getTableName()).stream() + .filter(r -> r.getReplicaId() >= newReplicaCount) + .forEach(env.getAssignmentManager().getRegionStates()::deleteRegion); + } + + private void assignNewReplicasIfNeeded(MasterProcedureEnv env) throws IOException { + final int oldReplicaCount = unmodifiedTableDescriptor.getRegionReplication(); + final int newReplicaCount = modifiedTableDescriptor.getRegionReplication(); + if (newReplicaCount <= oldReplicaCount) { + return; + } + if (isTableEnabled(env)) { + List newReplicas = env.getAssignmentManager().getRegionStates() + .getRegionsOfTable(getTableName()).stream().filter(RegionReplicaUtil::isDefaultReplica) + .flatMap(primaryRegion -> IntStream.range(oldReplicaCount, newReplicaCount).mapToObj( + replicaId -> RegionReplicaUtil.getRegionInfoForReplica(primaryRegion, replicaId))) + .collect(Collectors.toList()); + addChildProcedure(env.getAssignmentManager().createAssignProcedures(newReplicas)); + } + if (oldReplicaCount <= 1) { try { ServerRegionReplicaUtil.setupRegionReplicaReplication(env.getMasterServices()); } catch (ReplicationException e) { @@ -394,6 +416,16 @@ public class ModifyTableProcedure } } + private void closeExcessReplicasIfNeeded(MasterProcedureEnv env) { + final int oldReplicaCount = unmodifiedTableDescriptor.getRegionReplication(); + final int newReplicaCount = modifiedTableDescriptor.getRegionReplication(); + if (newReplicaCount >= oldReplicaCount) { + return; + } + addChildProcedure(env.getAssignmentManager() + .createUnassignProceduresForClosingExcessRegionReplicas(getTableName(), newReplicaCount)); + } + /** * Action after modifying table. * @param env MasterProcedureEnv diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java index 528b155cb94..1fb2f00d67c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java @@ -1846,11 +1846,9 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility { */ public static void setReplicas(Admin admin, TableName table, int replicaCount) throws IOException, InterruptedException { - admin.disableTable(table); TableDescriptor desc = TableDescriptorBuilder.newBuilder(admin.getDescriptor(table)) .setRegionReplication(replicaCount).build(); admin.modifyTable(desc); - admin.enableTable(table); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin3.java index c648db94aa3..441d40194a9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin3.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin3.java @@ -263,26 +263,6 @@ public class TestAdmin3 extends TestAdminBase { assertFalse(ADMIN.tableExists(tableName)); } - @Test(expected = TableNotDisabledException.class) - public void testModifyRegionReplicasEnabledTable() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); - TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close(); - - // Modify region replication count - TableDescriptor htd = TableDescriptorBuilder.newBuilder(ADMIN.getDescriptor(tableName)) - .setRegionReplication(3).build(); - try { - // try to modify the region replication count without disabling the table - ADMIN.modifyTable(htd); - fail("Expected an exception"); - } finally { - // Delete the table - ADMIN.disableTable(tableName); - ADMIN.deleteTable(tableName); - assertFalse(ADMIN.tableExists(tableName)); - } - } - @Test public void testDeleteLastColumnFamily() throws Exception { final TableName tableName = TableName.valueOf(name.getMethodName()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java index a53771d46c7..ad1340104a9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java @@ -408,7 +408,7 @@ public class TestRegionStateStore { MetaTableAccessor.addRegionsToMeta(UTIL.getConnection(), regionInfos, 3); final RegionStateStore regionStateStore = UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStateStore(); - regionStateStore.updateRegionReplicas(tableName, 3, 1); + regionStateStore.removeRegionReplicas(tableName, 3, 1); Get get = new Get(primary.getRegionName()); Result result = meta.get(get); for (int replicaId = 0; replicaId < 3; replicaId++) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java index 4461f2f0ede..f5cc543e86d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java @@ -371,7 +371,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase { long procId = procExec.submitProcedure(new ModifyTableProcedure(procExec.getEnvironment(), newTd)); - int lastStep = 3; // failing before MODIFY_TABLE_UPDATE_TABLE_DESCRIPTOR + int lastStep = 8; // failing before MODIFY_TABLE_UPDATE_TABLE_DESCRIPTOR MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, lastStep); // cf2 should not be present @@ -404,7 +404,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase { new ModifyTableProcedure(procExec.getEnvironment(), newTd)); // Restart the executor and rollback the step twice - int lastStep = 3; // failing before MODIFY_TABLE_UPDATE_TABLE_DESCRIPTOR + int lastStep = 8; // failing before MODIFY_TABLE_UPDATE_TABLE_DESCRIPTOR MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, lastStep); // cf2 should not be present diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicasWithModifyTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicasWithModifyTable.java index f01038eee70..4ca8059bc28 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicasWithModifyTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicasWithModifyTable.java @@ -20,51 +20,64 @@ package org.apache.hadoop.hbase.regionserver; import static org.junit.Assert.assertEquals; import java.io.IOException; +import java.util.Arrays; import java.util.List; 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.TableName; -import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.RegionSplitter; +import org.junit.After; import org.junit.AfterClass; 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.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameter; +import org.junit.runners.Parameterized.Parameters; +@RunWith(Parameterized.class) @Category({ RegionServerTests.class, MediumTests.class }) public class TestRegionReplicasWithModifyTable { @ClassRule public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestRegionReplicasWithModifyTable.class); + HBaseClassTestRule.forClass(TestRegionReplicasWithModifyTable.class); private static final int NB_SERVERS = 3; - private static Table table; private static final HBaseTestingUtility HTU = new HBaseTestingUtility(); private static final byte[] f = HConstants.CATALOG_FAMILY; + @Parameter + public boolean disableBeforeModifying; + @Rule - public TestName name = new TestName(); + public TableNameTestRule name = new TableNameTestRule(); + + @Parameters + public static List params() { + return Arrays.asList(new Object[] { true }, new Object[] { false }); + } @BeforeClass public static void before() throws Exception { HTU.startMiniCluster(NB_SERVERS); } - private static void enableReplicationByModification(final TableName tableName, - boolean withReplica, int initialReplicaCount, int enableReplicaCount, int splitCount) - throws IOException, InterruptedException { + private void enableReplicationByModification(boolean withReplica, int initialReplicaCount, + int enableReplicaCount, int splitCount) throws IOException, InterruptedException { + TableName tableName = name.getTableName(); TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName); if (withReplica) { builder.setRegionReplication(initialReplicaCount); @@ -72,14 +85,25 @@ public class TestRegionReplicasWithModifyTable { TableDescriptor htd = builder.build(); if (splitCount > 0) { byte[][] splits = getSplits(splitCount); - table = HTU.createTable(htd, new byte[][] { f }, splits, - new Configuration(HTU.getConfiguration())); - + HTU.createTable(htd, new byte[][] { f }, splits, new Configuration(HTU.getConfiguration())); } else { - table = HTU.createTable(htd, new byte[][] { f }, (byte[][]) null, + HTU.createTable(htd, new byte[][] { f }, (byte[][]) null, new Configuration(HTU.getConfiguration())); } - HBaseTestingUtility.setReplicas(HTU.getAdmin(), table.getName(), enableReplicaCount); + if (disableBeforeModifying) { + HTU.getAdmin().disableTable(tableName); + } + HBaseTestingUtility.setReplicas(HTU.getAdmin(), tableName, enableReplicaCount); + if (disableBeforeModifying) { + HTU.getAdmin().enableTable(tableName); + } + int expectedRegionCount; + if (splitCount > 0) { + expectedRegionCount = enableReplicaCount * splitCount; + } else { + expectedRegionCount = enableReplicaCount; + } + assertTotalRegions(expectedRegionCount); } private static byte[][] getSplits(int numRegions) { @@ -91,123 +115,50 @@ public class TestRegionReplicasWithModifyTable { @AfterClass public static void afterClass() throws Exception { - HRegionServer.TEST_SKIP_REPORTING_TRANSITION = false; - table.close(); HTU.shutdownMiniCluster(); } - private HRegionServer getRS() { - return HTU.getMiniHBaseCluster().getRegionServer(0); - } - - private HRegionServer getSecondaryRS() { - return HTU.getMiniHBaseCluster().getRegionServer(1); - } - - private HRegionServer getTertiaryRS() { - return HTU.getMiniHBaseCluster().getRegionServer(2); - } - - @Test - public void testRegionReplicasUsingEnableTable() throws Exception { - TableName tableName = null; - try { - tableName = TableName.valueOf(name.getMethodName()); - enableReplicationByModification(tableName, false, 0, 3, 0); - List onlineRegions = getRS().getRegions(tableName); - List onlineRegions2 = getSecondaryRS().getRegions(tableName); - List onlineRegions3 = getTertiaryRS().getRegions(tableName); - int totalRegions = onlineRegions.size() + onlineRegions2.size() + onlineRegions3.size(); - assertEquals("the number of regions should be more than 1", 3, totalRegions); - } finally { - disableAndDeleteTable(tableName); - } - } - - private void disableAndDeleteTable(TableName tableName) throws IOException { + @After + public void tearDown() throws IOException { + TableName tableName = name.getTableName(); HTU.getAdmin().disableTable(tableName); HTU.getAdmin().deleteTable(tableName); } + private void assertTotalRegions(int expected) { + int actual = HTU.getHBaseCluster().getRegions(name.getTableName()).size(); + assertEquals(expected, actual); + } + + @Test + public void testRegionReplicasUsingEnableTable() throws Exception { + enableReplicationByModification(false, 0, 3, 0); + } + @Test public void testRegionReplicasUsingEnableTableForMultipleRegions() throws Exception { - TableName tableName = null; - try { - tableName = TableName.valueOf(name.getMethodName()); - enableReplicationByModification(tableName, false, 0, 3, 10); - List onlineRegions = getRS().getRegions(tableName); - List onlineRegions2 = getSecondaryRS().getRegions(tableName); - List onlineRegions3 = getTertiaryRS().getRegions(tableName); - int totalRegions = onlineRegions.size() + onlineRegions2.size() + onlineRegions3.size(); - assertEquals("the number of regions should be equal to 30", 30, totalRegions); - } finally { - disableAndDeleteTable(tableName); - } + enableReplicationByModification(false, 0, 3, 10); } @Test public void testRegionReplicasByEnableTableWhenReplicaCountIsIncreased() throws Exception { - TableName tableName = null; - try { - tableName = TableName.valueOf(name.getMethodName()); - enableReplicationByModification(tableName, true, 2, 3, 0); - List onlineRegions = getRS().getRegions(tableName); - List onlineRegions2 = getSecondaryRS().getRegions(tableName); - List onlineRegions3 = getTertiaryRS().getRegions(tableName); - int totalRegions = onlineRegions.size() + onlineRegions2.size() + onlineRegions3.size(); - assertEquals("the number of regions should be 3", 3, totalRegions); - } finally { - disableAndDeleteTable(tableName); - } + enableReplicationByModification(true, 2, 3, 0); } @Test public void testRegionReplicasByEnableTableWhenReplicaCountIsDecreased() throws Exception { - TableName tableName = null; - try { - tableName = TableName.valueOf(name.getMethodName()); - enableReplicationByModification(tableName, true, 3, 2, 0); - List onlineRegions = getRS().getRegions(tableName); - List onlineRegions2 = getSecondaryRS().getRegions(tableName); - List onlineRegions3 = getTertiaryRS().getRegions(tableName); - int totalRegions = onlineRegions.size() + onlineRegions2.size() + onlineRegions3.size(); - assertEquals("the number of regions should be reduced to 2", 2, totalRegions); - } finally { - disableAndDeleteTable(tableName); - } + enableReplicationByModification(true, 3, 2, 0); } @Test public void testRegionReplicasByEnableTableWhenReplicaCountIsDecreasedWithMultipleRegions() - throws Exception { - TableName tableName = null; - try { - tableName = TableName.valueOf(name.getMethodName()); - enableReplicationByModification(tableName, true, 3, 2, 20); - List onlineRegions = getRS().getRegions(tableName); - List onlineRegions2 = getSecondaryRS().getRegions(tableName); - List onlineRegions3 = getTertiaryRS().getRegions(tableName); - int totalRegions = onlineRegions.size() + onlineRegions2.size() + onlineRegions3.size(); - assertEquals("the number of regions should be reduced to 40", 40, totalRegions); - } finally { - disableAndDeleteTable(tableName); - } + throws Exception { + enableReplicationByModification(true, 3, 2, 20); } @Test public void testRegionReplicasByEnableTableWhenReplicaCountIsIncreasedWithmultipleRegions() - throws Exception { - TableName tableName = null; - try { - tableName = TableName.valueOf(name.getMethodName()); - enableReplicationByModification(tableName, true, 2, 3, 15); - List onlineRegions = getRS().getRegions(tableName); - List onlineRegions2 = getSecondaryRS().getRegions(tableName); - List onlineRegions3 = getTertiaryRS().getRegions(tableName); - int totalRegions = onlineRegions.size() + onlineRegions2.size() + onlineRegions3.size(); - assertEquals("the number of regions should be equal to 45", 3 * 15, totalRegions); - } finally { - disableAndDeleteTable(tableName); - } + throws Exception { + enableReplicationByModification(true, 2, 3, 15); } }