Apache9 commented on a change in pull request #513: HBASE-22867 The
ForkJoinPool in CleanerChore will spawn thousands of threads in our cluster
with thousands table
URL: https://github.com/apache/hbase/pull/513#discussion_r315609444
##########
File path:
hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
##########
@@ -423,16 +426,18 @@ protected Boolean compute() {
allFilesDeleted = deleteAction(() -> checkAndDeleteFiles(files),
"files");
}
+ List<CompletableFuture<Boolean>> futures = new ArrayList<>();
boolean allSubdirsDeleted = true;
if (!subDirs.isEmpty()) {
- List<CleanerTask> tasks =
Lists.newArrayListWithCapacity(subDirs.size());
sortByConsumedSpace(subDirs);
- for (FileStatus subdir : subDirs) {
- CleanerTask task = new CleanerTask(subdir, false);
- tasks.add(task);
- task.fork();
- }
- allSubdirsDeleted = deleteAction(() -> getCleanResult(tasks),
"subdirs");
+ subDirs.forEach(dir -> futures.add(pool.execute(new CleanerTask(dir,
false))));
Review comment:
I think submitting the task to the same pool is not safe here. In
ForkJoinPool, it is safe to call join as in the join method, the current thread
will try to execute tasks if there are any, so in general, we can always make
progress and finally finish all the tasks. But here, the current thread will be
blocked on the future.get, so it is possible that all the threads in the pool
are blocked on a future.get and cause dead lock here.
----------------------------------------------------------------
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.
For queries about this service, please contact Infrastructure at:
[email protected]
With regards,
Apache Git Services