You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by "szehon-ho (via GitHub)" <gi...@apache.org> on 2023/02/18 07:25:42 UTC

[GitHub] [iceberg] szehon-ho commented on a diff in pull request #6876: Spark 3.3: Improve task and job abort handling

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


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java:
##########
@@ -541,8 +546,8 @@ public void abort() throws IOException {
       close();
 
       WriteResult result = delegate.result();
-      cleanFiles(io, Arrays.asList(result.dataFiles()));
-      cleanFiles(io, Arrays.asList(result.deleteFiles()));
+      SparkCleanupUtil.deleteTaskFiles(io, Arrays.asList(result.dataFiles()));

Review Comment:
   Don't we want to take advantage of bulk here and delete both data + delete in one call?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java:
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.source;
+
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.exceptions.NotFoundException;
+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.Lists;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.TaskContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** A utility for cleaning up written but not committed files. */
+class SparkCleanupUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkCleanupUtil.class);
+
+  private static final int DELETE_NUM_RETRIES = 3;
+  private static final int DELETE_MIN_RETRY_WAIT_MS = 100; // 100 ms
+  private static final int DELETE_MAX_RETRY_WAIT_MS = 30 * 1000; // 30 seconds
+  private static final int DELETE_TOTAL_RETRY_TIME_MS = 2 * 60 * 1000; // 2 minutes
+
+  private SparkCleanupUtil() {}
+
+  /**
+   * Attempts to delete as many files produced by a task as possible.
+   *
+   * <p>Note this method will log Spark task info and is supposed to be called only on executors.
+   * Use {@link #deleteFiles(String, FileIO, List)} to delete files on the driver.
+   *
+   * @param io a {@link FileIO} instance used for deleting files
+   * @param files a list of files to delete
+   */
+  public static void deleteTaskFiles(FileIO io, List<? extends ContentFile<?>> files) {
+    deleteFiles(taskInfo(), io, files);
+  }
+
+  // the format matches what Spark uses for internal logging
+  private static String taskInfo() {

Review Comment:
   Nit: what do you think to move private method to bottom?  Breaks the flow of code a bit (would have liked to see deleteFiles right after deleteTaskFiles as its the main delegate)



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java:
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.source;
+
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.exceptions.NotFoundException;
+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.Lists;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.TaskContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** A utility for cleaning up written but not committed files. */
+class SparkCleanupUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkCleanupUtil.class);
+
+  private static final int DELETE_NUM_RETRIES = 3;
+  private static final int DELETE_MIN_RETRY_WAIT_MS = 100; // 100 ms
+  private static final int DELETE_MAX_RETRY_WAIT_MS = 30 * 1000; // 30 seconds
+  private static final int DELETE_TOTAL_RETRY_TIME_MS = 2 * 60 * 1000; // 2 minutes
+
+  private SparkCleanupUtil() {}
+
+  /**
+   * Attempts to delete as many files produced by a task as possible.
+   *
+   * <p>Note this method will log Spark task info and is supposed to be called only on executors.
+   * Use {@link #deleteFiles(String, FileIO, List)} to delete files on the driver.
+   *
+   * @param io a {@link FileIO} instance used for deleting files
+   * @param files a list of files to delete
+   */
+  public static void deleteTaskFiles(FileIO io, List<? extends ContentFile<?>> files) {
+    deleteFiles(taskInfo(), io, files);
+  }
+
+  // the format matches what Spark uses for internal logging
+  private static String taskInfo() {
+    TaskContext taskContext = TaskContext.get();
+    if (taskContext == null) {
+      return "unknown task";
+    } else {
+      return String.format(
+          "partition %d (task %d, attempt %d, stage %d.%d)",
+          taskContext.partitionId(),
+          taskContext.taskAttemptId(),
+          taskContext.attemptNumber(),
+          taskContext.stageId(),
+          taskContext.stageAttemptNumber());
+    }
+  }
+
+  /**
+   * Attempts to delete as many given files as possible.
+   *
+   * @param io a {@link FileIO} instance used for deleting files
+   * @param files a list of files to delete
+   */
+  public static void deleteFiles(String context, FileIO io, List<? extends ContentFile<?>> files) {
+    List<String> paths = Lists.transform(files, file -> file.path().toString());
+    deletePaths(context, io, paths);
+  }
+
+  /**
+   * Attempts to delete as many files at given paths as possible.
+   *
+   * @param io a {@link FileIO} instance used for deleting files
+   * @param paths a list of file paths to delete
+   */
+  public static void deletePaths(String context, FileIO io, List<String> paths) {
+    if (io instanceof SupportsBulkOperations) {
+      SupportsBulkOperations bulkIO = (SupportsBulkOperations) io;
+      bulkDelete(context, bulkIO, paths);
+    } else {
+      delete(context, io, paths);
+    }
+  }
+
+  private static void bulkDelete(String context, SupportsBulkOperations io, List<String> paths) {
+    try {
+      io.deleteFiles(paths);
+      LOG.info("Deleted {} file(s) using bulk deletes ({})", paths.size(), context);
+
+    } catch (BulkDeletionFailureException e) {
+      int deletedFilesCount = paths.size() - e.numberFailedObjects();
+      LOG.warn(
+          "Deleted only {} of {} file(s) using bulk deletes ({})",
+          deletedFilesCount,
+          paths.size(),
+          context);
+    }
+  }
+
+  private static void delete(String context, FileIO io, List<String> paths) {
+    AtomicInteger failedDeletesCount = new AtomicInteger(0);
+
+    Tasks.foreach(paths)
+        .executeWith(ThreadPools.getWorkerPool())
+        .stopRetryOn(NotFoundException.class)
+        .suppressFailureWhenFinished()
+        .onFailure(
+            (path, exception) -> {
+              LOG.warn("Failed to delete {} ({})", path, context, exception);
+              failedDeletesCount.incrementAndGet();
+            })
+        .retry(DELETE_NUM_RETRIES)
+        .exponentialBackoff(
+            DELETE_MIN_RETRY_WAIT_MS,
+            DELETE_MAX_RETRY_WAIT_MS,
+            DELETE_TOTAL_RETRY_TIME_MS,
+            2 /* exponential */)
+        .run(io::deleteFile);
+
+    if (failedDeletesCount.get() > 0) {
+      int deletedFilesCount = paths.size() - failedDeletesCount.get();

Review Comment:
   Will failedDeletesCount overcount if files fails to be deleted repeatedly?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java:
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.source;
+
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.exceptions.NotFoundException;
+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.Lists;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.TaskContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** A utility for cleaning up written but not committed files. */
+class SparkCleanupUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkCleanupUtil.class);
+
+  private static final int DELETE_NUM_RETRIES = 3;
+  private static final int DELETE_MIN_RETRY_WAIT_MS = 100; // 100 ms
+  private static final int DELETE_MAX_RETRY_WAIT_MS = 30 * 1000; // 30 seconds
+  private static final int DELETE_TOTAL_RETRY_TIME_MS = 2 * 60 * 1000; // 2 minutes
+
+  private SparkCleanupUtil() {}
+
+  /**
+   * Attempts to delete as many files produced by a task as possible.
+   *
+   * <p>Note this method will log Spark task info and is supposed to be called only on executors.
+   * Use {@link #deleteFiles(String, FileIO, List)} to delete files on the driver.
+   *
+   * @param io a {@link FileIO} instance used for deleting files
+   * @param files a list of files to delete
+   */
+  public static void deleteTaskFiles(FileIO io, List<? extends ContentFile<?>> files) {
+    deleteFiles(taskInfo(), io, files);
+  }
+
+  // the format matches what Spark uses for internal logging
+  private static String taskInfo() {
+    TaskContext taskContext = TaskContext.get();
+    if (taskContext == null) {
+      return "unknown task";
+    } else {
+      return String.format(
+          "partition %d (task %d, attempt %d, stage %d.%d)",
+          taskContext.partitionId(),
+          taskContext.taskAttemptId(),
+          taskContext.attemptNumber(),
+          taskContext.stageId(),
+          taskContext.stageAttemptNumber());
+    }
+  }
+
+  /**
+   * Attempts to delete as many given files as possible.
+   *
+   * @param io a {@link FileIO} instance used for deleting files
+   * @param files a list of files to delete
+   */
+  public static void deleteFiles(String context, FileIO io, List<? extends ContentFile<?>> files) {
+    List<String> paths = Lists.transform(files, file -> file.path().toString());
+    deletePaths(context, io, paths);
+  }
+
+  /**
+   * Attempts to delete as many files at given paths as possible.
+   *
+   * @param io a {@link FileIO} instance used for deleting files
+   * @param paths a list of file paths to delete
+   */
+  public static void deletePaths(String context, FileIO io, List<String> paths) {

Review Comment:
   Can we add javadoc to context?  (same for other methods)



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java:
##########
@@ -229,40 +215,23 @@ private void commitOperation(SnapshotUpdate<?> operation, String description) {
 
   private void abort(WriterCommitMessage[] messages) {
     if (cleanupOnAbort) {
-      Map<String, String> props = table.properties();
-      Tasks.foreach(files(messages))
-          .executeWith(ThreadPools.getWorkerPool())
-          .retry(PropertyUtil.propertyAsInt(props, COMMIT_NUM_RETRIES, COMMIT_NUM_RETRIES_DEFAULT))
-          .exponentialBackoff(
-              PropertyUtil.propertyAsInt(
-                  props, COMMIT_MIN_RETRY_WAIT_MS, COMMIT_MIN_RETRY_WAIT_MS_DEFAULT),
-              PropertyUtil.propertyAsInt(
-                  props, COMMIT_MAX_RETRY_WAIT_MS, COMMIT_MAX_RETRY_WAIT_MS_DEFAULT),
-              PropertyUtil.propertyAsInt(
-                  props, COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT),
-              2.0 /* exponential */)
-          .throwFailureWhenFinished()
-          .run(
-              file -> {
-                table.io().deleteFile(file.path().toString());
-              });
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
     } else {
-      LOG.warn(
-          "Skipping cleaning up of data files because Iceberg was unable to determine the final commit state");
+      LOG.warn("Skipping cleanup of written files, unable to determine the final commit state");
     }
   }
 
-  private Iterable<DataFile> files(WriterCommitMessage[] messages) {
-    if (messages.length > 0) {
-      return Iterables.concat(
-          Iterables.transform(
-              Arrays.asList(messages),
-              message ->
-                  message != null
-                      ? ImmutableList.copyOf(((TaskCommit) message).files())
-                      : ImmutableList.of()));
-    }
-    return ImmutableList.of();
+  private List<DataFile> files(WriterCommitMessage[] messages) {

Review Comment:
   I have a little concern about memory, now we are manifesting paths into List, instead of keeping them as Iterable (if they are originally).  I see its mostly to log, I wonder if we can't implement a wrapping counter iterable for that?



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