aokolnychyi commented on code in PR #9611: URL: https://github.com/apache/iceberg/pull/9611#discussion_r1561909157
########## spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java: ########## @@ -359,16 +361,30 @@ private Result doExecuteWithPartialProgress( // stop commit service commitService.close(); - List<RewriteFileGroup> commitResults = commitService.results(); - if (commitResults.size() == 0) { - LOG.error( - "{} is true but no rewrite commits succeeded. Check the logs to determine why the individual " + + int failedCommits = maxCommits - commitService.succeededCommits(); + if (failedCommits > 0 && failedCommits <= maxFailedCommits) { + LOG.warn( + "{} is true but {} rewrite commits failed. Check the logs to determine why the individual " + "commits failed. If this is persistent it may help to increase {} which will break the rewrite operation " + "into smaller commits.", PARTIAL_PROGRESS_ENABLED, + failedCommits, PARTIAL_PROGRESS_MAX_COMMITS); + } else if (failedCommits > maxFailedCommits) { + String errorMessage = + String.format( + "%s is true but %d rewrite commits failed. This is more than the maximum allowed failures of %d. " + + "Check the logs to determine why the individual commits failed. If this is persistent it may help to " + + "increase %s which will break the rewrite operation into smaller commits.", + PARTIAL_PROGRESS_ENABLED, + failedCommits, + maxFailedCommits, + PARTIAL_PROGRESS_MAX_COMMITS); + throw new RuntimeException(errorMessage); } + List<RewriteFileGroup> commitResults = commitService.results(); Review Comment: Optional: What about an extra method to hide the transformation? ``` return ImmutableRewriteDataFiles.Result.builder() .rewriteResults(toRewriteResults(commitService.results())) .rewriteFailures(rewriteFailures) .build(); ... private Iterable<FileGroupRewriteResult> toRewriteResults(List<RewriteFileGroup> commitResults) { return commitResults.stream().map(RewriteFileGroup::asResult).collect(Collectors.toList()); } ``` -- 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: issues-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org