RussellSpitzer commented on code in PR #12254: URL: https://github.com/apache/iceberg/pull/12254#discussion_r2110075002
########## spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java: ########## @@ -303,39 +310,89 @@ private Dataset<String> listedFileDS() { List<String> subDirs = Lists.newArrayList(); List<String> matchingFiles = Lists.newArrayList(); - Predicate<FileStatus> predicate = file -> file.getModificationTime() < olderThanTimestamp; PathFilter pathFilter = PartitionAwareHiddenPathFilter.forSpecs(table.specs()); - // list at most MAX_DRIVER_LISTING_DEPTH levels and only dirs that have - // less than MAX_DRIVER_LISTING_DIRECT_SUB_DIRS direct sub dirs on the driver - listDirRecursively( - location, - predicate, - hadoopConf.value(), - MAX_DRIVER_LISTING_DEPTH, - MAX_DRIVER_LISTING_DIRECT_SUB_DIRS, - subDirs, - pathFilter, - matchingFiles); - - JavaRDD<String> matchingFileRDD = sparkContext().parallelize(matchingFiles, 1); - - if (subDirs.isEmpty()) { + if (usePrefixListing) { + Preconditions.checkArgument( + table.io() instanceof SupportsPrefixOperations, + "Table file io should prefix operations when using prefix list."); + + Predicate<org.apache.iceberg.io.FileInfo> predicate = + fileInfo -> fileInfo.createdAtMillis() < olderThanTimestamp; + listDirRecursivelyWithFileIO( + (SupportsPrefixOperations) table.io(), location, predicate, pathFilter, matchingFiles); + + JavaRDD<String> matchingFileRDD = sparkContext().parallelize(matchingFiles, 1); return spark().createDataset(matchingFileRDD.rdd(), Encoders.STRING()); + } else { + Predicate<FileStatus> predicate = file -> file.getModificationTime() < olderThanTimestamp; + // list at most MAX_DRIVER_LISTING_DEPTH levels and only dirs that have + // less than MAX_DRIVER_LISTING_DIRECT_SUB_DIRS direct sub dirs on the driver + listDirRecursivelyWithHadoop( + location, + predicate, + hadoopConf.value(), + MAX_DRIVER_LISTING_DEPTH, + MAX_DRIVER_LISTING_DIRECT_SUB_DIRS, + subDirs, + pathFilter, + matchingFiles); + + JavaRDD<String> matchingFileRDD = sparkContext().parallelize(matchingFiles, 1); + + if (subDirs.isEmpty()) { + return spark().createDataset(matchingFileRDD.rdd(), Encoders.STRING()); + } + + int parallelism = Math.min(subDirs.size(), listingParallelism); + JavaRDD<String> subDirRDD = sparkContext().parallelize(subDirs, parallelism); + + Broadcast<SerializableConfiguration> conf = sparkContext().broadcast(hadoopConf); + ListDirsRecursively listDirs = new ListDirsRecursively(conf, olderThanTimestamp, pathFilter); + JavaRDD<String> matchingLeafFileRDD = subDirRDD.mapPartitions(listDirs); + + JavaRDD<String> completeMatchingFileRDD = matchingFileRDD.union(matchingLeafFileRDD); + return spark().createDataset(completeMatchingFileRDD.rdd(), Encoders.STRING()); + } + } + + private static void listDirRecursivelyWithFileIO( + SupportsPrefixOperations io, + String dir, + Predicate<org.apache.iceberg.io.FileInfo> predicate, + PathFilter pathFilter, + List<String> matchingFiles) { + String listPath = dir; + if (!dir.endsWith("/")) { + listPath = dir + "/"; + } + + Iterable<org.apache.iceberg.io.FileInfo> files = io.listPrefix(listPath); + for (org.apache.iceberg.io.FileInfo file : files) { + Path path = new Path(file.location()); Review Comment: A note to watch this space later since we are implicitly resolving paths here as Hadoop paths which may not be the right thing to do in S3 etc ... This is the same behavior though that the action would have had before so we can maybe do a followup on this if it's an issue. -- 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