namrathamyske opened a new issue, #9792:
URL: https://github.com/apache/iceberg/issues/9792

   ### Apache Iceberg version
   
   None
   
   ### Query engine
   
   None
   
   ### Please describe the bug 🐞
   
   When there are any exceptions of type `CleanableFailure`, we go ahead and 
[delete manifest list 
files](https://github.com/apache/iceberg/blob/56da99b9f908237ce0a9be565258fa10cd4fe88b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java#L415),
 we [throw 
exception](https://github.com/apache/iceberg/blob/56da99b9f908237ce0a9be565258fa10cd4fe88b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java#L418)
 back to spark. This triggers the 
[abort](https://github.com/apache/iceberg/blob/56da99b9f908237ce0a9be565258fa10cd4fe88b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java#L243)
 which further cleans up data files. If there is an exception while deleting 
manifest list files, they don't get deleted but data files are deleted.
   
   Test case to reproduce:
   ```
       @Test
       public void testCommitRuntimeException() throws IOException {
           String desc = "test_tbl";
           File parent = Files.createTempDir().getAbsoluteFile();
           Schema SCHEMA =
                   new Schema(
                           optional(1, "id", Types.IntegerType.get()),
                           optional(2, "data", Types.StringType.get()));
   
           File location = new File(parent, "test");
           File dataFolder = new File(location, "data");
           assertThat(dataFolder.mkdirs()).as("mkdirs should succeed").isTrue();
   
           TestTables.TestTable table =
                   TestTables.create(location, desc, SCHEMA, 
PartitionSpec.unpartitioned(), 2);
   
           table.refresh();
   
           AppendFiles append = table.newFastAppend();
   
           AppendFiles spyAppend = spy(append);
   
           doAnswer(
                   invocation -> {
                       TestTables.TestTableOperations ops = 
(TestTables.TestTableOperations) table.operations();
                       ops.failCommits(6);
                       append
                               .deleteWith(
                                       (e) -> {
                                           throw new RuntimeException("Cannot 
delete manifest files");
                                       })
                               .commit();
                       return null;
                   })
                   .when(spyAppend)
                   .commit();
   
           TestTables.TestTable spyTable = spy(table);
           when(spyTable.newAppend()).thenReturn(spyAppend);
           SparkTable sparkTable = new SparkTable(spyTable, false);
   
           String manualTableName = "unknown_exception";
           ManualSource.setTable(manualTableName, sparkTable);
   
           List<SimpleRecord> records =
                   Lists.newArrayList(
                           new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), 
new SimpleRecord(3, "c"));
   
           Dataset<Row> df = spark.createDataFrame(records, SimpleRecord.class);
   
           // Although an exception is thrown here, write and commit have 
succeeded
           CommitFailedException commitFailedException =
                   Assertions.assertThrows(
                           CommitFailedException.class,
                           () ->
                                   df.select("id", "data")
                                           .write()
                                           
.format("org.apache.iceberg.spark.source.ManualSource")
                                           .option(ManualSource.TABLE_NAME, 
manualTableName)
                                           
.option(SparkWriteOptions.WRITE_FORMAT, "parquet")
                                           .mode(SaveMode.Append)
                                           .save(location.getAbsolutePath()));
   
           commitFailedException.printStackTrace();
           table.refresh();
           spyTable.refresh();
       }
       ```
       
       The exceptions after commit failure:
       ```
       org.apache.iceberg.exceptions.CommitFailedException: Injected failure
        at 
org.apache.iceberg.TestTables$TestTableOperations.commit(TestTables.java:264)
        at 
org.apache.iceberg.SnapshotProducer.lambda$commit$2(SnapshotProducer.java:408)
        at 
org.apache.iceberg.util.Tasks$Builder.runTaskWithRetry(Tasks.java:413)
        at 
org.apache.iceberg.util.Tasks$Builder.runSingleThreaded(Tasks.java:219)
        at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:203)
        at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:196)
        at org.apache.iceberg.SnapshotProducer.commit(SnapshotProducer.java:382)
        at 
TestCommitFailedException.lambda$testCommitRuntimeException$2(TestCommitFailedException.java:114)
        at 
org.mockito.internal.stubbing.StubbedInvocationMatcher.answer(StubbedInvocationMatcher.java:42)
        at 
org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:103)
        at 
org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29)
        at 
org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:34)
        at 
org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:82)
        at 
org.mockito.internal.creation.bytebuddy.MockMethodAdvice.handle(MockMethodAdvice.java:134)
        at org.apache.iceberg.SnapshotProducer.commit(SnapshotProducer.java:370)
        at 
org.apache.iceberg.spark.source.SparkWrite.commitOperation(SparkWrite.java:233)
        at 
org.apache.iceberg.spark.source.SparkWrite.access$1300(SparkWrite.java:84)
        at 
org.apache.iceberg.spark.source.SparkWrite$BatchAppend.commit(SparkWrite.java:301)
        at 
org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec.writeWithV2(WriteToDataSourceV2Exec.scala:399)
        at 
org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec.writeWithV2$(WriteToDataSourceV2Exec.scala:359)
        at 
org.apache.spark.sql.execution.datasources.v2.AppendDataExec.writeWithV2(WriteToDataSourceV2Exec.scala:225)
        at 
org.apache.spark.sql.execution.datasources.v2.V2ExistingTableWriteExec.run(WriteToDataSourceV2Exec.scala:337)
        at 
org.apache.spark.sql.execution.datasources.v2.V2ExistingTableWriteExec.run$(WriteToDataSourceV2Exec.scala:336)
        at 
org.apache.spark.sql.execution.datasources.v2.AppendDataExec.run(WriteToDataSourceV2Exec.scala:225)
        at 
org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result$lzycompute(V2CommandExec.scala:43)
        at 
org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result(V2CommandExec.scala:43)
        at 
org.apache.spark.sql.execution.datasources.v2.V2CommandExec.executeCollect(V2CommandExec.scala:49)
        at 
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.$anonfun$applyOrElse$1(QueryExecution.scala:107)
        at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:125)
        at 
org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:201)
        at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:108)
        at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:900)
        at 
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:66)
        at 
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:107)
        at 
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:98)
        at 
org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:461)
        at 
org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:76)
        at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:461)
        at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:32)
        at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267)
        at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263)
        at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:32)
        at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:32)
        at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:437)
        at 
org.apache.spark.sql.execution.QueryExecution.eagerlyExecuteCommands(QueryExecution.scala:98)
        at 
org.apache.spark.sql.execution.QueryExecution.commandExecuted$lzycompute(QueryExecution.scala:85)
        at 
org.apache.spark.sql.execution.QueryExecution.commandExecuted(QueryExecution.scala:83)
        at 
org.apache.spark.sql.execution.QueryExecution.assertCommandExecuted(QueryExecution.scala:142)
        at 
org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:859)
        at 
org.apache.spark.sql.DataFrameWriter.saveInternal(DataFrameWriter.scala:312)
        at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:240)
        at 
TestCommitFailedException.lambda$testCommitRuntimeException$3(TestCommitFailedException.java:144)
        at org.junit.jupiter.api.AssertThrows.assertThrows(AssertThrows.java:53)
        at org.junit.jupiter.api.AssertThrows.assertThrows(AssertThrows.java:35)
        at org.junit.jupiter.api.Assertions.assertThrows(Assertions.java:3115)
        at 
TestCommitFailedException.testCommitRuntimeException(TestCommitFailedException.java:135)
        at 
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at 
java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.base/java.lang.reflect.Method.invoke(Method.java:566)
        at 
org.junit.platform.commons.util.ReflectionUtils.invokeMethod(ReflectionUtils.java:728)
        at 
org.junit.jupiter.engine.execution.MethodInvocation.proceed(MethodInvocation.java:60)
        at 
org.junit.jupiter.engine.execution.InvocationInterceptorChain$ValidatingInvocation.proceed(InvocationInterceptorChain.java:131)
        at 
org.junit.jupiter.engine.extension.TimeoutExtension.intercept(TimeoutExtension.java:156)
        at 
org.junit.jupiter.engine.extension.TimeoutExtension.interceptTestableMethod(TimeoutExtension.java:147)
        at 
org.junit.jupiter.engine.extension.TimeoutExtension.interceptTestMethod(TimeoutExtension.java:86)
        at 
org.junit.jupiter.engine.execution.InterceptingExecutableInvoker$ReflectiveInterceptorCall.lambda$ofVoidMethod$0(InterceptingExecutableInvoker.java:103)
        at 
org.junit.jupiter.engine.execution.InterceptingExecutableInvoker.lambda$invoke$0(InterceptingExecutableInvoker.java:93)
        at 
org.junit.jupiter.engine.execution.InvocationInterceptorChain$InterceptedInvocation.proceed(InvocationInterceptorChain.java:106)
        at 
org.junit.jupiter.engine.execution.InvocationInterceptorChain.proceed(InvocationInterceptorChain.java:64)
        at 
org.junit.jupiter.engine.execution.InvocationInterceptorChain.chainAndInvoke(InvocationInterceptorChain.java:45)
        at 
org.junit.jupiter.engine.execution.InvocationInterceptorChain.invoke(InvocationInterceptorChain.java:37)
        at 
org.junit.jupiter.engine.execution.InterceptingExecutableInvoker.invoke(InterceptingExecutableInvoker.java:92)
        at 
org.junit.jupiter.engine.execution.InterceptingExecutableInvoker.invoke(InterceptingExecutableInvoker.java:86)
        at 
org.junit.jupiter.engine.descriptor.TestMethodTestDescriptor.lambda$invokeTestMethod$7(TestMethodTestDescriptor.java:218)
        at 
org.junit.platform.engine.support.hierarchical.ThrowableCollector.execute(ThrowableCollector.java:73)
        at 
org.junit.jupiter.engine.descriptor.TestMethodTestDescriptor.invokeTestMethod(TestMethodTestDescriptor.java:214)
        at 
org.junit.jupiter.engine.descriptor.TestMethodTestDescriptor.execute(TestMethodTestDescriptor.java:139)
        at 
org.junit.jupiter.engine.descriptor.TestMethodTestDescriptor.execute(TestMethodTestDescriptor.java:69)
        at 
org.junit.platform.engine.support.hierarchical.NodeTestTask.lambda$executeRecursively$6(NodeTestTask.java:151)
        at 
org.junit.platform.engine.support.hierarchical.ThrowableCollector.execute(ThrowableCollector.java:73)
        at 
org.junit.platform.engine.support.hierarchical.NodeTestTask.lambda$executeRecursively$8(NodeTestTask.java:141)
        at 
org.junit.platform.engine.support.hierarchical.Node.around(Node.java:137)
        at 
org.junit.platform.engine.support.hierarchical.NodeTestTask.lambda$executeRecursively$9(NodeTestTask.java:139)
        at 
org.junit.platform.engine.support.hierarchical.ThrowableCollector.execute(ThrowableCollector.java:73)
        at 
org.junit.platform.engine.support.hierarchical.NodeTestTask.executeRecursively(NodeTestTask.java:138)
        at 
org.junit.platform.engine.support.hierarchical.NodeTestTask.execute(NodeTestTask.java:95)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1541)
        at 
org.junit.platform.engine.support.hierarchical.SameThreadHierarchicalTestExecutorService.invokeAll(SameThreadHierarchicalTestExecutorService.java:41)
        at 
org.junit.platform.engine.support.hierarchical.NodeTestTask.lambda$executeRecursively$6(NodeTestTask.java:155)
        at 
org.junit.platform.engine.support.hierarchical.ThrowableCollector.execute(ThrowableCollector.java:73)
        at 
org.junit.platform.engine.support.hierarchical.NodeTestTask.lambda$executeRecursively$8(NodeTestTask.java:141)
        at 
org.junit.platform.engine.support.hierarchical.Node.around(Node.java:137)
        at 
org.junit.platform.engine.support.hierarchical.NodeTestTask.lambda$executeRecursively$9(NodeTestTask.java:139)
        at 
org.junit.platform.engine.support.hierarchical.ThrowableCollector.execute(ThrowableCollector.java:73)
        at 
org.junit.platform.engine.support.hierarchical.NodeTestTask.executeRecursively(NodeTestTask.java:138)
        at 
org.junit.platform.engine.support.hierarchical.NodeTestTask.execute(NodeTestTask.java:95)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1541)
        at 
org.junit.platform.engine.support.hierarchical.SameThreadHierarchicalTestExecutorService.invokeAll(SameThreadHierarchicalTestExecutorService.java:41)
        at 
org.junit.platform.engine.support.hierarchical.NodeTestTask.lambda$executeRecursively$6(NodeTestTask.java:155)
        at 
org.junit.platform.engine.support.hierarchical.ThrowableCollector.execute(ThrowableCollector.java:73)
        at 
org.junit.platform.engine.support.hierarchical.NodeTestTask.lambda$executeRecursively$8(NodeTestTask.java:141)
        at 
org.junit.platform.engine.support.hierarchical.Node.around(Node.java:137)
        at 
org.junit.platform.engine.support.hierarchical.NodeTestTask.lambda$executeRecursively$9(NodeTestTask.java:139)
        at 
org.junit.platform.engine.support.hierarchical.ThrowableCollector.execute(ThrowableCollector.java:73)
        at 
org.junit.platform.engine.support.hierarchical.NodeTestTask.executeRecursively(NodeTestTask.java:138)
        at 
org.junit.platform.engine.support.hierarchical.NodeTestTask.execute(NodeTestTask.java:95)
        at 
org.junit.platform.engine.support.hierarchical.SameThreadHierarchicalTestExecutorService.submit(SameThreadHierarchicalTestExecutorService.java:35)
        at 
org.junit.platform.engine.support.hierarchical.HierarchicalTestExecutor.execute(HierarchicalTestExecutor.java:57)
        at 
org.junit.platform.engine.support.hierarchical.HierarchicalTestEngine.execute(HierarchicalTestEngine.java:54)
        at 
org.junit.platform.launcher.core.EngineExecutionOrchestrator.execute(EngineExecutionOrchestrator.java:198)
        at 
org.junit.platform.launcher.core.EngineExecutionOrchestrator.execute(EngineExecutionOrchestrator.java:169)
        at 
org.junit.platform.launcher.core.EngineExecutionOrchestrator.execute(EngineExecutionOrchestrator.java:93)
        at 
org.junit.platform.launcher.core.EngineExecutionOrchestrator.lambda$execute$0(EngineExecutionOrchestrator.java:58)
        at 
org.junit.platform.launcher.core.EngineExecutionOrchestrator.withInterceptedStreams(EngineExecutionOrchestrator.java:141)
        at 
org.junit.platform.launcher.core.EngineExecutionOrchestrator.execute(EngineExecutionOrchestrator.java:57)
        at 
org.junit.platform.launcher.core.DefaultLauncher.execute(DefaultLauncher.java:103)
        at 
org.junit.platform.launcher.core.DefaultLauncher.execute(DefaultLauncher.java:85)
        at 
org.junit.platform.launcher.core.DelegatingLauncher.execute(DelegatingLauncher.java:47)
        at 
org.junit.platform.launcher.core.SessionPerRequestLauncher.execute(SessionPerRequestLauncher.java:63)
        at 
com.intellij.junit5.JUnit5IdeaTestRunner.startRunnerWithArgs(JUnit5IdeaTestRunner.java:57)
        at 
com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38)
        at 
com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11)
        at 
com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35)
        at 
com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232)
        at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55)
        Suppressed: java.lang.RuntimeException: Cannot delete manifest files
                at 
TestCommitFailedException.lambda$testCommitRuntimeException$1(TestCommitFailedException.java:112)
                at 
org.apache.iceberg.SnapshotProducer.deleteFile(SnapshotProducer.java:490)
                at 
org.apache.iceberg.SnapshotProducer.cleanAll(SnapshotProducer.java:483)
                at 
org.apache.iceberg.util.Exceptions.suppressExceptions(Exceptions.java:41)
                at 
org.apache.iceberg.util.Exceptions.suppressAndThrow(Exceptions.java:50)
                at 
org.apache.iceberg.SnapshotProducer.commit(SnapshotProducer.java:415)
                ... 119 more
   
   ```
   
   Getting logs after abort 
   
   ```
       24/02/24 12:26:36 ERROR AppendDataExec: Data source write support 
IcebergBatchWrite(table=test_tbl, format=PARQUET) is aborting.
   24/02/24 12:26:36 INFO SparkCleanupUtil: Deleted 2 file(s) (job abort)
   24/02/24 12:26:36 ERROR AppendDataExec: Data source write support 
IcebergBatchWrite(table=test_tbl, format=PARQUET) aborted.
   ```
   


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