This is an automated email from the ASF dual-hosted git repository.

russellspitzer pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/iceberg.git


The following commit(s) were added to refs/heads/master by this push:
     new 5028e4a928 Spark: Extend Timeout During Partial Progress Rewrites 
(#6378)
5028e4a928 is described below

commit 5028e4a928fee90f353dc820089761758817c0f0
Author: Russell Spitzer <[email protected]>
AuthorDate: Fri Jan 6 17:07:22 2023 -0500

    Spark: Extend Timeout During Partial Progress Rewrites (#6378)
    
    In order to avoid timing out when writing large manifest files, we increase
    the timeout allowed for the commit phase of partial progress based rewrites
---
 .../actions/RewriteDataFilesCommitManager.java     | 27 +++++++++++++++++-----
 1 file changed, 21 insertions(+), 6 deletions(-)

diff --git 
a/core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitManager.java
 
b/core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitManager.java
index f6fc53bba1..416786d79b 100644
--- 
a/core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitManager.java
+++ 
b/core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitManager.java
@@ -222,21 +222,23 @@ public class RewriteDataFilesCommitManager {
     public void close() {
       Preconditions.checkState(
           running.compareAndSet(true, false), "Cannot close already closed 
RewriteService");
-      LOG.info("Closing commit service for {}", table);
+      LOG.info("Closing commit service for {} waiting for all commits to 
finish", table);
       committerService.shutdown();
 
+      boolean timeout = false;
       try {
         // All rewrites have completed and all new files have been created, we 
are now waiting for
         // the commit
-        // pool to finish doing it's commits to Iceberg State. In the case of 
partial progress this
+        // pool to finish doing its commits to Iceberg State. In the case of 
partial progress this
         // should
         // have been occurring simultaneously with rewrites, if not there 
should be only a single
         // commit operation.
-        // In either case this should take much less than 10 minutes to 
actually complete.
-        if (!committerService.awaitTermination(10, TimeUnit.MINUTES)) {
+        if (!committerService.awaitTermination(120, TimeUnit.MINUTES)) {
           LOG.warn(
-              "Commit operation did not complete within 10 minutes of the 
files being written. This may mean "
-                  + "that changes were not successfully committed to the the 
Iceberg table.");
+              "Commit operation did not complete within 120 minutes of the all 
files "
+                  + "being rewritten. This may mean that some changes were not 
successfully committed to the "
+                  + "table.");
+          timeout = true;
         }
       } catch (InterruptedException e) {
         Thread.currentThread().interrupt();
@@ -244,6 +246,19 @@ public class RewriteDataFilesCommitManager {
             "Cannot complete commit for rewrite, commit service interrupted", 
e);
       }
 
+      if (!completedRewrites.isEmpty() && timeout) {
+        LOG.error("Attempting to cleanup uncommitted file groups");
+        
completedRewrites.forEach(RewriteDataFilesCommitManager.this::abortFileGroup);
+      }
+
+      Preconditions.checkArgument(
+          !timeout && completedRewrites.isEmpty(),
+          "Timeout occurred when waiting for commits to complete. "
+              + "{} file groups committed. {} file groups remain uncommitted. "
+              + "Retry this operation to attempt rewriting the failed groups.",
+          committedRewrites.size(),
+          completedRewrites.size());
+
       Preconditions.checkState(
           completedRewrites.isEmpty(),
           "File groups offered after service was closed, "

Reply via email to