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

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

singhpk234 commented on code in PR #6876:
URL: https://github.com/apache/iceberg/pull/6876#discussion_r1110744403


##########
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)",

Review Comment:
   can we show stage attempID better something like : 
   ```Task (id : <TaskID>, attempt : <attemptNumber>), Stage (id : <stageId>, attemp : <attempNumber>) ```
   in place of 
   (task 0, attempt 0, stage 0.0)



##########
spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.extensions;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.BulkDeletionFailureException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.SupportsBulkOperations;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.spark.SparkCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.source.SimpleRecord;
+import org.apache.spark.SparkException;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.junit.After;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runners.Parameterized;
+
+public class TestWriteAborts extends SparkExtensionsTestBase {
+
+  @Parameterized.Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+      {
+        "testhive",
+        SparkCatalog.class.getName(),
+        ImmutableMap.of(
+            "type",
+            "hive",
+            CatalogProperties.FILE_IO_IMPL,
+            CustomFileIO.class.getName(),
+            "default-namespace",
+            "default")
+      },
+      {
+        "testhivebulk",
+        SparkCatalog.class.getName(),
+        ImmutableMap.of(
+            "type",
+            "hive",
+            CatalogProperties.FILE_IO_IMPL,
+            CustomBulkFileIO.class.getName(),
+            "default-namespace",
+            "default")
+      }
+    };
+  }
+
+  @Rule public TemporaryFolder temp = new TemporaryFolder();
+
+  public TestWriteAborts(String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+  }
+
+  @Test
+  public void testBatchAppend() throws Exception {
+    String dataLocation = temp.newFolder().toString();
+
+    sql(
+        "CREATE TABLE %s (id INT, data STRING) "
+            + "USING iceberg "
+            + "PARTITIONED BY (data)"
+            + "TBLPROPERTIES ('%s' '%s')",
+        tableName, TableProperties.WRITE_DATA_LOCATION, dataLocation);
+
+    List<SimpleRecord> records =
+        ImmutableList.of(
+            new SimpleRecord(1, "a"),
+            new SimpleRecord(2, "b"),
+            new SimpleRecord(3, "a"),
+            new SimpleRecord(4, "b"));
+    Dataset<Row> inputDF = spark.createDataFrame(records, SimpleRecord.class);
+
+    AssertHelpers.assertThrows(
+        "Write must fail",
+        SparkException.class,
+        "Writing job aborted",
+        () -> {
+          try {
+            // incoming records are not ordered by partitions so the job must fail
+            inputDF
+                .coalesce(1)
+                .sortWithinPartitions("id")
+                .writeTo(tableName)
+                .option(SparkWriteOptions.USE_TABLE_DISTRIBUTION_AND_ORDERING, "false")
+                .append();
+          } catch (NoSuchTableException e) {
+            throw new RuntimeException(e);
+          }
+        });
+
+    assertEquals("Should be no records", sql("SELECT * FROM %s", tableName), ImmutableList.of());
+
+    assertEquals(
+        "Should be no orphan data files",
+        ImmutableList.of(),
+        sql(
+            "CALL %s.system.remove_orphan_files(table => '%s', older_than => %dL, location => '%s')",
+            catalogName, tableName, System.currentTimeMillis(), dataLocation));

Review Comment:
   [minor] can we give this a timestamp in future by 5000 ms to avoid flakyness like https://github.com/apache/iceberg/pull/4859, to make sure we will include all the orphan files if there.



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