HBASE-17850 Backup system repair utility (Vladimir Rodionov)

This commit is contained in:
tedyu 2017-05-22 16:25:59 -07:00
parent f1a9990328
commit 28d619b22b
11 changed files with 544 additions and 142 deletions

View File

@ -109,6 +109,8 @@ public class BackupDriver extends AbstractHBaseTool {
type = BackupCommand.PROGRESS;
} else if (BackupCommand.SET.name().equalsIgnoreCase(cmd)) {
type = BackupCommand.SET;
} else if (BackupCommand.REPAIR.name().equalsIgnoreCase(cmd)) {
type = BackupCommand.REPAIR;
} else {
System.out.println("Unsupported command for backup: " + cmd);
printToolUsage();

View File

@ -96,7 +96,7 @@ public class BackupInfo implements Comparable<BackupInfo> {
/**
* Backup phase
*/
private BackupPhase phase;
private BackupPhase phase = BackupPhase.REQUEST;
/**
* Backup failure message

View File

@ -117,7 +117,7 @@ public interface BackupRestoreConstants {
public static enum BackupCommand {
CREATE, CANCEL, DELETE, DESCRIBE, HISTORY, STATUS, CONVERT, MERGE, STOP, SHOW, HELP, PROGRESS,
SET, SET_ADD, SET_REMOVE, SET_DELETE, SET_DESCRIBE, SET_LIST
SET, SET_ADD, SET_REMOVE, SET_DELETE, SET_DESCRIBE, SET_LIST, REPAIR
}
}

View File

@ -93,9 +93,7 @@ public class BackupAdminImpl implements BackupAdmin {
@Override
public int deleteBackups(String[] backupIds) throws IOException {
// TODO: requires Fault tolerance support, failure will leave system
// in a non-consistent state
// see HBASE-15227
int totalDeleted = 0;
Map<String, HashSet<TableName>> allTablesMap = new HashMap<String, HashSet<TableName>>();

View File

@ -58,6 +58,7 @@ import org.apache.hadoop.hbase.backup.util.BackupUtils;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import com.google.common.collect.Lists;
@ -77,6 +78,7 @@ public final class BackupCommands {
+ " history show history of all successful backups\n"
+ " progress show the progress of the latest backup request\n"
+ " set backup set management\n"
+ " repair repair backup system table"
+ "Run \'hbase backup COMMAND -h\' to see help message for each command\n";
public static final String CREATE_CMD_USAGE =
@ -99,6 +101,8 @@ public final class BackupCommands {
public static final String DELETE_CMD_USAGE = "Usage: hbase backup delete <backup_id>\n"
+ " backup_id Backup image id\n";
public static final String REPAIR_CMD_USAGE = "Usage: hbase backup repair\n";
public static final String CANCEL_CMD_USAGE = "Usage: hbase backup cancel <backup_id>\n"
+ " backup_id Backup image id\n";
@ -191,6 +195,9 @@ public final class BackupCommands {
case SET:
cmd = new BackupSetCommand(conf, cmdline);
break;
case REPAIR:
cmd = new RepairCommand(conf, cmdline);
break;
case HELP:
default:
cmd = new HelpCommand(conf, cmdline);
@ -509,6 +516,9 @@ public final class BackupCommands {
try (BackupAdminImpl admin = new BackupAdminImpl(conn);) {
int deleted = admin.deleteBackups(backupIds);
System.out.println("Deleted " + deleted + " backups. Total requested: " + args.length);
} catch (IOException e) {
System.err.println("Delete command FAILED. Please run backup repair tool to restore backup system integrity");
throw e;
}
}
@ -519,6 +529,66 @@ public final class BackupCommands {
}
}
private static class RepairCommand extends Command {
RepairCommand(Configuration conf, CommandLine cmdline) {
super(conf);
this.cmdline = cmdline;
}
@Override
public void execute() throws IOException {
super.execute();
String[] args = cmdline == null ? null : cmdline.getArgs();
if (args != null && args.length > 1) {
System.err.println("ERROR: wrong number of arguments: " + args.length);
printUsage();
throw new IOException(INCORRECT_USAGE);
}
Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
try (final Connection conn = ConnectionFactory.createConnection(conf);
final BackupSystemTable sysTable = new BackupSystemTable(conn);) {
// Failed backup
BackupInfo backupInfo;
List<BackupInfo> list = sysTable.getBackupInfos(BackupState.RUNNING);
if (list.size() == 0) {
// No failed sessions found
System.out.println("REPAIR status: no failed sessions found.");
return;
}
backupInfo = list.get(0);
// If this is a cancel exception, then we've already cleaned.
// set the failure timestamp of the overall backup
backupInfo.setCompleteTs(EnvironmentEdgeManager.currentTime());
// set failure message
backupInfo.setFailedMsg("REPAIR status: repaired after failure:\n" + backupInfo);
// set overall backup status: failed
backupInfo.setState(BackupState.FAILED);
// compose the backup failed data
String backupFailedData =
"BackupId=" + backupInfo.getBackupId() + ",startts=" + backupInfo.getStartTs()
+ ",failedts=" + backupInfo.getCompleteTs() + ",failedphase="
+ backupInfo.getPhase() + ",failedmessage=" + backupInfo.getFailedMsg();
System.out.println(backupFailedData);
TableBackupClient.cleanupAndRestoreBackupSystem(conn, backupInfo, conf);
// If backup session is updated to FAILED state - means we
// processed recovery already.
sysTable.updateBackupInfo(backupInfo);
sysTable.finishBackupSession();
System.out.println("REPAIR status: finished repair failed session:\n "+ backupInfo);
}
}
@Override
protected void printUsage() {
System.out.println(REPAIR_CMD_USAGE);
}
}
// TODO Cancel command
private static class CancelCommand extends Command {

View File

@ -39,7 +39,6 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.backup.BackupCopyJob;
import org.apache.hadoop.hbase.backup.BackupInfo;
import org.apache.hadoop.hbase.backup.BackupInfo.BackupPhase;
import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
import org.apache.hadoop.hbase.backup.BackupRequest;
import org.apache.hadoop.hbase.backup.BackupRestoreFactory;
import org.apache.hadoop.hbase.backup.BackupType;
@ -64,6 +63,9 @@ import org.apache.hadoop.util.Tool;
public class IncrementalTableBackupClient extends TableBackupClient {
private static final Log LOG = LogFactory.getLog(IncrementalTableBackupClient.class);
protected IncrementalTableBackupClient() {
}
public IncrementalTableBackupClient(final Connection conn, final String backupId,
BackupRequest request) throws IOException {
super(conn, backupId, request);
@ -241,7 +243,6 @@ public class IncrementalTableBackupClient extends TableBackupClient {
// set overall backup status: complete. Here we make sure to complete the backup.
// After this checkpoint, even if entering cancel process, will let the backup finished
try {
backupInfo.setState(BackupState.COMPLETE);
// Set the previousTimestampMap which is before this current log roll to the manifest.
HashMap<TableName, HashMap<String, Long>> previousTimestampMap =
backupManager.readLogTimestampMap();

View File

@ -44,6 +44,8 @@ import org.apache.hadoop.hbase.client.SnapshotDescription;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import com.google.common.annotations.VisibleForTesting;
/**
* Base class for backup operation. Concrete implementation for
* full and incremental backup are delegated to corresponding sub-classes:
@ -55,6 +57,9 @@ public abstract class TableBackupClient {
public static final String BACKUP_CLIENT_IMPL_CLASS = "backup.client.impl.class";
@VisibleForTesting
public static final String BACKUP_TEST_MODE_STAGE = "backup.test.mode.stage";
private static final Log LOG = LogFactory.getLog(TableBackupClient.class);
protected Configuration conf;
@ -441,7 +446,6 @@ public abstract class TableBackupClient {
if (LOG.isDebugEnabled()) {
LOG.debug("Backup " + backupInfo.getBackupId() + " finished: " + backupCompleteData);
}
backupManager.updateBackupInfo(backupInfo);
// when full backup is done:
// - delete HBase snapshot
@ -454,6 +458,8 @@ public abstract class TableBackupClient {
cleanupDistCpLog(backupInfo, conf);
}
deleteBackupTableSnapshot(conn, conf);
backupManager.updateBackupInfo(backupInfo);
// Finish active session
backupManager.finishBackupSession();
@ -466,4 +472,20 @@ public abstract class TableBackupClient {
*/
public abstract void execute() throws IOException;
@VisibleForTesting
protected Stage getTestStage() {
return Stage.valueOf("stage_"+ conf.getInt(BACKUP_TEST_MODE_STAGE, 0));
}
@VisibleForTesting
protected void failStageIf(Stage stage) throws IOException {
Stage current = getTestStage();
if (current == stage) {
throw new IOException("Failed stage " + stage+" in testing");
}
}
public static enum Stage {
stage_0, stage_1, stage_2, stage_3, stage_4
}
}

View File

@ -20,8 +20,10 @@ package org.apache.hadoop.hbase.backup;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import org.apache.commons.logging.Log;
@ -38,10 +40,17 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.backup.BackupInfo.BackupPhase;
import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl;
import org.apache.hadoop.hbase.backup.impl.BackupManager;
import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
import org.apache.hadoop.hbase.backup.impl.FullTableBackupClient;
import org.apache.hadoop.hbase.backup.impl.IncrementalBackupManager;
import org.apache.hadoop.hbase.backup.impl.IncrementalTableBackupClient;
import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
import org.apache.hadoop.hbase.backup.util.BackupUtils;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Durability;
@ -55,6 +64,7 @@ import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.hbase.security.access.SecureTestUtil;
import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
import org.junit.AfterClass;
@ -94,6 +104,180 @@ public class TestBackupBase {
protected static String provider = "defaultProvider";
protected static boolean secure = false;
protected static boolean autoRestoreOnFailure = true;
static class IncrementalTableBackupClientForTest extends IncrementalTableBackupClient
{
public IncrementalTableBackupClientForTest() {
}
public IncrementalTableBackupClientForTest(Connection conn,
String backupId, BackupRequest request) throws IOException {
super(conn, backupId, request);
}
@Override
public void execute() throws IOException
{
// case INCREMENTAL_COPY:
try {
// case PREPARE_INCREMENTAL:
failStageIf(Stage.stage_0);
beginBackup(backupManager, backupInfo);
failStageIf(Stage.stage_1);
backupInfo.setPhase(BackupPhase.PREPARE_INCREMENTAL);
LOG.debug("For incremental backup, current table set is "
+ backupManager.getIncrementalBackupTableSet());
newTimestamps = ((IncrementalBackupManager) backupManager).getIncrBackupLogFileMap();
// copy out the table and region info files for each table
BackupUtils.copyTableRegionInfo(conn, backupInfo, conf);
// convert WAL to HFiles and copy them to .tmp under BACKUP_ROOT
convertWALsToHFiles(backupInfo);
incrementalCopyHFiles(backupInfo);
failStageIf(Stage.stage_2);
// Save list of WAL files copied
backupManager.recordWALFiles(backupInfo.getIncrBackupFileList());
// case INCR_BACKUP_COMPLETE:
// set overall backup status: complete. Here we make sure to complete the backup.
// After this checkpoint, even if entering cancel process, will let the backup finished
// Set the previousTimestampMap which is before this current log roll to the manifest.
HashMap<TableName, HashMap<String, Long>> previousTimestampMap =
backupManager.readLogTimestampMap();
backupInfo.setIncrTimestampMap(previousTimestampMap);
// The table list in backupInfo is good for both full backup and incremental backup.
// For incremental backup, it contains the incremental backup table set.
backupManager.writeRegionServerLogTimestamp(backupInfo.getTables(), newTimestamps);
failStageIf(Stage.stage_3);
HashMap<TableName, HashMap<String, Long>> newTableSetTimestampMap =
backupManager.readLogTimestampMap();
Long newStartCode =
BackupUtils.getMinValue(BackupUtils.getRSLogTimestampMins(newTableSetTimestampMap));
backupManager.writeBackupStartCode(newStartCode);
handleBulkLoad(backupInfo.getTableNames());
failStageIf(Stage.stage_4);
// backup complete
completeBackup(conn, backupInfo, backupManager, BackupType.INCREMENTAL, conf);
} catch (Exception e) {
failBackup(conn, backupInfo, backupManager, e, "Unexpected Exception : ",
BackupType.INCREMENTAL, conf);
throw new IOException(e);
}
}
}
static class FullTableBackupClientForTest extends FullTableBackupClient
{
public FullTableBackupClientForTest() {
}
public FullTableBackupClientForTest(Connection conn, String backupId, BackupRequest request)
throws IOException {
super(conn, backupId, request);
}
@Override
public void execute() throws IOException
{
// Get the stage ID to fail on
try (Admin admin = conn.getAdmin();) {
// Begin BACKUP
beginBackup(backupManager, backupInfo);
failStageIf(Stage.stage_0);
String savedStartCode = null;
boolean firstBackup = false;
// do snapshot for full table backup
savedStartCode = backupManager.readBackupStartCode();
firstBackup = savedStartCode == null || Long.parseLong(savedStartCode) == 0L;
if (firstBackup) {
// This is our first backup. Let's put some marker to system table so that we can hold the logs
// while we do the backup.
backupManager.writeBackupStartCode(0L);
}
failStageIf(Stage.stage_1);
// We roll log here before we do the snapshot. It is possible there is duplicate data
// in the log that is already in the snapshot. But if we do it after the snapshot, we
// could have data loss.
// A better approach is to do the roll log on each RS in the same global procedure as
// the snapshot.
LOG.info("Execute roll log procedure for full backup ...");
Map<String, String> props = new HashMap<String, String>();
props.put("backupRoot", backupInfo.getBackupRootDir());
admin.execProcedure(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE,
LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, props);
failStageIf(Stage.stage_2);
newTimestamps = backupManager.readRegionServerLastLogRollResult();
if (firstBackup) {
// Updates registered log files
// We record ALL old WAL files as registered, because
// this is a first full backup in the system and these
// files are not needed for next incremental backup
List<String> logFiles = BackupUtils.getWALFilesOlderThan(conf, newTimestamps);
backupManager.recordWALFiles(logFiles);
}
// SNAPSHOT_TABLES:
backupInfo.setPhase(BackupPhase.SNAPSHOT);
for (TableName tableName : tableList) {
String snapshotName =
"snapshot_" + Long.toString(EnvironmentEdgeManager.currentTime()) + "_"
+ tableName.getNamespaceAsString() + "_" + tableName.getQualifierAsString();
snapshotTable(admin, tableName, snapshotName);
backupInfo.setSnapshotName(tableName, snapshotName);
}
failStageIf(Stage.stage_3);
// SNAPSHOT_COPY:
// do snapshot copy
LOG.debug("snapshot copy for " + backupId);
snapshotCopy(backupInfo);
// Updates incremental backup table set
backupManager.addIncrementalBackupTableSet(backupInfo.getTables());
// BACKUP_COMPLETE:
// set overall backup status: complete. Here we make sure to complete the backup.
// After this checkpoint, even if entering cancel process, will let the backup finished
backupInfo.setState(BackupState.COMPLETE);
// The table list in backupInfo is good for both full backup and incremental backup.
// For incremental backup, it contains the incremental backup table set.
backupManager.writeRegionServerLogTimestamp(backupInfo.getTables(), newTimestamps);
HashMap<TableName, HashMap<String, Long>> newTableSetTimestampMap =
backupManager.readLogTimestampMap();
Long newStartCode =
BackupUtils.getMinValue(BackupUtils
.getRSLogTimestampMins(newTableSetTimestampMap));
backupManager.writeBackupStartCode(newStartCode);
failStageIf(Stage.stage_4);
// backup complete
completeBackup(conn, backupInfo, backupManager, BackupType.FULL, conf);
} catch (Exception e) {
if(autoRestoreOnFailure) {
failBackup(conn, backupInfo, backupManager, e, "Unexpected BackupException : ",
BackupType.FULL, conf);
}
throw new IOException(e);
}
}
}
/**
* @throws java.lang.Exception
*/

View File

@ -0,0 +1,91 @@
/**
* 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.backup;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import java.util.List;
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
import org.apache.hadoop.hbase.backup.impl.TableBackupClient;
import org.apache.hadoop.hbase.backup.impl.TableBackupClient.Stage;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.util.ToolRunner;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(LargeTests.class)
public class TestBackupRepair extends TestBackupBase {
private static final Log LOG = LogFactory.getLog(TestBackupRepair.class);
@Test
public void testFullBackupWithFailuresAndRestore() throws Exception {
autoRestoreOnFailure = false;
conf1.set(TableBackupClient.BACKUP_CLIENT_IMPL_CLASS,
FullTableBackupClientForTest.class.getName());
int maxStage = Stage.values().length -1;
// Fail stage in loop between 0 and 4 inclusive
for (int stage = 0; stage < maxStage; stage++) {
LOG.info("Running stage " + stage);
runBackupAndFailAtStageWithRestore(stage);
}
}
public void runBackupAndFailAtStageWithRestore(int stage) throws Exception {
conf1.setInt(FullTableBackupClientForTest.BACKUP_TEST_MODE_STAGE, stage);
try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) {
int before = table.getBackupHistory().size();
String[] args =
new String[] { "create", "full", BACKUP_ROOT_DIR, "-t",
table1.getNameAsString() + "," + table2.getNameAsString() };
// Run backup
int ret = ToolRunner.run(conf1, new BackupDriver(), args);
assertFalse(ret == 0);
// Now run restore
args = new String[] {"repair"};
ret = ToolRunner.run(conf1, new BackupDriver(), args);
assertTrue(ret == 0);
List<BackupInfo> backups = table.getBackupHistory();
int after = table.getBackupHistory().size();
assertTrue(after == before +1);
for (BackupInfo data : backups) {
String backupId = data.getBackupId();
assertFalse(checkSucceeded(backupId));
}
Set<TableName> tables = table.getIncrementalBackupTableSet(BACKUP_ROOT_DIR);
assertTrue(tables.size() == 0);
}
}
}

View File

@ -20,162 +20,35 @@ package org.apache.hadoop.hbase.backup;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.backup.BackupInfo.BackupPhase;
import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
import org.apache.hadoop.hbase.backup.impl.FullTableBackupClient;
import org.apache.hadoop.hbase.backup.impl.TableBackupClient;
import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
import org.apache.hadoop.hbase.backup.util.BackupUtils;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.backup.impl.TableBackupClient.Stage;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.util.ToolRunner;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import com.google.common.annotations.VisibleForTesting;
@Category(LargeTests.class)
public class TestFullBackupWithFailures extends TestBackupBase {
private static final Log LOG = LogFactory.getLog(TestFullBackupWithFailures.class);
static class FullTableBackupClientForTest extends FullTableBackupClient
{
public static final String BACKUP_TEST_MODE_STAGE = "backup.test.mode.stage";
public FullTableBackupClientForTest() {
}
public FullTableBackupClientForTest(Connection conn, String backupId, BackupRequest request)
throws IOException {
super(conn, backupId, request);
}
@Override
public void execute() throws IOException
{
// Get the stage ID to fail on
try (Admin admin = conn.getAdmin();) {
// Begin BACKUP
beginBackup(backupManager, backupInfo);
failStageIf(0);
String savedStartCode = null;
boolean firstBackup = false;
// do snapshot for full table backup
savedStartCode = backupManager.readBackupStartCode();
firstBackup = savedStartCode == null || Long.parseLong(savedStartCode) == 0L;
if (firstBackup) {
// This is our first backup. Let's put some marker to system table so that we can hold the logs
// while we do the backup.
backupManager.writeBackupStartCode(0L);
}
failStageIf(1);
// We roll log here before we do the snapshot. It is possible there is duplicate data
// in the log that is already in the snapshot. But if we do it after the snapshot, we
// could have data loss.
// A better approach is to do the roll log on each RS in the same global procedure as
// the snapshot.
LOG.info("Execute roll log procedure for full backup ...");
Map<String, String> props = new HashMap<String, String>();
props.put("backupRoot", backupInfo.getBackupRootDir());
admin.execProcedure(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE,
LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, props);
failStageIf(2);
newTimestamps = backupManager.readRegionServerLastLogRollResult();
if (firstBackup) {
// Updates registered log files
// We record ALL old WAL files as registered, because
// this is a first full backup in the system and these
// files are not needed for next incremental backup
List<String> logFiles = BackupUtils.getWALFilesOlderThan(conf, newTimestamps);
backupManager.recordWALFiles(logFiles);
}
// SNAPSHOT_TABLES:
backupInfo.setPhase(BackupPhase.SNAPSHOT);
for (TableName tableName : tableList) {
String snapshotName =
"snapshot_" + Long.toString(EnvironmentEdgeManager.currentTime()) + "_"
+ tableName.getNamespaceAsString() + "_" + tableName.getQualifierAsString();
snapshotTable(admin, tableName, snapshotName);
backupInfo.setSnapshotName(tableName, snapshotName);
}
failStageIf(3);
// SNAPSHOT_COPY:
// do snapshot copy
LOG.debug("snapshot copy for " + backupId);
snapshotCopy(backupInfo);
// Updates incremental backup table set
backupManager.addIncrementalBackupTableSet(backupInfo.getTables());
// BACKUP_COMPLETE:
// set overall backup status: complete. Here we make sure to complete the backup.
// After this checkpoint, even if entering cancel process, will let the backup finished
backupInfo.setState(BackupState.COMPLETE);
// The table list in backupInfo is good for both full backup and incremental backup.
// For incremental backup, it contains the incremental backup table set.
backupManager.writeRegionServerLogTimestamp(backupInfo.getTables(), newTimestamps);
HashMap<TableName, HashMap<String, Long>> newTableSetTimestampMap =
backupManager.readLogTimestampMap();
Long newStartCode =
BackupUtils.getMinValue(BackupUtils
.getRSLogTimestampMins(newTableSetTimestampMap));
backupManager.writeBackupStartCode(newStartCode);
failStageIf(4);
// backup complete
completeBackup(conn, backupInfo, backupManager, BackupType.FULL, conf);
} catch (Exception e) {
failBackup(conn, backupInfo, backupManager, e, "Unexpected BackupException : ",
BackupType.FULL, conf);
throw new IOException(e);
}
}
@VisibleForTesting
protected int getTestStageId() {
return conf.getInt(BACKUP_TEST_MODE_STAGE, 0);
}
@VisibleForTesting
protected void failStageIf(int stage) throws IOException {
int current = getTestStageId();
if (current == stage) {
throw new IOException("Failed stage " + stage+" in testing");
}
}
}
@Test
public void testFullBackupWithFailures() throws Exception {
conf1.set(TableBackupClient.BACKUP_CLIENT_IMPL_CLASS,
FullTableBackupClientForTest.class.getName());
int stage = (new Random()).nextInt(5);
// Fail random stage between 0 and 4 inclusive
LOG.info("Running stage " + stage);
runBackupAndFailAtStage(stage);
int maxStage = Stage.values().length -1;
// Fail stages between 0 and 4 inclusive
for (int stage = 0; stage <= maxStage; stage++) {
LOG.info("Running stage " + stage);
runBackupAndFailAtStage(stage);
}
}
public void runBackupAndFailAtStage(int stage) throws Exception {

View File

@ -0,0 +1,161 @@
/**
* 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.backup;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl;
import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
import org.apache.hadoop.hbase.backup.impl.TableBackupClient;
import org.apache.hadoop.hbase.backup.impl.TableBackupClient.Stage;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.util.ToolRunner;
import org.junit.Assert;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import com.google.common.collect.Lists;
@Category(LargeTests.class)
@RunWith(Parameterized.class)
public class TestIncrementalBackupWithFailures extends TestBackupBase {
private static final Log LOG = LogFactory.getLog(TestIncrementalBackupWithFailures.class);
@Parameterized.Parameters
public static Collection<Object[]> data() {
provider = "multiwal";
List<Object[]> params = new ArrayList<Object[]>();
params.add(new Object[] { Boolean.TRUE });
return params;
}
public TestIncrementalBackupWithFailures(Boolean b) {
}
// implement all test cases in 1 test since incremental backup/restore has dependencies
@Test
public void TestIncBackupRestore() throws Exception {
int ADD_ROWS = 99;
// #1 - create full backup for all tables
LOG.info("create full backup image for all tables");
List<TableName> tables = Lists.newArrayList(table1, table2);
final byte[] fam3Name = Bytes.toBytes("f3");
table1Desc.addFamily(new HColumnDescriptor(fam3Name));
HBaseTestingUtility.modifyTableSync(TEST_UTIL.getAdmin(), table1Desc);
Connection conn = ConnectionFactory.createConnection(conf1);
int NB_ROWS_FAM3 = 6;
insertIntoTable(conn, table1, fam3Name, 3, NB_ROWS_FAM3).close();
HBaseAdmin admin = null;
admin = (HBaseAdmin) conn.getAdmin();
BackupAdminImpl client = new BackupAdminImpl(conn);
BackupRequest request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR);
String backupIdFull = client.backupTables(request);
assertTrue(checkSucceeded(backupIdFull));
// #2 - insert some data to table
HTable t1 = insertIntoTable(conn, table1, famName, 1, ADD_ROWS);
LOG.debug("writing " + ADD_ROWS + " rows to " + table1);
Assert.assertEquals(TEST_UTIL.countRows(t1), NB_ROWS_IN_BATCH + ADD_ROWS + NB_ROWS_FAM3);
t1.close();
LOG.debug("written " + ADD_ROWS + " rows to " + table1);
HTable t2 = (HTable) conn.getTable(table2);
Put p2;
for (int i = 0; i < 5; i++) {
p2 = new Put(Bytes.toBytes("row-t2" + i));
p2.addColumn(famName, qualName, Bytes.toBytes("val" + i));
t2.put(p2);
}
Assert.assertEquals(TEST_UTIL.countRows(t2), NB_ROWS_IN_BATCH + 5);
t2.close();
LOG.debug("written " + 5 + " rows to " + table2);
// #3 - incremental backup for multiple tables
incrementalBackupWithFailures();
admin.close();
conn.close();
}
private void incrementalBackupWithFailures() throws Exception {
conf1.set(TableBackupClient.BACKUP_CLIENT_IMPL_CLASS,
IncrementalTableBackupClientForTest.class.getName());
int maxStage = Stage.values().length -1;
// Fail stages between 0 and 4 inclusive
for (int stage = 0; stage <= maxStage; stage++) {
LOG.info("Running stage " + stage);
runBackupAndFailAtStage(stage);
}
}
private void runBackupAndFailAtStage(int stage) throws Exception {
conf1.setInt(FullTableBackupClientForTest.BACKUP_TEST_MODE_STAGE, stage);
try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) {
int before = table.getBackupHistory().size();
String[] args =
new String[] { "create", "incremental", BACKUP_ROOT_DIR, "-t",
table1.getNameAsString() + "," + table2.getNameAsString() };
// Run backup
int ret = ToolRunner.run(conf1, new BackupDriver(), args);
assertFalse(ret == 0);
List<BackupInfo> backups = table.getBackupHistory();
int after = table.getBackupHistory().size();
assertTrue(after == before +1);
for (BackupInfo data : backups) {
if(data.getType() == BackupType.FULL) {
assertTrue(data.getState() == BackupState.COMPLETE);
} else {
assertTrue(data.getState() == BackupState.FAILED);
}
}
}
}
}