From 2c3abae18aa35e2693b64b143316817d4569d0c3 Mon Sep 17 00:00:00 2001 From: Andrew Purtell Date: Thu, 11 Aug 2022 00:02:17 -0700 Subject: [PATCH] HBASE-27201 Clean up error-prone findings in hbase-backup (#4643) Signed-off-by: Duo Zhang Signed-off-by: Viraj Jasani --- .../hadoop/hbase/backup/BackupInfo.java | 10 +-- .../hadoop/hbase/backup/RestoreDriver.java | 9 +-- .../hbase/backup/impl/BackupCommands.java | 18 ++--- .../hbase/backup/impl/BackupManifest.java | 6 +- .../hbase/backup/impl/BackupSystemTable.java | 31 ++++--- .../impl/IncrementalTableBackupClient.java | 3 +- .../backup/impl/RestoreTablesClient.java | 2 +- .../hbase/backup/impl/TableBackupClient.java | 2 +- .../mapreduce/MapReduceBackupCopyJob.java | 5 +- .../mapreduce/MapReduceBackupMergeJob.java | 5 +- .../LogRollBackupSubprocedure.java | 3 +- .../hadoop/hbase/backup/util/BackupUtils.java | 26 +++--- .../hadoop/hbase/backup/util/RestoreTool.java | 14 ++-- .../hadoop/hbase/backup/TestBackupBase.java | 3 - .../hbase/backup/TestBackupHFileCleaner.java | 20 ++--- .../hbase/backup/TestBackupSystemTable.java | 81 +++++++++---------- .../TestIncrementalBackupWithBulkLoad.java | 2 - 17 files changed, 111 insertions(+), 129 deletions(-) diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java index 8a8f65951ac..fdad0d54983 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java @@ -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 { return toProtosBackupInfo().toByteArray(); } - private void setBackupTableInfoMap(Builder builder) { + private void setBackupTableInfoMap(BackupProtos.BackupInfo.Builder builder) { for (Entry 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> entry : this.getTableSetTimestampMap().entrySet()) { builder.putTableSetTimestamp(entry.getKey().getNameAsString(), @@ -531,10 +530,9 @@ public class BackupInfo implements Comparable { 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 { } public String getTableListAsString() { - StringBuffer sb = new StringBuffer(); + StringBuilder sb = new StringBuilder(); sb.append("{"); sb.append(StringUtils.join(backupTableInfoMap.keySet(), ",")); sb.append("}"); diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java index b4d73e134fa..cb01469c8f1 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java @@ -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 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 { diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java index 53295401f76..ce9c5bbe8fa 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java @@ -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 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 { diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java index 482b2a266db..3a1cbd55c58 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java @@ -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 tables = new ArrayList(); 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()); } /** diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java index 19ddd814167..04f43b5b0ea 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java @@ -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>[] readBulkLoadedFiles(String backupId, List sTableList) throws IOException { Scan scan = BackupSystemTable.createScanForBulkLoadedFiles(backupId); + @SuppressWarnings("unchecked") Map>[] 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 parts = Splitter.onPattern(BLK_LD_DELIM).splitToList(rowStr); + Iterator 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; } /* diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java index 0e800ea520b..211e9f96c89 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java @@ -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, diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java index 3c0eafadb82..9ec2442a3d9 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java @@ -181,7 +181,7 @@ public class RestoreTablesClient { private List 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 list = new ArrayList<>(); RemoteIterator it = fs.listFiles(new Path(fileBackupDir), true); while (it.hasNext()) { diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java index 0ca5509262f..2bb2c13e4dd 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java @@ -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 + ";"); diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyJob.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyJob.java index 5dca4878885..51a276df4c5 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyJob.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyJob.java @@ -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, diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupMergeJob.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupMergeJob.java index 9a65ed929d7..3b4cf0246d7 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupMergeJob.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupMergeJob.java @@ -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 stack = new Stack(); + Deque stack = new ArrayDeque(); String name = null; while (true) { name = p.getName(); diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java index 4802e8b3ad6..5b21feeba75 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java @@ -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); } } diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java index 4b4ebd361a6..ef97b195e28 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java @@ -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; } diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java index e660ec7b157..bf2aa14046d 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java @@ -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 regionDirList, TableDescriptor htd, - boolean truncateIfExists) throws IOException { + private void checkAndCreateTable(Connection conn, TableName targetTableName, + ArrayList regionDirList, TableDescriptor htd, boolean truncateIfExists) + throws IOException { try (Admin admin = conn.getAdmin()) { boolean createNew = false; if (admin.tableExists(targetTableName)) { diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java index 9246c74172f..7b5095a897e 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java @@ -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 { diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupHFileCleaner.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupHFileCleaner.java index 307440a10ed..b71e084e840 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupHFileCleaner.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupHFileCleaner.java @@ -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 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 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 sTableList = new ArrayList<>(); sTableList.add(tableName); - Map>[] maps = new Map[1]; - maps[0] = new HashMap<>(); + @SuppressWarnings("unchecked") + IdentityHashMap>[] 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)) { diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java index db9d63bca94..21883fa6eaa 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java @@ -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 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 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 res1 = (TreeSet) table.getIncrementalBackupTableSet("root"); - assertTrue(tables1.size() == res1.size()); - Iterator desc1 = tables1.descendingIterator(); - Iterator desc2 = res1.descendingIterator(); - while (desc1.hasNext()) { - assertEquals(desc1.next(), desc2.next()); + + try (BackupSystemTable systemTable = new BackupSystemTable(conn)) { + TreeSet res1 = + (TreeSet) systemTable.getIncrementalBackupTableSet("root"); + assertTrue(tables1.size() == res1.size()); + Iterator desc1 = tables1.descendingIterator(); + Iterator desc2 = res1.descendingIterator(); + while (desc1.hasNext()) { + assertEquals(desc1.next(), desc2.next()); + } + systemTable.addIncrementalBackupTableSet(tables2, "root"); + TreeSet res2 = + (TreeSet) 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 res2 = (TreeSet) 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 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 tables1 = new TreeSet<>(); @@ -301,22 +301,22 @@ public class TestBackupSystemTable { Map 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 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 createBackupInfoList(int size) { + private List createBackupInfoList(int size) throws InterruptedException { List 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; } diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java index 28fb025bb32..a182144a8ab 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java @@ -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 };