HBASE-19893 restore_snapshot is broken in master branch when region splits

Signed-off-by: Sakthi <sakthivel.azhaku@gmail.com>
Signed-off-by: stack <stack@apache.org>
This commit is contained in:
Toshihiro Suzuki 2018-02-13 01:55:07 +09:00 committed by stack
parent 1fb3fa22fb
commit 8ec797d08e
8 changed files with 231 additions and 38 deletions

View File

@ -269,21 +269,25 @@ public class FavoredNodesManager {
return result; return result;
} }
public synchronized void deleteFavoredNodesForRegions(Collection<RegionInfo> regionInfoList) { public synchronized void deleteFavoredNodesForRegion(RegionInfo regionInfo) {
for (RegionInfo hri : regionInfoList) { List<ServerName> favNodes = getFavoredNodes(regionInfo);
List<ServerName> favNodes = getFavoredNodes(hri); if (favNodes != null) {
if (favNodes != null) { if (primaryRSToRegionMap.containsKey(favNodes.get(PRIMARY.ordinal()))) {
if (primaryRSToRegionMap.containsKey(favNodes.get(PRIMARY.ordinal()))) { primaryRSToRegionMap.get(favNodes.get(PRIMARY.ordinal())).remove(regionInfo);
primaryRSToRegionMap.get(favNodes.get(PRIMARY.ordinal())).remove(hri);
}
if (secondaryRSToRegionMap.containsKey(favNodes.get(SECONDARY.ordinal()))) {
secondaryRSToRegionMap.get(favNodes.get(SECONDARY.ordinal())).remove(hri);
}
if (teritiaryRSToRegionMap.containsKey(favNodes.get(TERTIARY.ordinal()))) {
teritiaryRSToRegionMap.get(favNodes.get(TERTIARY.ordinal())).remove(hri);
}
globalFavoredNodesAssignmentPlan.removeFavoredNodes(hri);
} }
if (secondaryRSToRegionMap.containsKey(favNodes.get(SECONDARY.ordinal()))) {
secondaryRSToRegionMap.get(favNodes.get(SECONDARY.ordinal())).remove(regionInfo);
}
if (teritiaryRSToRegionMap.containsKey(favNodes.get(TERTIARY.ordinal()))) {
teritiaryRSToRegionMap.get(favNodes.get(TERTIARY.ordinal())).remove(regionInfo);
}
globalFavoredNodesAssignmentPlan.removeFavoredNodes(regionInfo);
}
}
public synchronized void deleteFavoredNodesForRegions(Collection<RegionInfo> regionInfoList) {
for (RegionInfo regionInfo : regionInfoList) {
deleteFavoredNodesForRegion(regionInfo);
} }
} }

View File

@ -519,6 +519,10 @@ public class RegionStates {
} }
} }
public void deleteRegions(final List<RegionInfo> regionInfos) {
regionInfos.forEach(this::deleteRegion);
}
ArrayList<RegionStateNode> getTableRegionStateNodes(final TableName tableName) { ArrayList<RegionStateNode> getTableRegionStateNodes(final TableName tableName) {
final ArrayList<RegionStateNode> regions = new ArrayList<RegionStateNode>(); final ArrayList<RegionStateNode> regions = new ArrayList<RegionStateNode>();
for (RegionStateNode node: regionsMap.tailMap(tableName.getName()).values()) { for (RegionStateNode node: regionsMap.tailMap(tableName.getName()).values()) {
@ -778,7 +782,6 @@ public class RegionStates {
setServerState(serverName, ServerState.OFFLINE); setServerState(serverName, ServerState.OFFLINE);
} }
@VisibleForTesting
public void updateRegionState(final RegionInfo regionInfo, final State state) { public void updateRegionState(final RegionInfo regionInfo, final State state) {
final RegionStateNode regionNode = getOrCreateRegionStateNode(regionInfo); final RegionStateNode regionNode = getOrCreateRegionStateNode(regionInfo);
synchronized (regionNode) { synchronized (regionNode) {

View File

@ -34,11 +34,15 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.RegionInfo; 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.TableDescriptor;
import org.apache.hadoop.hbase.errorhandling.ForeignException; import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher; 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.MasterFileSystem;
import org.apache.hadoop.hbase.master.MetricsSnapshot; 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.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor; import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
@ -391,7 +395,7 @@ public class RestoreSnapshotProcedure
env.getMasterServices().getConfiguration(), env.getMasterServices().getConfiguration(),
fs, fs,
manifest, manifest,
modifiedTableDescriptor, modifiedTableDescriptor,
rootDir, rootDir,
monitorException, monitorException,
getMonitorStatus()); getMonitorStatus());
@ -419,12 +423,13 @@ public class RestoreSnapshotProcedure
private void updateMETA(final MasterProcedureEnv env) throws IOException { private void updateMETA(final MasterProcedureEnv env) throws IOException {
try { try {
Connection conn = env.getMasterServices().getConnection(); Connection conn = env.getMasterServices().getConnection();
int regionReplication = modifiedTableDescriptor.getRegionReplication();
// 1. Prepare to restore // 1. Prepare to restore
getMonitorStatus().setStatus("Preparing to restore each region"); getMonitorStatus().setStatus("Preparing to restore each region");
// 2. Applies changes to hbase:meta // 2. Applies changes to hbase:meta and in-memory states
// (2.1). Removes the current set of regions from META // (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 // 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 // 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. // that are not correct after the restore.
if (regionsToRemove != null) { if (regionsToRemove != null) {
MetaTableAccessor.deleteRegions(conn, regionsToRemove); 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. // 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. // 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 // All the information in hbase:meta are coming from the .regioninfo of each region present
// in the snapshot folder. // in the snapshot folder.
if (regionsToAdd != null) { if (regionsToAdd != null) {
MetaTableAccessor.addRegionsToMeta( MetaTableAccessor.addRegionsToMeta(conn, regionsToAdd, regionReplication);
conn, addRegionsToInMemoryStates(regionsToAdd, env, regionReplication);
regionsToAdd,
modifiedTableDescriptor.getRegionReplication());
} }
if (regionsToRestore != null) { if (regionsToRestore != null) {
MetaTableAccessor.overwriteRegions( MetaTableAccessor.overwriteRegions(conn, regionsToRestore, regionReplication);
conn,
regionsToRestore, deleteRegionsFromInMemoryStates(regionsToRestore, env, regionReplication);
modifiedTableDescriptor.getRegionReplication()); addRegionsToInMemoryStates(regionsToRestore, env, regionReplication);
} }
RestoreSnapshotHelper.RestoreMetaChanges metaChanges = RestoreSnapshotHelper.RestoreMetaChanges metaChanges =
@ -479,6 +483,63 @@ public class RestoreSnapshotProcedure
monitorStatus.getCompletionTimestamp() - monitorStatus.getStartTime()); 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 { private void restoreSnapshotAcl(final MasterProcedureEnv env) throws IOException {
if (restoreAcl && snapshot.hasUsersAndPermissions() && snapshot.getUsersAndPermissions() != null if (restoreAcl && snapshot.hasUsersAndPermissions() && snapshot.getUsersAndPermissions() != null
&& SnapshotDescriptionUtils && SnapshotDescriptionUtils

View File

@ -229,7 +229,8 @@ public class RestoreSnapshotHelper {
if (regionNames.contains(regionName)) { if (regionNames.contains(regionName)) {
LOG.info("region to restore: " + regionName); LOG.info("region to restore: " + regionName);
regionNames.remove(regionName); regionNames.remove(regionName);
metaChanges.addRegionToRestore(regionInfo); metaChanges.addRegionToRestore(ProtobufUtil.toRegionInfo(regionManifests.get(regionName)
.getRegionInfo()));
} else { } else {
LOG.info("region to remove: " + regionName); LOG.info("region to remove: " + regionName);
metaChanges.addRegionToRemove(regionInfo); metaChanges.addRegionToRemove(regionInfo);

View File

@ -2519,6 +2519,19 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
return rows; 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 * Find any other region server which is different from the one identified by parameter
* @param rs * @param rs
@ -2537,9 +2550,6 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
/** /**
* Tool to get the reference to the region server object that holds the * Tool to get the reference to the region server object that holds the
* region of the specified user table. * 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 * @param tableName user table to lookup in hbase:meta
* @return region server that holds it, null if the row doesn't exist * @return region server that holds it, null if the row doesn't exist
* @throws IOException * @throws IOException
@ -2547,21 +2557,27 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
*/ */
public HRegionServer getRSForFirstRegionInTable(TableName tableName) public HRegionServer getRSForFirstRegionInTable(TableName tableName)
throws IOException, InterruptedException { throws IOException, InterruptedException {
List<byte[]> metaRows = getMetaTableRows(tableName); List<RegionInfo> regions = getRegions(tableName);
if (metaRows == null || metaRows.isEmpty()) { if (regions == null || regions.isEmpty()) {
return null; return null;
} }
LOG.debug("Found " + metaRows.size() + " rows for table " + LOG.debug("Found " + regions.size() + " regions for table " +
tableName); 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, long pause = getConfiguration().getLong(HConstants.HBASE_CLIENT_PAUSE,
HConstants.DEFAULT_HBASE_CLIENT_PAUSE); HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
int numRetries = getConfiguration().getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, int numRetries = getConfiguration().getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER); HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
RetryCounter retrier = new RetryCounter(numRetries+1, (int)pause, TimeUnit.MICROSECONDS); RetryCounter retrier = new RetryCounter(numRetries+1, (int)pause, TimeUnit.MICROSECONDS);
while(retrier.shouldRetry()) { while(retrier.shouldRetry()) {
int index = getMiniHBaseCluster().getServerWith(firstrow); int index = getMiniHBaseCluster().getServerWith(firstRegionName);
if (index != -1) { if (index != -1) {
return getMiniHBaseCluster().getRegionServerThreads().get(index).getRegionServer(); return getMiniHBaseCluster().getRegionServerThreads().get(index).getRegionServer();
} }

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.client;
import java.io.IOException; import java.io.IOException;
import java.util.List; import java.util.List;
import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
import org.junit.Test; import org.junit.Test;
public class RestoreSnapshotFromClientAfterSplittingRegionsTestBase public class RestoreSnapshotFromClientAfterSplittingRegionsTestBase
@ -35,6 +36,12 @@ public class RestoreSnapshotFromClientAfterSplittingRegionsTestBase
// Take a snapshot // Take a snapshot
admin.snapshot(snapshotName1, tableName); 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 // Restore the snapshot
admin.disableTable(tableName); admin.disableTable(tableName);
admin.restoreSnapshot(snapshotName1); admin.restoreSnapshot(snapshotName1);

View File

@ -0,0 +1,48 @@
/**
* 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.client;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
import org.junit.Test;
public class RestoreSnapshotFromClientAfterTruncateTestBase
extends RestoreSnapshotFromClientTestBase {
@Test
public void testRestoreSnapshotAfterTruncate() throws Exception {
TableName tableName = TableName.valueOf(getValidMethodName());
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());
}
}

View File

@ -0,0 +1,53 @@
/**
* 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.client;
import java.util.Arrays;
import java.util.List;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.junit.ClassRule;
import org.junit.experimental.categories.Category;
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({ LargeTests.class, ClientTests.class })
public class TestRestoreSnapshotFromClientAfterTruncate
extends RestoreSnapshotFromClientAfterTruncateTestBase {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestRestoreSnapshotFromClientAfterTruncate.class);
@Parameter
public int numReplicas;
@Parameters(name = "{index}: regionReplication={0}")
public static List<Object[]> params() {
return Arrays.asList(new Object[] { 1 }, new Object[] { 3 });
}
@Override
protected int getNumReplicas() {
return numReplicas;
}
}