stoty commented on code in PR #6616:
URL: https://github.com/apache/hbase/pull/6616#discussion_r1930422082


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java:
##########
@@ -419,52 +423,97 @@ public static void archiveStoreFile(Configuration conf, 
FileSystem fs, RegionInf
    * @return the list of failed to archive files.
    * @throws IOException if an unexpected file operation exception occurred
    */
-  private static List<File> resolveAndArchive(FileSystem fs, Path 
baseArchiveDir,
-    Collection<File> toArchive, long start) throws IOException {
-    // short circuit if no files to move
+  private static List<File> resolveAndArchive(Configuration conf, FileSystem 
fs,
+    Path baseArchiveDir, Collection<File> toArchive, long start) throws 
IOException {
+    // 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(conf, fs, baseArchiveDir, failures, file, start);
+      }
+    }
+
+    // Archive files concurrently
+    archiveFilesConcurrently(conf, 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(Configuration conf, 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(conf, fs, subArchiveDir, children, 
start));
+    } catch (IOException e) {
+      LOG.warn("Failed to archive directory: {}", directory, e);
+      failures.add(directory);
+    }
+  }
+
+  private static void archiveFilesConcurrently(Configuration conf, Path 
baseArchiveDir,
+    List<File> files, Queue<File> failures, String startTime) {
+    LOG.trace("Archiving {} files concurrently into directory: {}", 
files.size(), baseArchiveDir);
+    Map<File, Future<Boolean>> futureMap = new HashMap<>();
+    // Submit file archiving tasks
+    // default is 16 which comes equal hbase.hstore.blockingStoreFiles default 
value
+    int maxThreads = 
conf.getInt("hbase.hfilearchiver.per.region.thread.pool.max", 16);

Review Comment:
   And the theoretical total number of threads is multiplied by the number of 
region servers, IIRC.
   
   Realistically (without introducing some new RPC mechanism), the only way to 
have a total limit on the number of delete threads would be running the chore 
on Master.
   
   Looking at the RS level, wouldn't it make more sense to us a common delete 
thread pool in  HFileArchiver and use that for all directories ? 
   
   That way, if the the limit is 128, and a few directories have a lot of files 
to archive, and lot of them with none, then we can keep all the threads loaded, 
and the maximum thread count doesn't depend on the number of regions that have 
files to archives.
   
   If the directory listing is fast (is it ?), then we could do also the 
directory listing sequentially, and use a single thread pool for the file 
operations.
   



-- 
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]

Reply via email to