wypoon commented on code in PR #12106: URL: https://github.com/apache/iceberg/pull/12106#discussion_r1932747579
########## spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveMissingFilesSparkAction.java: ########## @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.actions; + +import java.util.List; +import java.util.stream.Collectors; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.Partitioning; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.ImmutableRemoveMissingFiles; +import org.apache.iceberg.actions.RemoveMissingFiles; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.JobGroupInfo; +import org.apache.iceberg.spark.SparkDataFile; +import org.apache.iceberg.spark.SparkDeleteFile; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.types.StructType; + +public class RemoveMissingFilesSparkAction + extends BaseSnapshotUpdateSparkAction<RemoveMissingFilesSparkAction> + implements RemoveMissingFiles { + + private final Table table; + + RemoveMissingFilesSparkAction(SparkSession spark, Table table) { + super(spark); + this.table = table; + } + + @Override + protected RemoveMissingFilesSparkAction self() { + return this; + } + + @Override + public RemoveMissingFiles.Result execute() { + String jobDesc = String.format("Removing missing files from %s", table.name()); + JobGroupInfo info = newJobGroupInfo("REMOVE-MISSING-FILES", jobDesc); + return withJobGroupInfo(info, this::doExecute); + } + + private RemoveMissingFiles.Result doExecute() { + org.apache.iceberg.RemoveMissingFiles rmf = table.newRemoveFiles(); + + Dataset<Row> entries = loadMetadataTable(table, MetadataTableType.ENTRIES); + Dataset<Row> dataEntries = + entries.filter("data_file.content = 0 AND status < 2").select("data_file.*"); + Dataset<Row> deleteEntries = + entries.filter("data_file.content != 0 AND status < 2").select("data_file.*"); + + List<DataFile> dataFiles = + dataEntries.collectAsList().stream() + .map(row -> dataFileWrapper(dataEntries.schema(), row)) + .collect(Collectors.toList()); + List<DeleteFile> deleteFiles = + deleteEntries.collectAsList().stream() + .map(row -> deleteFileWrapper(deleteEntries.schema(), row)) + .collect(Collectors.toList()); + + FileIO fileIO = table.io(); + List<String> removedDataFiles = Lists.newArrayList(); + List<String> removedDeleteFiles = Lists.newArrayList(); + + for (DataFile f : dataFiles) { + if (!fileIO.newInputFile(f.location()).exists()) { Review Comment: I agree that this can be slow. `dataEntries` and `deleteEntries` are `DataFrame`s (`Dataset<Row>`s) computed on executors; ideally, I'd like to filter them to entries for files that exist in storage. I don't recall the exact issue I ran into, but I had tried something earlier along that line and ran into some Iceberg classes not being serializable so I cannot send the task to the executors. So I simply collected everything back to the driver and processed things there. Let me revisit that. Calling the API to remove the missing files has to be done on the driver though. -- 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