steveloughran commented on code in PR #6716: URL: https://github.com/apache/hadoop/pull/6716#discussion_r1569270966
########## hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CleanupJobStage.java: ########## @@ -142,64 +154,93 @@ protected Result executeStage( } Outcome outcome = null; - IOException exception; + IOException exception = null; + boolean baseDirDeleted = false; // to delete. LOG.info("{}: Deleting job directory {}", getName(), baseDir); if (args.deleteTaskAttemptDirsInParallel) { - // Attempt to do a parallel delete of task attempt dirs; - // don't overreact if a delete fails, but stop trying - // to delete the others, and fall back to deleting the - // job dir. - Path taskSubDir - = getStageConfig().getJobAttemptTaskSubDir(); - try (DurationInfo info = new DurationInfo(LOG, - "parallel deletion of task attempts in %s", - taskSubDir)) { - RemoteIterator<FileStatus> dirs = - RemoteIterators.filteringRemoteIterator( - listStatusIterator(taskSubDir), - FileStatus::isDirectory); - TaskPool.foreach(dirs) - .executeWith(getIOProcessors()) - .stopOnFailure() - .suppressExceptions(false) - .run(this::rmTaskAttemptDir); - getIOStatistics().aggregate((retrieveIOStatistics(dirs))); - - if (getLastDeleteException() != null) { - // one of the task attempts failed. - throw getLastDeleteException(); + + // parallel delete of task attempt dirs. + + if (args.parallelDeleteAttemptBaseDeleteFirst) { + // attempt to delete the base dir first. + // This can reduce ABFS delete load but may time out + // (which the fallback to parallel delete will handle). + // on GCS it is slow. + try (DurationInfo info = new DurationInfo(LOG, true, + "Initial delete of %s", baseDir)) { + exception = deleteOneDir(baseDir); + if (exception == null) { + // success: record this as the outcome, which + // will skip the parallel delete. + outcome = Outcome.DELETED; + baseDirDeleted = true; + } else { + // failure: log and continue + LOG.warn("{}: Exception on initial attempt at deleting base dir {}\n" + + "attempting parallel delete", + getName(), baseDir, exception); + } + } + } + if (!baseDirDeleted) { + // no base delete attempted or it failed. + // Attempt to do a parallel delete of task attempt dirs; + // don't overreact if a delete fails, but stop trying + // to delete the others, and fall back to deleting the + // job dir. + Path taskSubDir + = getStageConfig().getJobAttemptTaskSubDir(); + try (DurationInfo info = new DurationInfo(LOG, true, + "parallel deletion of task attempts in %s", + taskSubDir)) { + RemoteIterator<FileStatus> dirs = + RemoteIterators.filteringRemoteIterator( + listStatusIterator(taskSubDir), + FileStatus::isDirectory); + TaskPool.foreach(dirs) + .executeWith(getIOProcessors()) + .stopOnFailure() + .suppressExceptions(false) + .run(this::rmTaskAttemptDir); + getIOStatistics().aggregate((retrieveIOStatistics(dirs))); + + if (getLastDeleteException() != null) { + // one of the task attempts failed. + throw getLastDeleteException(); Review Comment: good q. rmTaskAttemptDir will not trigger a failure, just note and store the error. on L213 one of any such IOEs is rethrown, but that just gets to L222 which logs and then goes on to the final root dir attempt again so it should be recovered from...though if the first delete timed out then that root delete on L233 is probably doomed too... -- 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: common-issues-unsubscr...@hadoop.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org