gaborkaszab commented on code in PR #11837:
URL: https://github.com/apache/iceberg/pull/11837#discussion_r1955843549


##########
core/src/main/java/org/apache/iceberg/BulkDeleteConsumer.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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;
+
+import java.util.List;
+import java.util.function.Consumer;
+import org.apache.iceberg.io.SupportsBulkOperations;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+/**
+ * Consumer class to collect file paths one by one and perform a bulk deletion 
on them. Not thread
+ * safe.
+ */
+public class BulkDeleteConsumer implements Consumer<String> {
+  private final List<String> files = Lists.newArrayList();
+
+  private final SupportsBulkOperations ops;
+
+  public BulkDeleteConsumer(SupportsBulkOperations ops) {
+    this.ops = ops;
+  }
+
+  @Override
+  public void accept(String file) {
+    files.add(file);
+  }
+
+  public void consumeAll() {
+    if (files.isEmpty()) {
+      return;
+    }
+
+    ops.deleteFiles(files);

Review Comment:
   Thanks for the observations! I know the sequential delete approach here does 
retries, however I also checked 
[CatalogUtil.deleteFiles()](https://github.com/apache/iceberg/blob/main/core/src/main/java/org/apache/iceberg/CatalogUtil.java#L165)
 (that is used by the dropTable() functionality) and apparently there we don't 
do retries.
   I checked the 
[DeleteOrphanFileSperAction](https://github.com/apache/iceberg/blob/main/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java#L254)
 too and apparently that also doesn't do retries for neither the bulk nor the 
sequential deletes.
   Also it doesn't do bulking, it leaves it to the FileIO (relevant for the 
other question on this PR).
   
   Additionally, if bulk deletion fails, it's possible that the subset of the 
files were deleted, however we won't know which just get a number of failures 
from the Exception. So in case of partial success/failure if we retry the bulk 
delete with the same list of paths it will fail again and again.



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

Reply via email to