Skip to content

Commit

Permalink
HBASE-25000 Move delete region info related methods to RegionStateSto…
Browse files Browse the repository at this point in the history
…re (#2366)

Signed-off-by: Guanghao Zhang <[email protected]>
  • Loading branch information
Apache9 authored Sep 9, 2020
1 parent 2e96a5b commit 0511089
Show file tree
Hide file tree
Showing 7 changed files with 70 additions and 83 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -841,7 +841,7 @@ public static void addRegionsToMeta(Connection connection, List<RegionInfo> regi
* @param ts desired timestamp
* @throws IOException if problem connecting or updating meta
*/
private static void addRegionsToMeta(Connection connection, List<RegionInfo> regionInfos,
public static void addRegionsToMeta(Connection connection, List<RegionInfo> regionInfos,
int regionReplication, long ts) throws IOException {
List<Put> puts = new ArrayList<>();
for (RegionInfo regionInfo : regionInfos) {
Expand Down Expand Up @@ -937,70 +937,6 @@ private static void updateLocation(Connection connection, RegionInfo regionInfo,
LOG.info("Updated row {} with server=", regionInfo.getRegionNameAsString(), sn);
}

/**
* Deletes the specified region from META.
* @param connection connection we're using
* @param regionInfo region to be deleted from META
*/
public static void deleteRegionInfo(Connection connection, RegionInfo regionInfo)
throws IOException {
Delete delete = new Delete(regionInfo.getRegionName());
delete.addFamily(HConstants.CATALOG_FAMILY, HConstants.LATEST_TIMESTAMP);
deleteFromMetaTable(connection, delete);
LOG.info("Deleted " + regionInfo.getRegionNameAsString());
}

/**
* Deletes the specified regions from META.
* @param connection connection we're using
* @param regionsInfo list of regions to be deleted from META
*/
public static void deleteRegionInfos(Connection connection, List<RegionInfo> regionsInfo)
throws IOException {
deleteRegionInfos(connection, regionsInfo, EnvironmentEdgeManager.currentTime());
}

/**
* Deletes the specified regions from META.
* @param connection connection we're using
* @param regionsInfo list of regions to be deleted from META
*/
private static void deleteRegionInfos(Connection connection, List<RegionInfo> regionsInfo,
long ts) throws IOException {
List<Delete> deletes = new ArrayList<>(regionsInfo.size());
for (RegionInfo hri : regionsInfo) {
Delete e = new Delete(hri.getRegionName());
e.addFamily(HConstants.CATALOG_FAMILY, ts);
deletes.add(e);
}
deleteFromMetaTable(connection, deletes);
LOG.info("Deleted {} regions from META", regionsInfo.size());
LOG.debug("Deleted regions: {}", regionsInfo);
}

/**
* Overwrites the specified regions from hbase:meta. Deletes old rows for the given regions and
* adds new ones. Regions added back have state CLOSED.
* @param connection connection we're using
* @param regionInfos list of regions to be added to META
*/
public static void overwriteRegions(Connection connection, List<RegionInfo> regionInfos,
int regionReplication) throws IOException {
// use master time for delete marker and the Put
long now = EnvironmentEdgeManager.currentTime();
deleteRegionInfos(connection, regionInfos, now);
// Why sleep? This is the easiest way to ensure that the previous deletes does not
// eclipse the following puts, that might happen in the same ts from the server.
// See HBASE-9906, and HBASE-9879. Once either HBASE-9879, HBASE-8770 is fixed,
// or HBASE-9905 is fixed and meta uses seqIds, we do not need the sleep.
//
// HBASE-13875 uses master timestamp for the mutations. The 20ms sleep is not needed
addRegionsToMeta(connection, regionInfos, regionReplication, now + 1);
LOG.info("Overwritten " + regionInfos.size() + " regions to Meta");
LOG.debug("Overwritten regions: {} ", regionInfos);
}


public static Put addRegionInfo(final Put p, final RegionInfo hri) throws IOException {
p.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(p.getRow())
.setFamily(HConstants.CATALOG_FAMILY).setQualifier(HConstants.REGIONINFO_QUALIFIER)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@
import java.io.IOException;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.backup.HFileArchiver;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.favored.FavoredNodesManager;
Expand Down Expand Up @@ -115,7 +114,7 @@ protected Flow executeFromState(MasterProcedureEnv env, GCRegionState state)
am.getRegionStates().deleteRegion(getRegion());
}
}
MetaTableAccessor.deleteRegionInfo(masterServices.getConnection(), getRegion());
env.getAssignmentManager().getRegionStateStore().deleteRegion(getRegion());
masterServices.getServerManager().removeRegion(getRegion());
FavoredNodesManager fnm = masterServices.getFavoredNodesManager();
if (fnm != null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -297,15 +297,18 @@ table.<MultiRowMutationService, MutateRowsResponse> coprocessorService(
FutureUtils.get(future);
}

private Table getMetaTable() throws IOException {
return master.getConnection().getTable(TableName.META_TABLE_NAME);
}

private Result getRegionCatalogResult(RegionInfo region) throws IOException {
Get get =
new Get(CatalogFamilyFormat.getMetaKeyForRegion(region)).addFamily(HConstants.CATALOG_FAMILY);
try (Table table = master.getConnection().getTable(TableName.META_TABLE_NAME)) {
try (Table table = getMetaTable()) {
return table.get(get);
}
}


private static Put addSequenceNum(Put p, long openSeqNum, int replicaId) throws IOException {
return p.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(p.getRow())
.setFamily(HConstants.CATALOG_FAMILY)
Expand Down Expand Up @@ -490,12 +493,56 @@ static Put addMergeRegions(Put put, Collection<RegionInfo> mergeRegions) throws
// ============================================================================================
// Delete Region State helpers
// ============================================================================================
/**
* Deletes the specified region.
*/
public void deleteRegion(final RegionInfo regionInfo) throws IOException {
deleteRegions(Collections.singletonList(regionInfo));
}

/**
* Deletes the specified regions.
*/
public void deleteRegions(final List<RegionInfo> regions) throws IOException {
MetaTableAccessor.deleteRegionInfos(master.getConnection(), regions);
deleteRegions(regions, EnvironmentEdgeManager.currentTime());
}

private void deleteRegions(List<RegionInfo> regions, long ts) throws IOException {
List<Delete> deletes = new ArrayList<>(regions.size());
for (RegionInfo hri : regions) {
Delete e = new Delete(hri.getRegionName());
e.addFamily(HConstants.CATALOG_FAMILY, ts);
deletes.add(e);
}
try (Table table = getMetaTable()) {
debugLogMutations(deletes);
table.delete(deletes);
}
LOG.info("Deleted {} regions from META", regions.size());
LOG.debug("Deleted regions: {}", regions);
}

/**
* Overwrites the specified regions from hbase:meta. Deletes old rows for the given regions and
* adds new ones. Regions added back have state CLOSED.
* @param connection connection we're using
* @param regionInfos list of regions to be added to META
*/
public void overwriteRegions(List<RegionInfo> regionInfos, int regionReplication)
throws IOException {
// use master time for delete marker and the Put
long now = EnvironmentEdgeManager.currentTime();
deleteRegions(regionInfos, now);
// Why sleep? This is the easiest way to ensure that the previous deletes does not
// eclipse the following puts, that might happen in the same ts from the server.
// See HBASE-9906, and HBASE-9879. Once either HBASE-9879, HBASE-8770 is fixed,
// or HBASE-9905 is fixed and meta uses seqIds, we do not need the sleep.
//
// HBASE-13875 uses master timestamp for the mutations. The 20ms sleep is not needed
MetaTableAccessor.addRegionsToMeta(master.getConnection(), regionInfos, regionReplication,
now + 1);
LOG.info("Overwritten " + regionInfos.size() + " regions to Meta");
LOG.debug("Overwritten regions: {} ", regionInfos);
}

// ==========================================================================
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@
import org.apache.hadoop.hbase.master.MetricsSnapshot;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
import org.apache.hadoop.hbase.master.assignment.RegionStateStore;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
Expand Down Expand Up @@ -413,12 +414,11 @@ private void restoreSnapshot(final MasterProcedureEnv env) throws IOException {

/**
* Apply changes to hbase:meta
* @param env MasterProcedureEnv
* @throws IOException
**/
private void updateMETA(final MasterProcedureEnv env) throws IOException {
try {
Connection conn = env.getMasterServices().getConnection();
RegionStateStore regionStateStore = env.getAssignmentManager().getRegionStateStore();
int regionReplication = modifiedTableDescriptor.getRegionReplication();

// 1. Prepare to restore
Expand All @@ -433,7 +433,7 @@ private void updateMETA(final MasterProcedureEnv env) throws IOException {
// not overwritten/removed, so you end up with old informations
// that are not correct after the restore.
if (regionsToRemove != null) {
MetaTableAccessor.deleteRegionInfos(conn, regionsToRemove);
regionStateStore.deleteRegions(regionsToRemove);
deleteRegionsFromInMemoryStates(regionsToRemove, env, regionReplication);
}

Expand All @@ -449,7 +449,7 @@ private void updateMETA(final MasterProcedureEnv env) throws IOException {
}

if (regionsToRestore != null) {
MetaTableAccessor.overwriteRegions(conn, regionsToRestore, regionReplication);
regionStateStore.overwriteRegions(regionsToRestore, regionReplication);

deleteRegionsFromInMemoryStates(regionsToRestore, env, regionReplication);
addRegionsToInMemoryStates(regionsToRestore, env, regionReplication);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -201,7 +201,6 @@ public static HRegion createHDFSRegionDir(Configuration conf,
* Remove parent
*/
public static void removeParentInMeta(Configuration conf, RegionInfo hri) throws IOException {
Connection conn = ConnectionFactory.createConnection(conf);
MetaTableAccessor.deleteRegionInfo(conn, hri);
throw new UnsupportedOperationException("HBCK1 is read-only now, use HBCK2 instead");
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.master.assignment.RegionStateStore;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.util.Bytes;
Expand Down Expand Up @@ -107,13 +108,15 @@ public void after() throws Exception {
@Test
public void testConsistency() throws IOException {
CatalogJanitor janitor = TEST_UTIL.getHBaseCluster().getMaster().getCatalogJanitor();
RegionStateStore regionStateStore =
TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStateStore();
int gc = janitor.scan();
CatalogJanitor.Report report = janitor.getLastReport();
// Assert no problems.
assertTrue(report.isEmpty());
// Now remove first region in table t2 to see if catalogjanitor scan notices.
List<RegionInfo> t2Ris = MetaTableAccessor.getTableRegions(TEST_UTIL.getConnection(), T2);
MetaTableAccessor.deleteRegionInfo(TEST_UTIL.getConnection(), t2Ris.get(0));
regionStateStore.deleteRegion(t2Ris.get(0));
gc = janitor.scan();
report = janitor.getLastReport();
assertFalse(report.isEmpty());
Expand Down Expand Up @@ -184,7 +187,7 @@ public void testConsistency() throws IOException {
// Make sure only overlaps and no holes are reported.
List<RegionInfo> t4Ris = MetaTableAccessor.getTableRegions(TEST_UTIL.getConnection(), T4);
// delete the region [bb, cc)
MetaTableAccessor.deleteRegionInfo(TEST_UTIL.getConnection(), t4Ris.get(2));
regionStateStore.deleteRegion(t4Ris.get(2));

// add a new region [a, cc)
RegionInfo newRiT4 = RegionInfoBuilder.newBuilder(T4).
Expand All @@ -207,7 +210,7 @@ public void testConsistency() throws IOException {
// Make sure only overlaps and no holes are reported.
List<RegionInfo> t5Ris = MetaTableAccessor.getTableRegions(TEST_UTIL.getConnection(), T5);
// delete the region [cc, dd)
MetaTableAccessor.deleteRegionInfo(TEST_UTIL.getConnection(), t5Ris.get(2));
regionStateStore.deleteRegion(t5Ris.get(2));

// add a new region [a, g)
RegionInfo newRiT5 = RegionInfoBuilder.newBuilder(T5).
Expand Down Expand Up @@ -268,7 +271,8 @@ private void verifyMiddleHole(CatalogJanitor janitor) throws IOException {
RegionInfo firstRegion = getRegionInfo(T3, "".getBytes());
RegionInfo secondRegion = getRegionInfo(T3, "bbb".getBytes());
RegionInfo thirdRegion = getRegionInfo(T3, "ccc".getBytes());
MetaTableAccessor.deleteRegionInfo(TEST_UTIL.getConnection(), secondRegion);
TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStateStore()
.deleteRegion(secondRegion);
LinkedList<Pair<RegionInfo, RegionInfo>> holes = getHoles(janitor, T3);
Pair<RegionInfo, RegionInfo> regionInfoRegionInfoPair = holes.getFirst();
assertTrue(regionInfoRegionInfoPair.getFirst().getTable().equals(T3));
Expand All @@ -280,9 +284,11 @@ private void verifyMiddleHole(CatalogJanitor janitor) throws IOException {
}

private void verifyCornerHoles(CatalogJanitor janitor, TableName tableName) throws IOException {
RegionStateStore regionStateStore =
TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStateStore();
RegionInfo firstRegion = getRegionInfo(tableName, "".getBytes());
RegionInfo secondRegion = getRegionInfo(tableName, "bbb".getBytes());
MetaTableAccessor.deleteRegionInfo(TEST_UTIL.getConnection(), firstRegion);
regionStateStore.deleteRegion(firstRegion);
LinkedList<Pair<RegionInfo, RegionInfo>> holes = getHoles(janitor, tableName);

assertEquals(1, holes.size());
Expand All @@ -295,7 +301,7 @@ private void verifyCornerHoles(CatalogJanitor janitor, TableName tableName) thro

RegionInfo lastRegion = getRegionInfo(tableName, "zzz".getBytes());
RegionInfo secondLastRegion = getRegionInfo(tableName, "yyy".getBytes());
MetaTableAccessor.deleteRegionInfo(TEST_UTIL.getConnection(), lastRegion);
regionStateStore.deleteRegion(lastRegion);
holes = getHoles(janitor, tableName);
assertEquals(2, holes.size());
regionInfoRegionInfoPair = holes.get(1);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,7 @@ public static void tearDownAfterClass() throws Exception {
}

private void deleteRegion(MasterServices services, RegionInfo ri) throws IOException {
MetaTableAccessor.deleteRegionInfo(TEST_UTIL.getConnection(), ri);
services.getAssignmentManager().getRegionStateStore().deleteRegion(ri);
// Delete it from Master context too else it sticks around.
services.getAssignmentManager().getRegionStates().deleteRegion(ri);
}
Expand Down

0 comments on commit 0511089

Please sign in to comment.