aokolnychyi commented on a change in pull request #2415: URL: https://github.com/apache/iceberg/pull/2415#discussion_r639315720
########## File path: api/src/main/java/org/apache/iceberg/actions/RemoveFiles.java ########## @@ -0,0 +1,77 @@ +/* + * 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.actions; + +import java.util.concurrent.ExecutorService; +import java.util.function.Consumer; + +/** + * An action that deletes data, manifest, manifest lists in a table. Review comment: I think we should add a bit more details to the description of this class. Something like this? ``` /** * An action that removes all files referenced by a table metadata file. * <p> * This action will irreversibly delete all reachable files such as data files, manifests, * manifest lists and should be used to clean up the underlying storage once a table is dropped * and no longer needed. * <p> * Implementations may use a query engine to distribute parts of work. */ ``` ########## File path: api/src/main/java/org/apache/iceberg/actions/RemoveFiles.java ########## @@ -0,0 +1,77 @@ +/* + * 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.actions; + +import java.util.concurrent.ExecutorService; +import java.util.function.Consumer; + +/** + * An action that deletes data, manifest, manifest lists in a table. + * <p> + * Implementations may use a query engine to distribute parts of work. + */ +public interface RemoveFiles extends Action<RemoveFiles, RemoveFiles.Result> { + + /** + * Passes an alternative delete implementation that will be used for manifests and data files. + * + * @param deleteFunc a function that will be called to delete manifests and data files. Review comment: nit: `... to delete files` ########## File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRemoveFilesSparkAction.java ########## @@ -0,0 +1,194 @@ +/* + * 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.Iterator; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.ReachableFileUtil; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.actions.RemoveFiles; +import org.apache.iceberg.actions.RemoveFilesActionResult; +import org.apache.iceberg.exceptions.NotFoundException; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.JobGroupInfo; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.Tasks; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.functions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * An action that performs the same operation as {@link RemoveFiles} but uses Spark + * to determine the files that needs to be deleted. The action uses metadata tables to + * find the files to be deleted. + * Deletes are performed locally after retrieving the results from the Spark executors. + */ +@SuppressWarnings("UnnecessaryAnonymousClass") +public class BaseRemoveFilesSparkAction + extends BaseSparkAction<RemoveFiles, RemoveFiles.Result> implements RemoveFiles { + private static final Logger LOG = LoggerFactory.getLogger(BaseRemoveFilesSparkAction.class); + + private static final String DATA_FILE = "Data File"; + private static final String MANIFEST = "Manifest"; + private static final String MANIFEST_LIST = "Manifest List"; + private static final String OTHERS = "Others"; + + private static final String STREAM_RESULTS = "stream-results"; + + // Creates an executor service that runs each task in the thread that invokes execute/submit. + private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = null; + + private final Table table; + private final TableOperations ops; + private final Consumer<String> defaultDelete = new Consumer<String>() { + @Override + public void accept(String file) { + ops.io().deleteFile(file); + } + }; + + private Consumer<String> deleteFunc = defaultDelete; + private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE; + + public BaseRemoveFilesSparkAction(SparkSession spark, Table table) { + super(spark); + this.table = table; + this.ops = ((HasTableOperations) table).operations(); + } + + @Override + protected RemoveFiles self() { + return this; + } + + @Override + public Result execute() { + JobGroupInfo info = newJobGroupInfo("REMOVE-FILES", + String.format("Removing files from %s", table.location())); + return withJobGroupInfo(info, this::doExecute); + } + + private Result doExecute() { + boolean streamResults = PropertyUtil.propertyAsBoolean(options(), STREAM_RESULTS, false); + Dataset<Row> validFileDataset = buildValidFileDF(ops.current()).distinct(); + RemoveFilesActionResult result; + if (streamResults) { + result = deleteFiles(validFileDataset.toLocalIterator()); + } else { + result = deleteFiles(validFileDataset.collectAsList().iterator()); + } + return result; + } + + private Dataset<Row> appendTypeString(Dataset<Row> ds, String type) { + return ds.select(new Column("file_path"), functions.lit(type).as("file_type")); + } + + private Dataset<Row> buildValidFileDF(TableMetadata metadata) { + Table staticTable = newStaticTable(metadata, this.table.io()); + return appendTypeString(buildValidDataFileDF(staticTable), DATA_FILE) + .union(appendTypeString(buildManifestFileDF(staticTable), MANIFEST)) + .union(appendTypeString(buildManifestListDF(staticTable), MANIFEST_LIST)) + .union(appendTypeString(buildOtherMetadataFileDF(staticTable), OTHERS)); + } + + @Override + protected Dataset<Row> buildOtherMetadataFileDF(Table tbl) { + List<String> otherMetadataFiles = Lists.newArrayList(); + otherMetadataFiles.addAll(ReachableFileUtil.metadataFileLocations(table, true)); + otherMetadataFiles.add(ReachableFileUtil.versionHintLocation(table)); + return spark().createDataset(otherMetadataFiles, Encoders.STRING()).toDF("file_path"); + } + + /** + * Deletes files passed to it. + * + * @param deleted an Iterator of Spark Rows of the structure (path: String, type: String) + * @return Statistics on which files were deleted + */ + private RemoveFilesActionResult deleteFiles(Iterator<Row> deleted) { + AtomicLong dataFileCount = new AtomicLong(0L); + AtomicLong manifestCount = new AtomicLong(0L); + AtomicLong manifestListCount = new AtomicLong(0L); + AtomicLong otherFilesCount = new AtomicLong(0L); + + Tasks.foreach(deleted) + .retry(3).stopRetryOn(NotFoundException.class).suppressFailureWhenFinished() + .executeWith(deleteExecutorService) + .onFailure((fileInfo, exc) -> { + String file = fileInfo.getString(0); + String type = fileInfo.getString(1); + LOG.warn("Delete failed for {}: {}", type, file, exc); + }) + .run(fileInfo -> { + String file = fileInfo.getString(0); + String type = fileInfo.getString(1); + deleteFunc.accept(file); + switch (type) { + case DATA_FILE: + dataFileCount.incrementAndGet(); + LOG.trace("Deleted Data File: {}", file); + break; + case MANIFEST: + manifestCount.incrementAndGet(); + LOG.debug("Deleted Manifest: {}", file); + break; + case MANIFEST_LIST: + manifestListCount.incrementAndGet(); + LOG.debug("Deleted Manifest List: {}", file); + break; + case OTHERS: + otherFilesCount.incrementAndGet(); + LOG.debug("Others: {}", file); + break; + } + }); + + LOG.info("Deleted {} total files", dataFileCount.get() + manifestCount.get() + manifestListCount.get() + Review comment: nit: maybe introduce a var to simplify? ``` long fileCount = ... LOG.info("Deleted {} total files", fileCount); ``` ########## File path: api/src/main/java/org/apache/iceberg/actions/RemoveFiles.java ########## @@ -0,0 +1,77 @@ +/* + * 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.actions; + +import java.util.concurrent.ExecutorService; +import java.util.function.Consumer; + +/** + * An action that deletes data, manifest, manifest lists in a table. + * <p> + * Implementations may use a query engine to distribute parts of work. + */ +public interface RemoveFiles extends Action<RemoveFiles, RemoveFiles.Result> { + + /** + * Passes an alternative delete implementation that will be used for manifests and data files. + * + * @param deleteFunc a function that will be called to delete manifests and data files. + * The function accepts path to file as an argument. + * @return this for method chaining + */ + RemoveFiles deleteWith(Consumer<String> deleteFunc); + + /** + * Passes an alternative executor service that will be used for manifests and data files deletion. + * <p> + * If this method is not called, manifests and data files will still be deleted in Review comment: nit: just `... files ..` ########## File path: api/src/main/java/org/apache/iceberg/actions/RemoveFiles.java ########## @@ -0,0 +1,77 @@ +/* + * 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.actions; + +import java.util.concurrent.ExecutorService; +import java.util.function.Consumer; + +/** + * An action that deletes data, manifest, manifest lists in a table. + * <p> + * Implementations may use a query engine to distribute parts of work. + */ +public interface RemoveFiles extends Action<RemoveFiles, RemoveFiles.Result> { + + /** + * Passes an alternative delete implementation that will be used for manifests and data files. + * + * @param deleteFunc a function that will be called to delete manifests and data files. + * The function accepts path to file as an argument. + * @return this for method chaining + */ + RemoveFiles deleteWith(Consumer<String> deleteFunc); + + /** + * Passes an alternative executor service that will be used for manifests and data files deletion. Review comment: nit: just `... files deletion`? ########## File path: core/src/main/java/org/apache/iceberg/actions/RemoveFilesActionResult.java ########## @@ -0,0 +1,58 @@ +/* + * 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.actions; + +public class RemoveFilesActionResult implements RemoveFiles.Result { + + private final long deletedDataFilesCount; + private final long deletedManifestsCount; + private final long deletedManifestListsCount; + private final long deletedOtherFilesCount; + + public RemoveFilesActionResult(long deletedDataFilesCount, + long deletedManifestsCount, + long deletedManifestListsCount, + long otherDeletedFilesCount) { + this.deletedDataFilesCount = deletedDataFilesCount; Review comment: `deletedXXX` -> `removedXXX` ########## File path: api/src/main/java/org/apache/iceberg/actions/RemoveFiles.java ########## @@ -0,0 +1,77 @@ +/* + * 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.actions; + +import java.util.concurrent.ExecutorService; +import java.util.function.Consumer; + +/** + * An action that deletes data, manifest, manifest lists in a table. + * <p> + * Implementations may use a query engine to distribute parts of work. + */ +public interface RemoveFiles extends Action<RemoveFiles, RemoveFiles.Result> { + + /** + * Passes an alternative delete implementation that will be used for manifests and data files. Review comment: nit: just `... for files`? Unlike `ExpireSnapshotsAction`, this will remove other files too. ########## File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRemoveFilesSparkAction.java ########## @@ -0,0 +1,194 @@ +/* + * 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.Iterator; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.ReachableFileUtil; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.actions.RemoveFiles; +import org.apache.iceberg.actions.RemoveFilesActionResult; +import org.apache.iceberg.exceptions.NotFoundException; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.JobGroupInfo; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.Tasks; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.functions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + Review comment: nit: extra line? ########## File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRemoveFilesSparkAction.java ########## @@ -0,0 +1,194 @@ +/* + * 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.Iterator; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.ReachableFileUtil; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.actions.RemoveFiles; +import org.apache.iceberg.actions.RemoveFilesActionResult; +import org.apache.iceberg.exceptions.NotFoundException; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.JobGroupInfo; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.Tasks; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.functions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * An action that performs the same operation as {@link RemoveFiles} but uses Spark + * to determine the files that needs to be deleted. The action uses metadata tables to + * find the files to be deleted. + * Deletes are performed locally after retrieving the results from the Spark executors. + */ +@SuppressWarnings("UnnecessaryAnonymousClass") +public class BaseRemoveFilesSparkAction + extends BaseSparkAction<RemoveFiles, RemoveFiles.Result> implements RemoveFiles { + private static final Logger LOG = LoggerFactory.getLogger(BaseRemoveFilesSparkAction.class); + + private static final String DATA_FILE = "Data File"; + private static final String MANIFEST = "Manifest"; + private static final String MANIFEST_LIST = "Manifest List"; + private static final String OTHERS = "Others"; + + private static final String STREAM_RESULTS = "stream-results"; Review comment: @RussellSpitzer, do we want to expose such constants? Is that the plan for compaction actions? ########## File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRemoveFilesSparkAction.java ########## @@ -0,0 +1,194 @@ +/* + * 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.Iterator; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.ReachableFileUtil; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.actions.RemoveFiles; +import org.apache.iceberg.actions.RemoveFilesActionResult; +import org.apache.iceberg.exceptions.NotFoundException; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.JobGroupInfo; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.Tasks; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.functions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * An action that performs the same operation as {@link RemoveFiles} but uses Spark + * to determine the files that needs to be deleted. The action uses metadata tables to + * find the files to be deleted. + * Deletes are performed locally after retrieving the results from the Spark executors. + */ +@SuppressWarnings("UnnecessaryAnonymousClass") +public class BaseRemoveFilesSparkAction + extends BaseSparkAction<RemoveFiles, RemoveFiles.Result> implements RemoveFiles { + private static final Logger LOG = LoggerFactory.getLogger(BaseRemoveFilesSparkAction.class); + + private static final String DATA_FILE = "Data File"; + private static final String MANIFEST = "Manifest"; + private static final String MANIFEST_LIST = "Manifest List"; + private static final String OTHERS = "Others"; + + private static final String STREAM_RESULTS = "stream-results"; + + // Creates an executor service that runs each task in the thread that invokes execute/submit. + private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = null; + + private final Table table; + private final TableOperations ops; + private final Consumer<String> defaultDelete = new Consumer<String>() { + @Override + public void accept(String file) { + ops.io().deleteFile(file); + } + }; + + private Consumer<String> deleteFunc = defaultDelete; + private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE; + + public BaseRemoveFilesSparkAction(SparkSession spark, Table table) { + super(spark); + this.table = table; + this.ops = ((HasTableOperations) table).operations(); + } + + @Override + protected RemoveFiles self() { + return this; + } + + @Override + public Result execute() { + JobGroupInfo info = newJobGroupInfo("REMOVE-FILES", + String.format("Removing files from %s", table.location())); + return withJobGroupInfo(info, this::doExecute); + } + + private Result doExecute() { + boolean streamResults = PropertyUtil.propertyAsBoolean(options(), STREAM_RESULTS, false); + Dataset<Row> validFileDataset = buildValidFileDF(ops.current()).distinct(); + RemoveFilesActionResult result; + if (streamResults) { + result = deleteFiles(validFileDataset.toLocalIterator()); + } else { + result = deleteFiles(validFileDataset.collectAsList().iterator()); + } + return result; + } + + private Dataset<Row> appendTypeString(Dataset<Row> ds, String type) { + return ds.select(new Column("file_path"), functions.lit(type).as("file_type")); + } + + private Dataset<Row> buildValidFileDF(TableMetadata metadata) { + Table staticTable = newStaticTable(metadata, this.table.io()); + return appendTypeString(buildValidDataFileDF(staticTable), DATA_FILE) + .union(appendTypeString(buildManifestFileDF(staticTable), MANIFEST)) + .union(appendTypeString(buildManifestListDF(staticTable), MANIFEST_LIST)) + .union(appendTypeString(buildOtherMetadataFileDF(staticTable), OTHERS)); + } + + @Override + protected Dataset<Row> buildOtherMetadataFileDF(Table tbl) { + List<String> otherMetadataFiles = Lists.newArrayList(); + otherMetadataFiles.addAll(ReachableFileUtil.metadataFileLocations(table, true)); + otherMetadataFiles.add(ReachableFileUtil.versionHintLocation(table)); + return spark().createDataset(otherMetadataFiles, Encoders.STRING()).toDF("file_path"); + } + + /** + * Deletes files passed to it. + * + * @param deleted an Iterator of Spark Rows of the structure (path: String, type: String) + * @return Statistics on which files were deleted + */ + private RemoveFilesActionResult deleteFiles(Iterator<Row> deleted) { + AtomicLong dataFileCount = new AtomicLong(0L); + AtomicLong manifestCount = new AtomicLong(0L); + AtomicLong manifestListCount = new AtomicLong(0L); + AtomicLong otherFilesCount = new AtomicLong(0L); + + Tasks.foreach(deleted) + .retry(3).stopRetryOn(NotFoundException.class).suppressFailureWhenFinished() + .executeWith(deleteExecutorService) + .onFailure((fileInfo, exc) -> { + String file = fileInfo.getString(0); + String type = fileInfo.getString(1); + LOG.warn("Delete failed for {}: {}", type, file, exc); + }) + .run(fileInfo -> { + String file = fileInfo.getString(0); + String type = fileInfo.getString(1); + deleteFunc.accept(file); + switch (type) { + case DATA_FILE: + dataFileCount.incrementAndGet(); + LOG.trace("Deleted Data File: {}", file); + break; + case MANIFEST: + manifestCount.incrementAndGet(); + LOG.debug("Deleted Manifest: {}", file); + break; + case MANIFEST_LIST: + manifestListCount.incrementAndGet(); + LOG.debug("Deleted Manifest List: {}", file); + break; + case OTHERS: + otherFilesCount.incrementAndGet(); + LOG.debug("Others: {}", file); + break; + } + }); + + LOG.info("Deleted {} total files", dataFileCount.get() + manifestCount.get() + manifestListCount.get() + + otherFilesCount.get()); + return new RemoveFilesActionResult(dataFileCount.get(), manifestCount.get(), manifestListCount.get(), + otherFilesCount.get()); + } + + @Override + public RemoveFiles deleteWith(Consumer<String> deleteFn) { Review comment: I think it is best to push these methods before execute(). ########## File path: api/src/main/java/org/apache/iceberg/actions/RemoveFiles.java ########## @@ -0,0 +1,77 @@ +/* + * 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.actions; + +import java.util.concurrent.ExecutorService; +import java.util.function.Consumer; + +/** + * An action that deletes data, manifest, manifest lists in a table. + * <p> + * Implementations may use a query engine to distribute parts of work. + */ +public interface RemoveFiles extends Action<RemoveFiles, RemoveFiles.Result> { Review comment: As we switch to accepting a JSON metadata location, we should probably expose a method called `fileIO` to set a particular `FileIO` implementation. Should default to `HadoopFileIO`. ########## File path: api/src/main/java/org/apache/iceberg/actions/RemoveFiles.java ########## @@ -0,0 +1,77 @@ +/* + * 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.actions; + +import java.util.concurrent.ExecutorService; +import java.util.function.Consumer; + +/** + * An action that deletes data, manifest, manifest lists in a table. + * <p> + * Implementations may use a query engine to distribute parts of work. + */ +public interface RemoveFiles extends Action<RemoveFiles, RemoveFiles.Result> { + + /** + * Passes an alternative delete implementation that will be used for manifests and data files. + * + * @param deleteFunc a function that will be called to delete manifests and data files. + * The function accepts path to file as an argument. + * @return this for method chaining + */ + RemoveFiles deleteWith(Consumer<String> deleteFunc); + + /** + * Passes an alternative executor service that will be used for manifests and data files deletion. + * <p> + * If this method is not called, manifests and data files will still be deleted in + * the current thread. + * <p> + * + * @param executorService the service to use + * @return this for method chaining + */ + RemoveFiles executeDeleteWith(ExecutorService executorService); + + /** + * The action result that contains a summary of the execution. + */ + interface Result { + /** + * Returns the number of deleted data files. + */ + long deletedDataFilesCount(); Review comment: Yeah, I think `removedDeleteFilesCount` would match what we have in `SnapshotSummary`. Let's use `removedXXX` naming. ########## File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRemoveFilesSparkAction.java ########## @@ -0,0 +1,194 @@ +/* + * 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.Iterator; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.ReachableFileUtil; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.actions.RemoveFiles; +import org.apache.iceberg.actions.RemoveFilesActionResult; +import org.apache.iceberg.exceptions.NotFoundException; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.JobGroupInfo; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.Tasks; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.functions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * An action that performs the same operation as {@link RemoveFiles} but uses Spark + * to determine the files that needs to be deleted. The action uses metadata tables to + * find the files to be deleted. + * Deletes are performed locally after retrieving the results from the Spark executors. + */ +@SuppressWarnings("UnnecessaryAnonymousClass") +public class BaseRemoveFilesSparkAction + extends BaseSparkAction<RemoveFiles, RemoveFiles.Result> implements RemoveFiles { + private static final Logger LOG = LoggerFactory.getLogger(BaseRemoveFilesSparkAction.class); + + private static final String DATA_FILE = "Data File"; + private static final String MANIFEST = "Manifest"; + private static final String MANIFEST_LIST = "Manifest List"; + private static final String OTHERS = "Others"; + + private static final String STREAM_RESULTS = "stream-results"; + + // Creates an executor service that runs each task in the thread that invokes execute/submit. + private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = null; + + private final Table table; + private final TableOperations ops; + private final Consumer<String> defaultDelete = new Consumer<String>() { + @Override + public void accept(String file) { + ops.io().deleteFile(file); + } + }; + + private Consumer<String> deleteFunc = defaultDelete; + private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE; + + public BaseRemoveFilesSparkAction(SparkSession spark, Table table) { + super(spark); + this.table = table; + this.ops = ((HasTableOperations) table).operations(); + } + + @Override + protected RemoveFiles self() { + return this; + } + + @Override + public Result execute() { + JobGroupInfo info = newJobGroupInfo("REMOVE-FILES", + String.format("Removing files from %s", table.location())); Review comment: nit: should fit on one line ``` String format = String.format("Removing files reachable from %s", metadataFileLocation); JobGroupInfo info = newJobGroupInfo("REMOVE-FILES", format); return withJobGroupInfo(info, this::doExecute); ``` ########## File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRemoveFilesSparkAction.java ########## @@ -0,0 +1,171 @@ +/* + * 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.Iterator; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.actions.RemoveFiles; +import org.apache.iceberg.actions.RemoveFilesActionResult; +import org.apache.iceberg.exceptions.NotFoundException; +import org.apache.iceberg.spark.JobGroupInfo; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.Tasks; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.functions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An action that performs the same operation as {@link RemoveFiles} but uses Spark + * to determine the files that needs to be deleted. The action uses metadata tables to + * find the files to be deleted. + * Deletes are performed locally after retrieving the results from the Spark executors. + */ +@SuppressWarnings("UnnecessaryAnonymousClass") +public class BaseRemoveFilesSparkAction + extends BaseSparkAction<RemoveFiles, RemoveFiles.Result> implements RemoveFiles { + private static final Logger LOG = LoggerFactory.getLogger(BaseRemoveFilesSparkAction.class); + + private static final String DATA_FILE = "Data File"; + private static final String MANIFEST = "Manifest"; + private static final String MANIFEST_LIST = "Manifest List"; + + private static final String STREAM_RESULTS = "stream-results"; + + // Creates an executor service that runs each task in the thread that invokes execute/submit. + private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = null; + + private final Table table; + private final TableOperations ops; + private final Consumer<String> defaultDelete = new Consumer<String>() { + @Override + public void accept(String file) { + ops.io().deleteFile(file); + } + }; + + private Consumer<String> deleteFunc = defaultDelete; + private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE; + + public BaseRemoveFilesSparkAction(SparkSession spark, Table table) { + super(spark); + this.table = table; + this.ops = ((HasTableOperations) table).operations(); + } + + @Override + protected RemoveFiles self() { + return this; + } + + @Override + public Result execute() { + JobGroupInfo info = newJobGroupInfo("DROP-TABLE", "DROP-TABLE"); + return withJobGroupInfo(info, this::doExecute); + } + + private Result doExecute() { + boolean streamResults = PropertyUtil.propertyAsBoolean(options(), STREAM_RESULTS, false); + Dataset<Row> validFileDataset = buildValidFileDF(ops.current()); + RemoveFilesActionResult result; + if (streamResults) { + result = deleteFiles(validFileDataset.toLocalIterator()); + } else { + result = deleteFiles(validFileDataset.collectAsList().iterator()); + } + return result; + } + + private Dataset<Row> appendTypeString(Dataset<Row> ds, String type) { + return ds.select(new Column("file_path"), functions.lit(type).as("file_type")); Review comment: The name is also a bit weird as it not only adds file type, it also projects `file_path`. We could call it `projectFilePathWithType` or something. Also, I'd use `functions.col()` instead of `new Column` for consistency. ########## File path: spark/src/main/java/org/apache/iceberg/actions/Actions.java ########## @@ -88,6 +89,10 @@ public ExpireSnapshotsAction expireSnapshots() { return new ExpireSnapshotsAction(delegate); } + public BaseRemoveFilesSparkAction removeFilesAction() { Review comment: Let's also switch to accepting a JSON metadata file location in `ActionsProvider`. ########## File path: api/src/main/java/org/apache/iceberg/actions/RemoveFiles.java ########## @@ -0,0 +1,77 @@ +/* + * 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.actions; + +import java.util.concurrent.ExecutorService; +import java.util.function.Consumer; + +/** + * An action that deletes data, manifest, manifest lists in a table. + * <p> + * Implementations may use a query engine to distribute parts of work. + */ +public interface RemoveFiles extends Action<RemoveFiles, RemoveFiles.Result> { + + /** + * Passes an alternative delete implementation that will be used for manifests and data files. + * <p> + * + * @param deleteFunc a function that will be called to delete manifests and data files + * @return this for method chaining + */ + RemoveFiles deleteWith(Consumer<String> deleteFunc); + + /** + * Passes an alternative executor service that will be used for manifests and data files deletion. + * <p> + * If this method is not called, manifests and data files will still be deleted in + * the current thread. + * <p> + * + * @param executorService the service to use + * @return this for method chaining + */ + RemoveFiles executeDeleteWith(ExecutorService executorService); + + /** + * The action result that contains a summary of the execution. + */ + interface Result { + /** + * Returns the number of deleted data files. + */ + long deletedDataFilesCount(); + + /** + * Returns the number of deleted manifests. + */ + long deletedManifestsCount(); + + /** + * Returns the number of deleted manifest lists. + */ + long deletedManifestListsCount(); + + /** + * Returns the number of files deleted other than data, manifest and manifest list. Review comment: Let's refine the doc to reflect the concern @jackye1995 mentioned. -- 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. For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
