[ 
https://issues.apache.org/jira/browse/MAPREDUCE-7474?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17838326#comment-17838326
 ] 

ASF GitHub Bot commented on MAPREDUCE-7474:
-------------------------------------------

steveloughran commented on code in PR #6716:
URL: https://github.com/apache/hadoop/pull/6716#discussion_r1569228824


##########
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/AbstractJobOrTaskStage.java:
##########
@@ -582,19 +605,46 @@ protected final Path directoryMustExist(
    * Save a task manifest or summary. This will be done by
    * writing to a temp path and then renaming.
    * If the destination path exists: Delete it.
+   * This will retry so that a rename failure from abfs load or IO errors
+   * will not fail the task.
    * @param manifestData the manifest/success file
    * @param tempPath temp path for the initial save
    * @param finalPath final path for rename.
-   * @throws IOException failure to load/parse
+   * @throws IOException failure to rename after retries.
    */
   @SuppressWarnings("unchecked")
   protected final <T extends AbstractManifestData> void save(T manifestData,
       final Path tempPath,
       final Path finalPath) throws IOException {
-    LOG.trace("{}: save('{}, {}, {}')", getName(), manifestData, tempPath, 
finalPath);
-    trackDurationOfInvocation(getIOStatistics(), OP_SAVE_TASK_MANIFEST, () ->
-        operations.save(manifestData, tempPath, true));
-    renameFile(tempPath, finalPath);
+    boolean success = false;
+    int failures = 0;
+    while (!success) {
+      try {
+        LOG.trace("{}: attempt {} save('{}, {}, {}')",
+            getName(), failures, manifestData, tempPath, finalPath);
+
+        trackDurationOfInvocation(getIOStatistics(), OP_SAVE_TASK_MANIFEST, () 
->
+            operations.save(manifestData, tempPath, true));
+        renameFile(tempPath, finalPath);

Review Comment:
   any error raised during rename triggers fallback of
   * catch IOE
   * save temp file again
   * delete dest path
   * rename temp path to final path
   
   this is attempted a configurable number of times, with a sleep in between.
   no attempt to be clever about which IOEs are unrecoverable (permissions 
etc), just catch, log, retry





> [ABFS] Improve commit resilience and performance in Manifest Committer
> ----------------------------------------------------------------------
>
>                 Key: MAPREDUCE-7474
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-7474
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>          Components: client
>    Affects Versions: 3.4.0, 3.3.6
>            Reporter: Steve Loughran
>            Assignee: Steve Loughran
>            Priority: Major
>              Labels: pull-request-available
>
> * Manifest committer is not resilient to rename failures on task commit 
> without HADOOP-18012 rename recovery enabled. 
> * large burst of delete calls noted: are they needed?
> relates to HADOOP-19093 but takes a more minimal approach with goal of 
> changes in manifest committer only.
> Initial proposed changes
> * retry recovery on task commit rename, always (repeat save, delete, rename)
> * audit delete use and see if it can be pruned



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: mapreduce-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: mapreduce-issues-h...@hadoop.apache.org

Reply via email to