RussellSpitzer commented on a change in pull request #4451:
URL: https://github.com/apache/iceberg/pull/4451#discussion_r841087546
##########
File path:
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/actions/BaseDeleteOrphanFilesSparkAction.java
##########
@@ -172,18 +173,34 @@ private String jobDesc() {
Column nameEqual = actualFileName.equalTo(validFileName);
Column actualContains =
actualFileDF.col("file_path").contains(validFileDF.col("file_path"));
Column joinCond = nameEqual.and(actualContains);
- List<String> orphanFiles = actualFileDF.join(validFileDF, joinCond,
"leftanti")
- .as(Encoders.STRING())
- .collectAsList();
-
- Tasks.foreach(orphanFiles)
- .noRetry()
- .executeWith(deleteExecutorService)
- .suppressFailureWhenFinished()
- .onFailure((file, exc) -> LOG.warn("Failed to delete file: {}", file,
exc))
- .run(deleteFunc::accept);
-
- return new BaseDeleteOrphanFilesActionResult(orphanFiles);
+ Dataset<String> orphanDF = actualFileDF.join(validFileDF, joinCond,
"leftanti")
+ .as(Encoders.STRING());
+ return deleteFiles(orphanDF);
+ }
+
+ private BaseDeleteOrphanFilesActionResult deleteFiles(Dataset<String>
orphanDF) {
+ boolean streamResults = PropertyUtil.propertyAsBoolean(options(),
STREAM_RESULTS, false);
+ Iterable<String> itr;
+ if (streamResults) {
+ // cache df to avoid recomputing
+ orphanDF.cache();
Review comment:
Does this prevent redoing the shuffle? I thought (I could be wrong about
this) that we would basically leave the job in the state where the join had
written all shuffle files, then spark issues a new job for each reducer in the
iterator. I'm not pro or con this move, although we probably will need to
document this.
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]