yyanyy commented on a change in pull request #2415: URL: https://github.com/apache/iceberg/pull/2415#discussion_r623479608
########## File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRemoveFilesSparkAction.java ########## @@ -0,0 +1,216 @@ +/* + * 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.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Set; +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.TableMetadataParser; +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.io.FileIO; +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", "REMOVE-FILES"); + 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(ops), OTHERS)); + } + + protected Dataset<Row> buildOtherMetadataFileDF(TableOperations tableOps) { Review comment: `BaseAction` and `BaseSparkAction` seem to have similar methods `getOtherMetadataFilePaths` and `getOtherMetadataFilePaths` that are less comprehensive than this since this seems to look for `metadata.previousFiles` in older metadata files, and their only usage is within `BaseRemoveOrphanFilesSparkAction`; wondering if we want to clean them up/if remove orphan action would benefit from it, although it could be a separate issue/PR ########## 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(); Review comment: `ContentFiles` since I guess positional/equality delete files will also be deleted? ########## 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: might be able to put as part of `BaseSparkAction` since it's used in `BaseExpireSnapshotsSparkAction` too ########## File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRemoveFilesSparkAction.java ########## @@ -0,0 +1,216 @@ +/* + * 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.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Set; +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.TableMetadataParser; +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.io.FileIO; +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", "REMOVE-FILES"); + 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(ops), OTHERS)); + } + + protected Dataset<Row> buildOtherMetadataFileDF(TableOperations tableOps) { + List<String> otherMetadataFiles = Lists.newArrayList(); + otherMetadataFiles.add(tableOps.metadataFileLocation("version-hint.text")); + + Set<String> metadataFiles = new HashSet<>(); + TableMetadata metadata = tableOps.current(); + String location = metadata.metadataFileLocation(); + metadataFiles.add(location); + getMetadataFiles(location, metadataFiles, tableOps.io()); + otherMetadataFiles.addAll(metadataFiles); + return spark().createDataset(otherMetadataFiles, Encoders.STRING()).toDF("file_path"); + } + + private void getMetadataFiles(String metadataFileLocation, Set<String> metaFiles, FileIO io) { + if (metadataFileLocation == null) { + return; + } + long minTimeStamp = Long.MAX_VALUE; + String minMetadataLocation = null; + TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation); Review comment: Since this seems to be running recursively, is it possible that a previous metadata file's "previousFiles" is already cleaned up an no longer exist, and here when we read a non existing file it will throw exception? ########## File path: spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceHiveTables.java ########## @@ -54,7 +54,8 @@ public void dropTable() throws IOException { @Override public Table createTable(TableIdentifier ident, Schema schema, PartitionSpec spec) { TestIcebergSourceHiveTables.currentIdentifier = ident; - return TestIcebergSourceHiveTables.catalog.createTable(ident, schema, spec); + Table table = TestIcebergSourceHiveTables.catalog.createTable(ident, schema, spec); Review comment: do we need this change? ########## File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRemoveFilesSparkAction.java ########## @@ -0,0 +1,216 @@ +/* + * 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.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Set; +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.TableMetadataParser; +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.io.FileIO; +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", "REMOVE-FILES"); + return withJobGroupInfo(info, this::doExecute); + } + + private Result doExecute() { + boolean streamResults = PropertyUtil.propertyAsBoolean(options(), STREAM_RESULTS, false); Review comment: sorry I'm not super familiar with other actions' code base, how would this be set? looks like it's used in `ExpireSnapshotsAction` but it seems like just for passing a parameter to `BaseExpireSnapshotsSparkAction` and is not something user can control, and here we don't really use this base class? -- 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]
