You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/08/01 18:04:59 UTC

[GitHub] [iceberg] szehon-ho commented on a diff in pull request #5375: Chores: using bulk delete AMAP

szehon-ho commented on code in PR #5375:
URL: https://github.com/apache/iceberg/pull/5375#discussion_r934783633


##########
core/src/main/java/org/apache/iceberg/util/FileIOUtil.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.util;
+
+import java.util.concurrent.ExecutorService;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.io.BulkDeletionFailureException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.SupportsBulkOperations;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FileIOUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(FileIOUtil.class);
+
+  public static BulkDeleter bulkDeleteManifests(FileIO io, Iterable<ManifestFile> files) {
+    return bulkDelete(io, Iterables.transform(files, ManifestFile::path));
+  }
+
+  public static <C extends ContentFile<?>> BulkDeleter bulkDeleteFiles(
+      FileIO io, Iterable<C> files) {
+    return bulkDelete(io, Iterables.transform(files, file -> file.path().toString()));
+  }
+
+  public static BulkDeleter bulkDelete(FileIO io, Iterable<String> files) {
+    return new BulkDeleter(io, files);
+  }
+
+  public static BulkDeleter bulkDelete(FileIO io, String file) {
+    return new BulkDeleter(io, Sets.newHashSet(file));
+  }
+
+  public static class BulkDeleter {
+    private final FileIO io;
+    private final Iterable<String> files;
+    private String name = "files";
+    private ExecutorService service = null;
+
+    private BulkDeleter(FileIO io, Iterable<String> files) {
+      this.io = io;
+      this.files = files;
+    }
+
+    public BulkDeleter name(String newName) {
+      this.name = newName;
+      return this;
+    }
+
+    public BulkDeleter executeWith(ExecutorService svc) {
+      this.service = svc;
+      return this;
+    }
+
+    public void execute() {
+      if (io instanceof SupportsBulkOperations) {
+        try {
+          SupportsBulkOperations bulkIO = (SupportsBulkOperations) io;
+          bulkIO.deleteFiles(files);

Review Comment:
   Should we use the configured 'service' pool?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java:
##########
@@ -678,11 +678,9 @@ public static List<SparkPartition> filterPartitions(
   }
 
   private static void deleteManifests(FileIO io, List<ManifestFile> manifests) {
-    Tasks.foreach(manifests)
+    FileIOUtil.bulkDeleteManifests(io, manifests)

Review Comment:
   Looks like we dont pass a name here.  Thinking about it, if a name is always recommended, why not make it mandatory parameter  of the bulkDelete API?



##########
core/src/main/java/org/apache/iceberg/util/FileIOUtil.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.util;
+
+import java.util.concurrent.ExecutorService;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.io.BulkDeletionFailureException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.SupportsBulkOperations;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FileIOUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(FileIOUtil.class);
+
+  public static BulkDeleter bulkDeleteManifests(FileIO io, Iterable<ManifestFile> files) {
+    return bulkDelete(io, Iterables.transform(files, ManifestFile::path));
+  }
+
+  public static <C extends ContentFile<?>> BulkDeleter bulkDeleteFiles(
+      FileIO io, Iterable<C> files) {
+    return bulkDelete(io, Iterables.transform(files, file -> file.path().toString()));
+  }
+
+  public static BulkDeleter bulkDelete(FileIO io, Iterable<String> files) {
+    return new BulkDeleter(io, files);
+  }
+
+  public static BulkDeleter bulkDelete(FileIO io, String file) {
+    return new BulkDeleter(io, Sets.newHashSet(file));
+  }
+
+  public static class BulkDeleter {
+    private final FileIO io;
+    private final Iterable<String> files;
+    private String name = "files";
+    private ExecutorService service = null;
+
+    private BulkDeleter(FileIO io, Iterable<String> files) {
+      this.io = io;
+      this.files = files;
+    }
+
+    public BulkDeleter name(String newName) {
+      this.name = newName;
+      return this;
+    }
+
+    public BulkDeleter executeWith(ExecutorService svc) {
+      this.service = svc;
+      return this;
+    }
+
+    public void execute() {
+      if (io instanceof SupportsBulkOperations) {
+        try {
+          SupportsBulkOperations bulkIO = (SupportsBulkOperations) io;
+          bulkIO.deleteFiles(files);
+        } catch (BulkDeletionFailureException e) {
+          LOG.warn("Failed to delete {} {}", e.numberFailedObjects(), name);
+        } catch (Exception e) {
+          // ignore
+        }
+      } else {
+        Tasks.foreach(files)
+            .noRetry()
+            .executeWith(service)
+            .suppressFailureWhenFinished()
+            .onFailure((file, exc) -> LOG.warn("Delete failed for {}: {}", name, file, exc))

Review Comment:
   Are we missing a parameter?



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org