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 8a8f65951acf..fdad0d549830 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.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 byte[] toByteArray() throws IOException { 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 String getShortDescription() { 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 String getStatusAndProgressAsString() { } 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 b4d73e134fab..cb01469c8f18 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 @@ protected void init() { 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 @@ private int parseAndRun(String[] args) throws IOException { 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 @@ private int parseAndRun(String[] args) throws IOException { 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 @@ protected void processOptions(CommandLine cmd) { @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 53295401f761..eb1a12c9551f 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 @@ -310,7 +310,7 @@ public void execute() throws IOException { 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 +371,7 @@ private boolean verifyPath(String path) { } } - 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 +1001,14 @@ public void execute() throws IOException { 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 @@ -1081,13 +1081,13 @@ private TableName[] toTableNames(String[] tables) { return arr; } + @SuppressWarnings("StringSplitter") private void processSetAdd(String[] args) throws IOException { if (args == null || args.length != 4) { printUsage(); throw new IOException(INCORRECT_USAGE); } super.execute(); - String setName = args[2]; String[] tables = args[3].split(","); TableName[] tableNames = new TableName[tables.length]; 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 482b2a266db7..4916689a3a1b 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 @@ private void setIncrTimeRanges(Map> incrTimeRanges) } // backup image directory - private String tableBackupDir = null; private BackupImage backupImage; /** @@ -385,7 +384,6 @@ public BackupManifest(BackupInfo backup) { * @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(); @@ -470,7 +468,7 @@ public List getTableList() { * TODO: fix it. Persist the manifest file. * @throws IOException IOException when storing the manifest file. */ - public void store(Configuration conf) throws BackupException { + public void store(Configuration conf) throws IOException { byte[] data = backupImage.toProto().toByteArray(); // write the file, overwrite if already exist Path manifestFilePath = @@ -479,10 +477,7 @@ public void store(Configuration conf) throws BackupException { try (FSDataOutputStream out = manifestFilePath.getFileSystem(conf).create(manifestFilePath, true)) { out.write(data); - } catch (IOException e) { - throw new BackupException(e.getMessage()); } - LOG.info("Manifest file stored to " + manifestFilePath); } @@ -526,7 +521,7 @@ public ArrayList getRestoreDependentList(boolean reverse) { 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 19ddd8141677..9eb371abecb4 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 @@ 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; @@ -237,6 +239,7 @@ private void waitForSystemTable(Admin admin, TableName tableName) throws IOExcep try { Thread.sleep(100); } catch (InterruptedException e) { + throw (IOException) new InterruptedIOException().initCause(e); } if (EnvironmentEdgeManager.currentTime() - startTime > TIMEOUT) { throw new IOException( @@ -574,7 +577,7 @@ public String readBackupStartCode(String backupRoot) throws IOException { if (val.length == 0) { return null; } - return new String(val); + return new String(val, StandardCharsets.UTF_8); } } @@ -1639,7 +1642,7 @@ public String[] getListOfBackupIdsFromDeleteOperation() throws IOException { if (val.length == 0) { return null; } - return new String(val).split(","); + return new String(val, StandardCharsets.UTF_8).split(","); } } @@ -1654,7 +1657,7 @@ public boolean isMergeInProgress() throws IOException { 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 +1723,7 @@ public String[] getListOfBackupIdsFromMergeOperation() throws IOException { if (val.length == 0) { return null; } - return new String(val).split(","); + return new String(val, StandardCharsets.UTF_8).split(","); } } @@ -1736,11 +1739,13 @@ static Scan createScanForOrigBulkLoadedFiles(TableName table) { return scan; } + @SuppressWarnings("StringSplitter") static String getTableNameFromOrigBulkLoadRow(String rowStr) { String[] parts = rowStr.split(BLK_LD_DELIM); return parts[1]; } + @SuppressWarnings("StringSplitter") static String getRegionNameFromOrigBulkLoadRow(String rowStr) { // format is bulk : namespace : table : region : file String[] parts = rowStr.split(BLK_LD_DELIM); 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 0e800ea520b4..211e9f96c89c 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 @@ private void copyBulkLoadedFiles(List activeFiles, List archiveF 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 3c0eafadb82a..9ec2442a3d93 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 @@ private void restoreImages(BackupImage[] images, TableName sTable, TableName tTa 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 0ca5509262fa..2bb2c13e4dd8 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 @@ protected void addManifest(BackupInfo backupInfo, BackupManager backupManager, B * @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 5dca48788855..18dfd0cd01e9 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.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,6 @@ public BackupInfo getBackupInfo() { * @param backupInfo backup info * @param newProgress progress * @param bytesCopied bytes copied - * @throws NoNodeException exception */ static void updateProgress(BackupInfo backupInfo, BackupManager backupManager, int newProgress, long bytesCopied) throws IOException { @@ -361,7 +359,6 @@ private SequenceFile.Writer getWriter(Path pathToListFile) throws IOException { * @param conf The hadoop configuration * @param copyType The backup copy type * @param options Options for customized ExportSnapshot or DistCp - * @throws Exception 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 9a65ed929d7f..a66e2bfd0b74 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,11 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.util.ArrayDeque; import java.util.ArrayList; 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 +257,7 @@ protected void copyFile(FileSystem fs, Path p, Path newPath) throws IOException */ protected Path convertToDest(Path p, Path backupDirPath) { String backupId = backupDirPath.getName(); - Stack stack = new Stack(); + ArrayDeque 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 4802e8b3ad63..5b21feeba75f 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 LogRollBackupSubprocedure(RegionServerServices rss, ProcedureMember membe 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 4b4ebd361a64..24cf189b124b 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 @@ -68,7 +68,7 @@ */ @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 +136,10 @@ public static void copyTableRegionInfo(Connection conn, BackupInfo backupInfo, C // 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."); @@ -275,12 +276,12 @@ public static List getWALFilesOlderThan(final Configuration c, return logFiles; } + @SuppressWarnings("StringSplitter") public static TableName[] parseTableNames(String tables) { 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]); @@ -593,6 +594,7 @@ public int compare(BackupInfo o1, BackupInfo o2) { return ts1 < ts2 ? 1 : -1; } + @SuppressWarnings("StringSplitter") private long getTimestamp(String backupId) { String[] split = backupId.split("_"); return Long.parseLong(split[1]); @@ -731,6 +733,7 @@ public static BulkLoadHFiles createLoader(Configuration config) { return BulkLoadHFiles.create(conf); } + @SuppressWarnings("StringSplitter") public static String findMostRecentBackupId(String[] backupIds) { long recentTimestamp = Long.MIN_VALUE; for (String backupId : backupIds) { 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 e660ec7b157e..bf2aa14046db 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 @@ private void createAndRestoreTable(Connection conn, TableName tableName, TableNa + ", 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 @@ private void createAndRestoreTable(Connection conn, TableName tableName, TableNa // 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 @@ byte[][] generateBoundaryKeys(ArrayList regionDirList) throws IOException * 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 9246c74172fe..7b5095a897e2 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 @@ private static void populateFromMasterConfig(Configuration masterConf, Configura } } - /** - * @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 307440a10ed7..b71e084e8404 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 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 static void setUpBeforeClass() throws Exception { 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 void testGetDeletableFiles() throws IOException { 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 void testGetDeletableFiles() throws IOException { 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 db9d63bca943..cda738585044 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 @@ -112,7 +112,7 @@ public void testWriteReadBackupStartCode() throws IOException { Long code = 100L; table.writeBackupStartCode(code, "root"); String readCode = table.readBackupStartCode("root"); - assertEquals(code, new Long(Long.parseLong(readCode))); + assertEquals(code, Long.valueOf(readCode)); cleanBackupTable(); } @@ -126,7 +126,7 @@ private void cleanBackupTable() throws IOException { } @Test - public void testBackupHistory() throws IOException { + public void testBackupHistory() throws Exception { int n = 10; List list = createBackupInfoList(n); @@ -153,7 +153,7 @@ public void testBackupHistory() throws IOException { } @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 void testIncrementalBackupTableSet() throws IOException { 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()); - } - - 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()); + 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()); + } } - cleanBackupTable(); + cleanBackupTable(); } @Test @@ -274,9 +274,9 @@ public void testRegionServerLogTimestampMap() throws IOException { 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 void testRegionServerLogTimestampMap() throws IOException { 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 @@ private BackupInfo createBackupInfo() { 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(100); } 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 28fb025bb32a..a182144a8abd 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 void TestIncBackupDeleteTable() throws Exception { 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 };