szehon-ho commented on a change in pull request #2415:
URL: https://github.com/apache/iceberg/pull/2415#discussion_r639574164



##########
File path: 
spark/src/main/java/org/apache/iceberg/spark/actions/BaseRemoveFilesSparkAction.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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) {

Review comment:
       Hi, I was wondering too, if we call this to drop the files without the 
table being dropped, doesn't it leave behind a broken table in the catalog that 
cannot be loaded anymore?  
   
   Best is if we can drop the table itself in the same call but probably that's 
not possible.  Maybe as another option, this still is an action on Table but it 
commits back a single empty metadata like create-table?  Then it still keeps 
atomicity, and also we can expose this action via SparkSQL procedure, to run 
before dropping a big table, and not worry about broken table if user fails to 
actually drop the table.
   
   Metadata json option could be a good indication that table needs to be 
dropped before, but there's no opportunity to sanity check that (as it break 
the table otherwise)?  It's also a bit harder to expose this action  eventually 
to Spark or Hive users via procedures, given metadata location is harder to get 
there.
   
   That being said, I think this is a nice feature (catalog.dropTable with 
purge=true can often timeout), and drop table is rare enough that I might be 
over-complicating it, I was just considering that Iceberg actions in general 
are atomic and do not have potential to  break the table.




-- 
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]

Reply via email to