HBASE-27201 Clean up error-prone findings in hbase-backup (#4643)

Signed-off-by: Duo Zhang <zhangduo@apache.org>
Signed-off-by: Viraj Jasani <vjasani@apache.org>
This commit is contained in:
Andrew Purtell 2022-08-11 00:02:17 -07:00 committed by GitHub
parent 5eaeff5fcd
commit 2c3abae18a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
17 changed files with 111 additions and 129 deletions

View File

@ -37,7 +37,6 @@ import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.Builder;
/**
* An object to encapsulate the information for each backup session
@ -451,13 +450,13 @@ public class BackupInfo implements Comparable<BackupInfo> {
return toProtosBackupInfo().toByteArray();
}
private void setBackupTableInfoMap(Builder builder) {
private void setBackupTableInfoMap(BackupProtos.BackupInfo.Builder builder) {
for (Entry<TableName, BackupTableInfo> entry : backupTableInfoMap.entrySet()) {
builder.addBackupTableInfo(entry.getValue().toProto());
}
}
private void setTableSetTimestampMap(Builder builder) {
private void setTableSetTimestampMap(BackupProtos.BackupInfo.Builder builder) {
if (this.getTableSetTimestampMap() != null) {
for (Entry<TableName, Map<String, Long>> entry : this.getTableSetTimestampMap().entrySet()) {
builder.putTableSetTimestamp(entry.getKey().getNameAsString(),
@ -531,10 +530,9 @@ public class BackupInfo implements Comparable<BackupInfo> {
sb.append("Type=" + getType()).append(",");
sb.append("Tables=" + getTableListAsString()).append(",");
sb.append("State=" + getState()).append(",");
Date date = null;
Calendar cal = Calendar.getInstance();
cal.setTimeInMillis(getStartTs());
date = cal.getTime();
Date date = cal.getTime();
sb.append("Start time=" + date).append(",");
if (state == BackupState.FAILED) {
sb.append("Failed message=" + getFailedMsg()).append(",");
@ -560,7 +558,7 @@ public class BackupInfo implements Comparable<BackupInfo> {
}
public String getTableListAsString() {
StringBuffer sb = new StringBuffer();
StringBuilder sb = new StringBuilder();
sb.append("{");
sb.append(StringUtils.join(backupTableInfoMap.keySet(), ","));
sb.append("}");

View File

@ -83,7 +83,7 @@ public class RestoreDriver extends AbstractHBaseTool {
Log4jUtils.disableZkAndClientLoggers();
}
private int parseAndRun(String[] args) throws IOException {
private int parseAndRun() throws IOException {
// Check if backup is enabled
if (!BackupManager.isBackupEnabled(getConf())) {
System.err.println(BackupRestoreConstants.ENABLE_BACKUP);
@ -146,7 +146,7 @@ public class RestoreDriver extends AbstractHBaseTool {
if (cmd.hasOption(OPTION_SET)) {
String setName = cmd.getOptionValue(OPTION_SET);
try {
tables = getTablesForSet(conn, setName, conf);
tables = getTablesForSet(conn, setName);
} catch (IOException e) {
System.out.println("ERROR: " + e.getMessage() + " for setName=" + setName);
printToolUsage();
@ -182,8 +182,7 @@ public class RestoreDriver extends AbstractHBaseTool {
return 0;
}
private String getTablesForSet(Connection conn, String name, Configuration conf)
throws IOException {
private String getTablesForSet(Connection conn, String name) throws IOException {
try (final BackupSystemTable table = new BackupSystemTable(conn)) {
List<TableName> tables = table.describeBackupSet(name);
@ -214,7 +213,7 @@ public class RestoreDriver extends AbstractHBaseTool {
@Override
protected int doWork() throws Exception {
return parseAndRun(cmd.getArgs());
return parseAndRun();
}
public static void main(String[] args) throws Exception {

View File

@ -65,6 +65,7 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.common.base.Splitter;
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
import org.apache.hbase.thirdparty.org.apache.commons.cli.HelpFormatter;
@ -310,7 +311,7 @@ public final class BackupCommands {
String setName = null;
if (cmdline.hasOption(OPTION_SET)) {
setName = cmdline.getOptionValue(OPTION_SET);
tables = getTablesForSet(setName, getConf());
tables = getTablesForSet(setName);
if (tables == null) {
System.out
@ -371,7 +372,7 @@ public final class BackupCommands {
}
}
private String getTablesForSet(String name, Configuration conf) throws IOException {
private String getTablesForSet(String name) throws IOException {
try (final BackupSystemTable table = new BackupSystemTable(conn)) {
List<TableName> tables = table.describeBackupSet(name);
@ -1001,14 +1002,14 @@ public final class BackupCommands {
processSetDescribe(args);
break;
case SET_LIST:
processSetList(args);
processSetList();
break;
default:
break;
}
}
private void processSetList(String[] args) throws IOException {
private void processSetList() throws IOException {
super.execute();
// List all backup set names
@ -1087,17 +1088,12 @@ public final class BackupCommands {
throw new IOException(INCORRECT_USAGE);
}
super.execute();
String setName = args[2];
String[] tables = args[3].split(",");
TableName[] tableNames = new TableName[tables.length];
for (int i = 0; i < tables.length; i++) {
tableNames[i] = TableName.valueOf(tables[i]);
}
TableName[] tableNames =
Splitter.on(',').splitToStream(args[3]).map(TableName::valueOf).toArray(TableName[]::new);
try (final BackupAdminImpl admin = new BackupAdminImpl(conn)) {
admin.addToBackupSet(setName, tableNames);
}
}
private BackupCommand getCommand(String cmdStr) throws IOException {

View File

@ -366,7 +366,6 @@ public class BackupManifest {
}
// backup image directory
private String tableBackupDir = null;
private BackupImage backupImage;
/**
@ -385,7 +384,6 @@ public class BackupManifest {
* @param backup The ongoing backup session info
*/
public BackupManifest(BackupInfo backup, TableName table) {
this.tableBackupDir = backup.getTableBackupDir(table);
List<TableName> tables = new ArrayList<TableName>();
tables.add(table);
BackupImage.Builder builder = BackupImage.newBuilder();
@ -468,7 +466,7 @@ public class BackupManifest {
/**
* TODO: fix it. Persist the manifest file.
* @throws IOException IOException when storing the manifest file.
* @throws BackupException if an error occurred while storing the manifest file.
*/
public void store(Configuration conf) throws BackupException {
byte[] data = backupImage.toProto().toByteArray();
@ -526,7 +524,7 @@ public class BackupManifest {
restoreImages.put(Long.valueOf(image.startTs), image);
}
return new ArrayList<>(
reverse ? (restoreImages.descendingMap().values()) : (restoreImages.values()));
reverse ? restoreImages.descendingMap().values() : restoreImages.values());
}
/**

View File

@ -19,6 +19,8 @@ package org.apache.hadoop.hbase.backup.impl;
import java.io.Closeable;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
@ -69,6 +71,9 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.base.Splitter;
import org.apache.hbase.thirdparty.com.google.common.collect.Iterators;
import org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
@ -237,6 +242,7 @@ public final class BackupSystemTable implements Closeable {
try {
Thread.sleep(100);
} catch (InterruptedException e) {
throw (IOException) new InterruptedIOException().initCause(e);
}
if (EnvironmentEdgeManager.currentTime() - startTime > TIMEOUT) {
throw new IOException(
@ -302,6 +308,7 @@ public final class BackupSystemTable implements Closeable {
public Map<byte[], List<Path>>[] readBulkLoadedFiles(String backupId, List<TableName> sTableList)
throws IOException {
Scan scan = BackupSystemTable.createScanForBulkLoadedFiles(backupId);
@SuppressWarnings("unchecked")
Map<byte[], List<Path>>[] mapForSrc = new Map[sTableList == null ? 1 : sTableList.size()];
try (Table table = connection.getTable(bulkLoadTableName);
ResultScanner scanner = table.getScanner(scan)) {
@ -574,7 +581,7 @@ public final class BackupSystemTable implements Closeable {
if (val.length == 0) {
return null;
}
return new String(val);
return new String(val, StandardCharsets.UTF_8);
}
}
@ -1639,7 +1646,8 @@ public final class BackupSystemTable implements Closeable {
if (val.length == 0) {
return null;
}
return new String(val).split(",");
return Splitter.on(',').splitToStream(new String(val, StandardCharsets.UTF_8))
.toArray(String[]::new);
}
}
@ -1654,7 +1662,7 @@ public final class BackupSystemTable implements Closeable {
Get get = new Get(MERGE_OP_ROW);
try (Table table = connection.getTable(tableName)) {
Result res = table.get(get);
return (!res.isEmpty());
return !res.isEmpty();
}
}
@ -1720,7 +1728,8 @@ public final class BackupSystemTable implements Closeable {
if (val.length == 0) {
return null;
}
return new String(val).split(",");
return Splitter.on(',').splitToStream(new String(val, StandardCharsets.UTF_8))
.toArray(String[]::new);
}
}
@ -1737,20 +1746,22 @@ public final class BackupSystemTable implements Closeable {
}
static String getTableNameFromOrigBulkLoadRow(String rowStr) {
String[] parts = rowStr.split(BLK_LD_DELIM);
return parts[1];
// format is bulk : namespace : table : region : file
return Iterators.get(Splitter.onPattern(BLK_LD_DELIM).split(rowStr).iterator(), 1);
}
static String getRegionNameFromOrigBulkLoadRow(String rowStr) {
// format is bulk : namespace : table : region : file
String[] parts = rowStr.split(BLK_LD_DELIM);
List<String> parts = Splitter.onPattern(BLK_LD_DELIM).splitToList(rowStr);
Iterator<String> i = parts.iterator();
int idx = 3;
if (parts.length == 4) {
if (parts.size() == 4) {
// the table is in default namespace
idx = 2;
}
LOG.debug("bulk row string " + rowStr + " region " + parts[idx]);
return parts[idx];
String region = Iterators.get(i, idx);
LOG.debug("bulk row string " + rowStr + " region " + region);
return region;
}
/*

View File

@ -204,8 +204,7 @@ public class IncrementalTableBackupClient extends TableBackupClient {
String tgtDest = backupInfo.getBackupRootDir() + Path.SEPARATOR + backupInfo.getBackupId();
int attempt = 1;
while (activeFiles.size() > 0) {
LOG.info(
"Copy " + activeFiles.size() + " active bulk loaded files. Attempt =" + (attempt++));
LOG.info("Copy " + activeFiles.size() + " active bulk loaded files. Attempt =" + attempt++);
String[] toCopy = new String[activeFiles.size()];
activeFiles.toArray(toCopy);
// Active file can be archived during copy operation,

View File

@ -181,7 +181,7 @@ public class RestoreTablesClient {
private List<Path> getFilesRecursively(String fileBackupDir)
throws IllegalArgumentException, IOException {
FileSystem fs = FileSystem.get((new Path(fileBackupDir)).toUri(), new Configuration());
FileSystem fs = FileSystem.get(new Path(fileBackupDir).toUri(), new Configuration());
List<Path> list = new ArrayList<>();
RemoteIterator<LocatedFileStatus> it = fs.listFiles(new Path(fileBackupDir), true);
while (it.hasNext()) {

View File

@ -322,7 +322,7 @@ public abstract class TableBackupClient {
* @return meta data dir
*/
protected String obtainBackupMetaDataStr(BackupInfo backupInfo) {
StringBuffer sb = new StringBuffer();
StringBuilder sb = new StringBuilder();
sb.append("type=" + backupInfo.getType() + ",tablelist=");
for (TableName table : backupInfo.getTables()) {
sb.append(table + ";");

View File

@ -47,7 +47,6 @@ import org.apache.hadoop.tools.DistCp;
import org.apache.hadoop.tools.DistCpConstants;
import org.apache.hadoop.tools.DistCpOptions;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException.NoNodeException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -127,7 +126,7 @@ public class MapReduceBackupCopyJob implements BackupCopyJob {
* @param backupInfo backup info
* @param newProgress progress
* @param bytesCopied bytes copied
* @throws NoNodeException exception
* @throws IOException exception
*/
static void updateProgress(BackupInfo backupInfo, BackupManager backupManager, int newProgress,
long bytesCopied) throws IOException {
@ -361,7 +360,7 @@ public class MapReduceBackupCopyJob implements BackupCopyJob {
* @param conf The hadoop configuration
* @param copyType The backup copy type
* @param options Options for customized ExportSnapshot or DistCp
* @throws Exception exception
* @throws IOException exception
*/
@Override
public int copy(BackupInfo context, BackupManager backupManager, Configuration conf,

View File

@ -22,11 +22,12 @@ import static org.apache.hadoop.hbase.backup.util.BackupUtils.succeeded;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.ArrayDeque;
import java.util.ArrayList;
import java.util.Deque;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.Stack;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
@ -257,7 +258,7 @@ public class MapReduceBackupMergeJob implements BackupMergeJob {
*/
protected Path convertToDest(Path p, Path backupDirPath) {
String backupId = backupDirPath.getName();
Stack<String> stack = new Stack<String>();
Deque<String> stack = new ArrayDeque<String>();
String name = null;
while (true) {
name = p.getName();

View File

@ -17,6 +17,7 @@
*/
package org.apache.hadoop.hbase.backup.regionserver;
import java.nio.charset.StandardCharsets;
import java.util.HashMap;
import java.util.List;
import java.util.concurrent.Callable;
@ -56,7 +57,7 @@ public class LogRollBackupSubprocedure extends Subprocedure {
this.rss = rss;
this.taskManager = taskManager;
if (data != null) {
backupRoot = new String(data);
backupRoot = new String(data, StandardCharsets.UTF_8);
}
}

View File

@ -63,12 +63,15 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.base.Splitter;
import org.apache.hbase.thirdparty.com.google.common.collect.Iterators;
/**
* A collection for methods used by multiple classes to backup HBase tables.
*/
@InterfaceAudience.Private
public final class BackupUtils {
protected static final Logger LOG = LoggerFactory.getLogger(BackupUtils.class);
private static final Logger LOG = LoggerFactory.getLogger(BackupUtils.class);
public static final String LOGNAME_SEPARATOR = ".";
public static final int MILLISEC_IN_HOUR = 3600000;
@ -136,9 +139,10 @@ public final class BackupUtils {
// write a copy of descriptor to the target directory
Path target = new Path(backupInfo.getTableBackupDir(table));
FileSystem targetFs = target.getFileSystem(conf);
FSTableDescriptors descriptors =
new FSTableDescriptors(targetFs, CommonFSUtils.getRootDir(conf));
descriptors.createTableDescriptorForTableDirectory(target, orig, false);
try (FSTableDescriptors descriptors =
new FSTableDescriptors(targetFs, CommonFSUtils.getRootDir(conf))) {
descriptors.createTableDescriptorForTableDirectory(target, orig, false);
}
LOG.debug("Attempting to copy table info for:" + table + " target: " + target
+ " descriptor: " + orig);
LOG.debug("Finished copying tableinfo.");
@ -279,13 +283,8 @@ public final class BackupUtils {
if (tables == null) {
return null;
}
String[] tableArray = tables.split(BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND);
TableName[] ret = new TableName[tableArray.length];
for (int i = 0; i < tableArray.length; i++) {
ret[i] = TableName.valueOf(tableArray[i]);
}
return ret;
return Splitter.on(BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND).splitToStream(tables)
.map(TableName::valueOf).toArray(TableName[]::new);
}
/**
@ -594,8 +593,7 @@ public final class BackupUtils {
}
private long getTimestamp(String backupId) {
String[] split = backupId.split("_");
return Long.parseLong(split[1]);
return Long.parseLong(Iterators.get(Splitter.on('_').split(backupId).iterator(), 1));
}
});
return infos;
@ -734,7 +732,7 @@ public final class BackupUtils {
public static String findMostRecentBackupId(String[] backupIds) {
long recentTimestamp = Long.MIN_VALUE;
for (String backupId : backupIds) {
long ts = Long.parseLong(backupId.split("_")[1]);
long ts = Long.parseLong(Iterators.get(Splitter.on('_').split(backupId).iterator(), 1));
if (ts > recentTimestamp) {
recentTimestamp = ts;
}

View File

@ -325,8 +325,7 @@ public class RestoreTool {
+ ", will only create table");
}
tableDescriptor = TableDescriptorBuilder.copy(newTableName, tableDescriptor);
checkAndCreateTable(conn, tableBackupPath, tableName, newTableName, null, tableDescriptor,
truncateIfExists);
checkAndCreateTable(conn, newTableName, null, tableDescriptor, truncateIfExists);
return;
} else {
throw new IllegalStateException(
@ -347,8 +346,7 @@ public class RestoreTool {
// should only try to create the table with all region informations, so we could pre-split
// the regions in fine grain
checkAndCreateTable(conn, tableBackupPath, tableName, newTableName, regionPathList,
tableDescriptor, truncateIfExists);
checkAndCreateTable(conn, newTableName, regionPathList, tableDescriptor, truncateIfExists);
RestoreJob restoreService = BackupRestoreFactory.getRestoreJob(conf);
Path[] paths = new Path[regionPathList.size()];
regionPathList.toArray(paths);
@ -460,17 +458,15 @@ public class RestoreTool {
* Prepare the table for bulkload, most codes copied from {@code createTable} method in
* {@code BulkLoadHFilesTool}.
* @param conn connection
* @param tableBackupPath path
* @param tableName table name
* @param targetTableName target table name
* @param regionDirList region directory list
* @param htd table descriptor
* @param truncateIfExists truncates table if exists
* @throws IOException exception
*/
private void checkAndCreateTable(Connection conn, Path tableBackupPath, TableName tableName,
TableName targetTableName, ArrayList<Path> regionDirList, TableDescriptor htd,
boolean truncateIfExists) throws IOException {
private void checkAndCreateTable(Connection conn, TableName targetTableName,
ArrayList<Path> regionDirList, TableDescriptor htd, boolean truncateIfExists)
throws IOException {
try (Admin admin = conn.getAdmin()) {
boolean createNew = false;
if (admin.tableExists(targetTableName)) {

View File

@ -330,9 +330,6 @@ public class TestBackupBase {
}
}
/**
* @throws Exception if deleting the archive directory or shutting down the mini cluster fails
*/
@AfterClass
public static void tearDown() throws Exception {
try {

View File

@ -22,9 +22,8 @@ import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.IdentityHashMap;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@ -63,9 +62,6 @@ public class TestBackupHFileCleaner {
static FileSystem fs = null;
Path root;
/**
* @throws Exception if starting the mini cluster or getting the filesystem fails
*/
@BeforeClass
public static void setUpBeforeClass() throws Exception {
conf.setBoolean(BackupRestoreConstants.BACKUP_ENABLE_KEY, true);
@ -74,9 +70,6 @@ public class TestBackupHFileCleaner {
fs = FileSystem.get(conf);
}
/**
* @throws Exception if closing the filesystem or shutting down the mini cluster fails
*/
@AfterClass
public static void tearDownAfterClass() throws Exception {
if (fs != null) {
@ -109,12 +102,13 @@ public class TestBackupHFileCleaner {
BackupHFileCleaner cleaner = new BackupHFileCleaner();
cleaner.setConf(conf);
cleaner.setCheckForFullyBackedUpTables(false);
// 3. Assert that file as is should be deletable
List<FileStatus> stats = new ArrayList<>();
// Prime the cleaner
cleaner.getDeletableFiles(stats);
// 3. Assert that file as is should be deletable
FileStatus stat = fs.getFileStatus(file);
stats.add(stat);
Iterable<FileStatus> deletable = cleaner.getDeletableFiles(stats);
deletable = cleaner.getDeletableFiles(stats);
boolean found = false;
for (FileStatus stat1 : deletable) {
if (stat.equals(stat1)) {
@ -132,15 +126,15 @@ public class TestBackupHFileCleaner {
BackupSystemTable sysTbl = new BackupSystemTable(conn)) {
List<TableName> sTableList = new ArrayList<>();
sTableList.add(tableName);
Map<byte[], List<Path>>[] maps = new Map[1];
maps[0] = new HashMap<>();
@SuppressWarnings("unchecked")
IdentityHashMap<byte[], List<Path>>[] maps = new IdentityHashMap[1];
maps[0] = new IdentityHashMap<>();
maps[0].put(Bytes.toBytes(famName), list);
sysTbl.writeBulkLoadedFiles(sTableList, maps, "1");
}
// 5. Assert file should not be deletable
deletable = cleaner.getDeletableFiles(stats);
deletable = cleaner.getDeletableFiles(stats);
found = false;
for (FileStatus stat1 : deletable) {
if (stat.equals(stat1)) {

View File

@ -109,10 +109,10 @@ public class TestBackupSystemTable {
@Test
public void testWriteReadBackupStartCode() throws IOException {
Long code = 100L;
long code = 100L;
table.writeBackupStartCode(code, "root");
String readCode = table.readBackupStartCode("root");
assertEquals(code, new Long(Long.parseLong(readCode)));
assertEquals(code, Long.parseLong(readCode));
cleanBackupTable();
}
@ -126,7 +126,7 @@ public class TestBackupSystemTable {
}
@Test
public void testBackupHistory() throws IOException {
public void testBackupHistory() throws Exception {
int n = 10;
List<BackupInfo> list = createBackupInfoList(n);
@ -153,7 +153,7 @@ public class TestBackupSystemTable {
}
@Test
public void testBackupDelete() throws IOException {
public void testBackupDelete() throws Exception {
try (BackupSystemTable table = new BackupSystemTable(conn)) {
int n = 10;
List<BackupInfo> list = createBackupInfoList(n);
@ -226,29 +226,29 @@ public class TestBackupSystemTable {
tables2.add(TableName.valueOf("t5"));
table.addIncrementalBackupTableSet(tables1, "root");
BackupSystemTable table = new BackupSystemTable(conn);
TreeSet<TableName> res1 = (TreeSet<TableName>) table.getIncrementalBackupTableSet("root");
assertTrue(tables1.size() == res1.size());
Iterator<TableName> desc1 = tables1.descendingIterator();
Iterator<TableName> desc2 = res1.descendingIterator();
while (desc1.hasNext()) {
assertEquals(desc1.next(), desc2.next());
try (BackupSystemTable systemTable = new BackupSystemTable(conn)) {
TreeSet<TableName> res1 =
(TreeSet<TableName>) systemTable.getIncrementalBackupTableSet("root");
assertTrue(tables1.size() == res1.size());
Iterator<TableName> desc1 = tables1.descendingIterator();
Iterator<TableName> desc2 = res1.descendingIterator();
while (desc1.hasNext()) {
assertEquals(desc1.next(), desc2.next());
}
systemTable.addIncrementalBackupTableSet(tables2, "root");
TreeSet<TableName> res2 =
(TreeSet<TableName>) systemTable.getIncrementalBackupTableSet("root");
assertTrue((tables2.size() + tables1.size() - 1) == res2.size());
tables1.addAll(tables2);
desc1 = tables1.descendingIterator();
desc2 = res2.descendingIterator();
while (desc1.hasNext()) {
assertEquals(desc1.next(), desc2.next());
}
}
table.addIncrementalBackupTableSet(tables2, "root");
TreeSet<TableName> res2 = (TreeSet<TableName>) table.getIncrementalBackupTableSet("root");
assertTrue((tables2.size() + tables1.size() - 1) == res2.size());
tables1.addAll(tables2);
desc1 = tables1.descendingIterator();
desc2 = res2.descendingIterator();
while (desc1.hasNext()) {
assertEquals(desc1.next(), desc2.next());
}
cleanBackupTable();
}
@Test
@ -274,9 +274,9 @@ public class TestBackupSystemTable {
for (TableName t : tables) {
Map<String, Long> rstm = result.get(t);
assertNotNull(rstm);
assertEquals(rstm.get("rs1:100"), new Long(100L));
assertEquals(rstm.get("rs2:100"), new Long(101L));
assertEquals(rstm.get("rs3:100"), new Long(103L));
assertEquals(rstm.get("rs1:100"), Long.valueOf(100L));
assertEquals(rstm.get("rs2:100"), Long.valueOf(101L));
assertEquals(rstm.get("rs3:100"), Long.valueOf(103L));
}
Set<TableName> tables1 = new TreeSet<>();
@ -301,22 +301,22 @@ public class TestBackupSystemTable {
Map<String, Long> rstm = result.get(t);
assertNotNull(rstm);
if (t.equals(TableName.valueOf("t3")) == false) {
assertEquals(rstm.get("rs1:100"), new Long(100L));
assertEquals(rstm.get("rs2:100"), new Long(101L));
assertEquals(rstm.get("rs3:100"), new Long(103L));
assertEquals(rstm.get("rs1:100"), Long.valueOf(100L));
assertEquals(rstm.get("rs2:100"), Long.valueOf(101L));
assertEquals(rstm.get("rs3:100"), Long.valueOf(103L));
} else {
assertEquals(rstm.get("rs1:100"), new Long(200L));
assertEquals(rstm.get("rs2:100"), new Long(201L));
assertEquals(rstm.get("rs3:100"), new Long(203L));
assertEquals(rstm.get("rs1:100"), Long.valueOf(200L));
assertEquals(rstm.get("rs2:100"), Long.valueOf(201L));
assertEquals(rstm.get("rs3:100"), Long.valueOf(203L));
}
}
for (TableName t : tables1) {
Map<String, Long> rstm = result.get(t);
assertNotNull(rstm);
assertEquals(rstm.get("rs1:100"), new Long(200L));
assertEquals(rstm.get("rs2:100"), new Long(201L));
assertEquals(rstm.get("rs3:100"), new Long(203L));
assertEquals(rstm.get("rs1:100"), Long.valueOf(200L));
assertEquals(rstm.get("rs2:100"), Long.valueOf(201L));
assertEquals(rstm.get("rs3:100"), Long.valueOf(203L));
}
cleanBackupTable();
@ -485,15 +485,12 @@ public class TestBackupSystemTable {
return ctxt;
}
private List<BackupInfo> createBackupInfoList(int size) {
private List<BackupInfo> createBackupInfoList(int size) throws InterruptedException {
List<BackupInfo> list = new ArrayList<>();
for (int i = 0; i < size; i++) {
list.add(createBackupInfo());
try {
Thread.sleep(10);
} catch (InterruptedException e) {
e.printStackTrace();
}
// XXX Why do we need this sleep?
Thread.sleep(10);
}
return list;
}

View File

@ -114,8 +114,6 @@ public class TestIncrementalBackupWithBulkLoad extends TestBackupBase {
assertTrue(checkSucceeded(backupIdIncMultiple1));
// Delete all data in table1
TEST_UTIL.deleteTableData(table1);
// #5.1 - check tables for full restore */
Admin hAdmin = TEST_UTIL.getAdmin();
// #6 - restore incremental backup for table1
TableName[] tablesRestoreIncMultiple = new TableName[] { table1 };