stoty commented on code in PR #6616:
URL: https://github.com/apache/hbase/pull/6616#discussion_r1925484298
##########
hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java:
##########
@@ -421,50 +427,94 @@ public static void archiveStoreFile(Configuration conf,
FileSystem fs, RegionInf
*/
private static List<File> resolveAndArchive(FileSystem fs, Path
baseArchiveDir,
Collection<File> toArchive, long start) throws IOException {
- // short circuit if no files to move
+ // Early exit if no files to archive
if (toArchive.isEmpty()) {
+ LOG.trace("No files to archive, returning an empty list.");
return Collections.emptyList();
}
- LOG.trace("Moving files to the archive directory {}", baseArchiveDir);
+ LOG.trace("Preparing to archive files into directory: {}", baseArchiveDir);
- // make sure the archive directory exists
- if (!fs.exists(baseArchiveDir)) {
- if (!fs.mkdirs(baseArchiveDir)) {
- throw new IOException("Failed to create the archive directory:" +
baseArchiveDir
- + ", quitting archive attempt.");
- }
- LOG.trace("Created archive directory {}", baseArchiveDir);
- }
+ // Ensure the archive directory exists
+ ensureArchiveDirectoryExists(fs, baseArchiveDir);
- List<File> failures = new ArrayList<>();
+ // Thread-safe collection for storing failures
+ Queue<File> failures = new ConcurrentLinkedQueue<>();
String startTime = Long.toString(start);
+
+ // Separate files and directories for processing
+ List<File> filesOnly = new ArrayList<>();
for (File file : toArchive) {
- // if its a file archive it
- try {
- LOG.trace("Archiving {}", file);
- if (file.isFile()) {
- // attempt to archive the file
- if (!resolveAndArchiveFile(baseArchiveDir, file, startTime)) {
- LOG.warn("Couldn't archive " + file + " into backup directory: " +
baseArchiveDir);
+ if (file.isFile()) {
+ filesOnly.add(file);
+ } else {
+ handleDirectory(fs, baseArchiveDir, failures, file, start);
+ }
+ }
+
+ // Archive files concurrently
+ archiveFilesConcurrently(baseArchiveDir, filesOnly, failures, startTime);
+
+ return new ArrayList<>(failures); // Convert to a List for the return value
+ }
+
+ private static void ensureArchiveDirectoryExists(FileSystem fs, Path
baseArchiveDir)
+ throws IOException {
+ if (!fs.exists(baseArchiveDir) && !fs.mkdirs(baseArchiveDir)) {
+ throw new IOException("Failed to create the archive directory: " +
baseArchiveDir);
+ }
+ LOG.trace("Archive directory ready: {}", baseArchiveDir);
+ }
+
+ private static void handleDirectory(FileSystem fs, Path baseArchiveDir,
Queue<File> failures,
+ File directory, long start) {
+ LOG.trace("Processing directory: {}, archiving its children.", directory);
+ Path subArchiveDir = new Path(baseArchiveDir, directory.getName());
+
+ try {
+ Collection<File> children = directory.getChildren();
+ failures.addAll(resolveAndArchive(fs, subArchiveDir, children, start));
+ } catch (IOException e) {
+ LOG.warn("Failed to archive directory: {}", directory, e);
+ failures.add(directory);
+ }
+ }
+
+ private static void archiveFilesConcurrently(Path baseArchiveDir, List<File>
files,
+ Queue<File> failures, String startTime) {
+ LOG.trace("Archiving {} files concurrently into directory: {}",
files.size(), baseArchiveDir);
+
+ ExecutorService executorService = Executors.newCachedThreadPool();
Review Comment:
TBH I don't have a lot of experience with object storage deletion
performance.
Are resolveAndArchive calls serial, or is it possible to have multiple
invocations running at the same time ?
What do you think @wchevreuil, @BukrosSzabolcs ?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]