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:
parent
1fb3fa22fb
commit
8ec797d08e
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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) {
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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();
|
||||||
}
|
}
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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());
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
Loading…
Reference in New Issue