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:
parent
5eaeff5fcd
commit
2c3abae18a
|
@ -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("}");
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
/*
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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()) {
|
||||
|
|
|
@ -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 + ";");
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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)) {
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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)) {
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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 };
|
||||
|
|
Loading…
Reference in New Issue