HBASE-19893 restore_snapshot is broken in master branch when region splits
Signed-off-by: tedyu <yuzhihong@gmail.com>
This commit is contained in:
parent
2eee7a3f8b
commit
682aa69677
|
@ -269,22 +269,26 @@ public class FavoredNodesManager {
|
|||
return result;
|
||||
}
|
||||
|
||||
public synchronized void deleteFavoredNodesForRegions(Collection<RegionInfo> regionInfoList) {
|
||||
for (RegionInfo hri : regionInfoList) {
|
||||
List<ServerName> favNodes = getFavoredNodes(hri);
|
||||
public synchronized void deleteFavoredNodesForRegion(RegionInfo regionInfo) {
|
||||
List<ServerName> favNodes = getFavoredNodes(regionInfo);
|
||||
if (favNodes != null) {
|
||||
if (primaryRSToRegionMap.containsKey(favNodes.get(PRIMARY.ordinal()))) {
|
||||
primaryRSToRegionMap.get(favNodes.get(PRIMARY.ordinal())).remove(hri);
|
||||
primaryRSToRegionMap.get(favNodes.get(PRIMARY.ordinal())).remove(regionInfo);
|
||||
}
|
||||
if (secondaryRSToRegionMap.containsKey(favNodes.get(SECONDARY.ordinal()))) {
|
||||
secondaryRSToRegionMap.get(favNodes.get(SECONDARY.ordinal())).remove(hri);
|
||||
secondaryRSToRegionMap.get(favNodes.get(SECONDARY.ordinal())).remove(regionInfo);
|
||||
}
|
||||
if (teritiaryRSToRegionMap.containsKey(favNodes.get(TERTIARY.ordinal()))) {
|
||||
teritiaryRSToRegionMap.get(favNodes.get(TERTIARY.ordinal())).remove(hri);
|
||||
teritiaryRSToRegionMap.get(favNodes.get(TERTIARY.ordinal())).remove(regionInfo);
|
||||
}
|
||||
globalFavoredNodesAssignmentPlan.removeFavoredNodes(hri);
|
||||
globalFavoredNodesAssignmentPlan.removeFavoredNodes(regionInfo);
|
||||
}
|
||||
}
|
||||
|
||||
public synchronized void deleteFavoredNodesForRegions(Collection<RegionInfo> regionInfoList) {
|
||||
for (RegionInfo regionInfo : regionInfoList) {
|
||||
deleteFavoredNodesForRegion(regionInfo);
|
||||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
|
|
|
@ -516,6 +516,10 @@ public class RegionStates {
|
|||
}
|
||||
}
|
||||
|
||||
public void deleteRegions(final List<RegionInfo> regionInfos) {
|
||||
regionInfos.forEach(this::deleteRegion);
|
||||
}
|
||||
|
||||
ArrayList<RegionStateNode> getTableRegionStateNodes(final TableName tableName) {
|
||||
final ArrayList<RegionStateNode> regions = new ArrayList<RegionStateNode>();
|
||||
for (RegionStateNode node: regionsMap.tailMap(tableName.getName()).values()) {
|
||||
|
@ -769,7 +773,6 @@ public class RegionStates {
|
|||
setServerState(serverName, ServerState.OFFLINE);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public void updateRegionState(final RegionInfo regionInfo, final State state) {
|
||||
final RegionStateNode regionNode = getOrCreateRegionStateNode(regionInfo);
|
||||
synchronized (regionNode) {
|
||||
|
|
|
@ -34,11 +34,15 @@ import org.apache.hadoop.hbase.TableName;
|
|||
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.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.errorhandling.ForeignException;
|
||||
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
|
||||
import org.apache.hadoop.hbase.favored.FavoredNodesManager;
|
||||
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
||||
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.monitoring.MonitoredTask;
|
||||
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
|
||||
|
@ -419,12 +423,13 @@ public class RestoreSnapshotProcedure
|
|||
private void updateMETA(final MasterProcedureEnv env) throws IOException {
|
||||
try {
|
||||
Connection conn = env.getMasterServices().getConnection();
|
||||
int regionReplication = modifiedTableDescriptor.getRegionReplication();
|
||||
|
||||
// 1. Prepare to restore
|
||||
getMonitorStatus().setStatus("Preparing to restore each region");
|
||||
|
||||
// 2. Applies changes to hbase:meta
|
||||
// (2.1). Removes the current set of regions from META
|
||||
// 2. Applies changes to hbase:meta and in-memory states
|
||||
// (2.1). Removes the current set of regions from META and in-memory states
|
||||
//
|
||||
// By removing also the regions to restore (the ones present both in the snapshot
|
||||
// and in the current state) we ensure that no extra fields are present in META
|
||||
|
@ -433,26 +438,25 @@ public class RestoreSnapshotProcedure
|
|||
// that are not correct after the restore.
|
||||
if (regionsToRemove != null) {
|
||||
MetaTableAccessor.deleteRegions(conn, regionsToRemove);
|
||||
deleteRegionsFromInMemoryStates(regionsToRemove, env, regionReplication);
|
||||
}
|
||||
|
||||
// (2.2). Add the new set of regions to META
|
||||
// (2.2). Add the new set of regions to META and in-memory states
|
||||
//
|
||||
// At this point the old regions are no longer present in META.
|
||||
// and the set of regions present in the snapshot will be written to META.
|
||||
// All the information in hbase:meta are coming from the .regioninfo of each region present
|
||||
// in the snapshot folder.
|
||||
if (regionsToAdd != null) {
|
||||
MetaTableAccessor.addRegionsToMeta(
|
||||
conn,
|
||||
regionsToAdd,
|
||||
modifiedTableDescriptor.getRegionReplication());
|
||||
MetaTableAccessor.addRegionsToMeta(conn, regionsToAdd, regionReplication);
|
||||
addRegionsToInMemoryStates(regionsToAdd, env, regionReplication);
|
||||
}
|
||||
|
||||
if (regionsToRestore != null) {
|
||||
MetaTableAccessor.overwriteRegions(
|
||||
conn,
|
||||
regionsToRestore,
|
||||
modifiedTableDescriptor.getRegionReplication());
|
||||
MetaTableAccessor.overwriteRegions(conn, regionsToRestore, regionReplication);
|
||||
|
||||
deleteRegionsFromInMemoryStates(regionsToRestore, env, regionReplication);
|
||||
addRegionsToInMemoryStates(regionsToRestore, env, regionReplication);
|
||||
}
|
||||
|
||||
RestoreSnapshotHelper.RestoreMetaChanges metaChanges =
|
||||
|
@ -479,6 +483,63 @@ public class RestoreSnapshotProcedure
|
|||
monitorStatus.getCompletionTimestamp() - monitorStatus.getStartTime());
|
||||
}
|
||||
|
||||
/**
|
||||
* Delete regions from in-memory states
|
||||
* @param regionInfos regions to delete
|
||||
* @param env MasterProcedureEnv
|
||||
* @param regionReplication the number of region replications
|
||||
*/
|
||||
private void deleteRegionsFromInMemoryStates(List<RegionInfo> regionInfos,
|
||||
MasterProcedureEnv env, int regionReplication) {
|
||||
FavoredNodesManager fnm = env.getMasterServices().getFavoredNodesManager();
|
||||
|
||||
env.getAssignmentManager().getRegionStates().deleteRegions(regionInfos);
|
||||
env.getMasterServices().getServerManager().removeRegions(regionInfos);
|
||||
if (fnm != null) {
|
||||
fnm.deleteFavoredNodesForRegions(regionInfos);
|
||||
}
|
||||
|
||||
// For region replicas
|
||||
if (regionReplication > 1) {
|
||||
for (RegionInfo regionInfo : regionInfos) {
|
||||
for (int i = 1; i < regionReplication; i++) {
|
||||
RegionInfo regionInfoForReplica =
|
||||
RegionReplicaUtil.getRegionInfoForReplica(regionInfo, i);
|
||||
env.getAssignmentManager().getRegionStates().deleteRegion(regionInfoForReplica);
|
||||
env.getMasterServices().getServerManager().removeRegion(regionInfoForReplica);
|
||||
if (fnm != null) {
|
||||
fnm.deleteFavoredNodesForRegion(regionInfoForReplica);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Add regions to in-memory states
|
||||
* @param regionInfos regions to add
|
||||
* @param env MasterProcedureEnv
|
||||
* @param regionReplication the number of region replications
|
||||
*/
|
||||
private void addRegionsToInMemoryStates(List<RegionInfo> regionInfos, MasterProcedureEnv env,
|
||||
int regionReplication) {
|
||||
AssignmentManager am = env.getAssignmentManager();
|
||||
for (RegionInfo regionInfo : regionInfos) {
|
||||
if (regionInfo.isSplit()) {
|
||||
am.getRegionStates().updateRegionState(regionInfo, RegionState.State.SPLIT);
|
||||
} else {
|
||||
am.getRegionStates().updateRegionState(regionInfo, RegionState.State.CLOSED);
|
||||
|
||||
// For region replicas
|
||||
for (int i = 1; i < regionReplication; i++) {
|
||||
RegionInfo regionInfoForReplica =
|
||||
RegionReplicaUtil.getRegionInfoForReplica(regionInfo, i);
|
||||
am.getRegionStates().updateRegionState(regionInfoForReplica, RegionState.State.CLOSED);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void restoreSnapshotAcl(final MasterProcedureEnv env) throws IOException {
|
||||
if (restoreAcl && snapshot.hasUsersAndPermissions() && snapshot.getUsersAndPermissions() != null
|
||||
&& SnapshotDescriptionUtils
|
||||
|
|
|
@ -229,7 +229,8 @@ public class RestoreSnapshotHelper {
|
|||
if (regionNames.contains(regionName)) {
|
||||
LOG.info("region to restore: " + regionName);
|
||||
regionNames.remove(regionName);
|
||||
metaChanges.addRegionToRestore(regionInfo);
|
||||
metaChanges.addRegionToRestore(ProtobufUtil.toRegionInfo(regionManifests.get(regionName)
|
||||
.getRegionInfo()));
|
||||
} else {
|
||||
LOG.info("region to remove: " + regionName);
|
||||
metaChanges.addRegionToRemove(regionInfo);
|
||||
|
|
|
@ -2411,6 +2411,19 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
|
|||
return rows;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns all regions of the specified table
|
||||
*
|
||||
* @param tableName the table name
|
||||
* @return all regions of the specified table
|
||||
* @throws IOException when getting the regions fails.
|
||||
*/
|
||||
private List<RegionInfo> getRegions(TableName tableName) throws IOException {
|
||||
try (Admin admin = getConnection().getAdmin()) {
|
||||
return admin.getRegions(tableName);
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* Find any other region server which is different from the one identified by parameter
|
||||
* @param rs
|
||||
|
@ -2429,9 +2442,6 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
|
|||
/**
|
||||
* Tool to get the reference to the region server object that holds the
|
||||
* region of the specified user table.
|
||||
* It first searches for the meta rows that contain the region of the
|
||||
* specified table, then gets the index of that RS, and finally retrieves
|
||||
* the RS's reference.
|
||||
* @param tableName user table to lookup in hbase:meta
|
||||
* @return region server that holds it, null if the row doesn't exist
|
||||
* @throws IOException
|
||||
|
@ -2439,21 +2449,27 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
|
|||
*/
|
||||
public HRegionServer getRSForFirstRegionInTable(TableName tableName)
|
||||
throws IOException, InterruptedException {
|
||||
List<byte[]> metaRows = getMetaTableRows(tableName);
|
||||
if (metaRows == null || metaRows.isEmpty()) {
|
||||
List<RegionInfo> regions = getRegions(tableName);
|
||||
if (regions == null || regions.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
LOG.debug("Found " + metaRows.size() + " rows for table " +
|
||||
LOG.debug("Found " + regions.size() + " regions for table " +
|
||||
tableName);
|
||||
byte [] firstrow = metaRows.get(0);
|
||||
LOG.debug("FirstRow=" + Bytes.toString(firstrow));
|
||||
|
||||
byte[] firstRegionName = regions.stream()
|
||||
.filter(r -> !r.isOffline())
|
||||
.map(RegionInfo::getRegionName)
|
||||
.findFirst()
|
||||
.orElseThrow(() -> new IOException("online regions not found in table " + tableName));
|
||||
|
||||
LOG.debug("firstRegionName=" + Bytes.toString(firstRegionName));
|
||||
long pause = getConfiguration().getLong(HConstants.HBASE_CLIENT_PAUSE,
|
||||
HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
|
||||
int numRetries = getConfiguration().getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
|
||||
HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
|
||||
RetryCounter retrier = new RetryCounter(numRetries+1, (int)pause, TimeUnit.MICROSECONDS);
|
||||
while(retrier.shouldRetry()) {
|
||||
int index = getMiniHBaseCluster().getServerWith(firstrow);
|
||||
int index = getMiniHBaseCluster().getServerWith(firstRegionName);
|
||||
if (index != -1) {
|
||||
return getMiniHBaseCluster().getRegionServerThreads().get(index).getRegionServer();
|
||||
}
|
||||
|
|
|
@ -25,7 +25,6 @@ import java.io.IOException;
|
|||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
|
@ -307,6 +306,12 @@ public class TestRestoreSnapshotFromClient {
|
|||
// Take a snapshot
|
||||
admin.snapshot(snapshotName1, tableName);
|
||||
|
||||
// Load more data
|
||||
SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 500, FAMILY);
|
||||
|
||||
// Split the second region
|
||||
splitRegion(regionInfos.get(1));
|
||||
|
||||
// Restore the snapshot
|
||||
admin.disableTable(tableName);
|
||||
admin.restoreSnapshot(snapshotName1);
|
||||
|
@ -315,6 +320,28 @@ public class TestRestoreSnapshotFromClient {
|
|||
verifyRowCount(TEST_UTIL, tableName, snapshot1Rows);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRestoreSnapshotAfterTruncate() throws Exception {
|
||||
TableName tableName = TableName.valueOf("testRestoreSnapshotAfterTruncate");
|
||||
SnapshotTestingUtils.createTable(TEST_UTIL, tableName, getNumReplicas(), FAMILY);
|
||||
SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 500, FAMILY);
|
||||
int numOfRows = 0;
|
||||
|
||||
try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
|
||||
numOfRows = countRows(table);
|
||||
}
|
||||
// take snapshot
|
||||
admin.snapshot("snap", tableName);
|
||||
admin.disableTable(tableName);
|
||||
admin.truncateTable(tableName, false);
|
||||
admin.disableTable(tableName);
|
||||
admin.restoreSnapshot("snap");
|
||||
|
||||
admin.enableTable(tableName);
|
||||
verifyRowCount(TEST_UTIL, tableName, numOfRows);
|
||||
SnapshotTestingUtils.verifyReplicasCameOnline(tableName, admin, getNumReplicas());
|
||||
}
|
||||
|
||||
// ==========================================================================
|
||||
// Helpers
|
||||
// ==========================================================================
|
||||
|
|
Loading…
Reference in New Issue