Kontinuation commented on code in PR #12348:
URL: https://github.com/apache/iceberg/pull/12348#discussion_r1966100776


##########
spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java:
##########
@@ -118,6 +120,7 @@ public void testStreamingWriteAppendMode() throws Exception 
{
       // remove the last commit to force Spark to reprocess batch #1
       File lastCommitFile = new File(checkpoint + "/commits/1");
       Assert.assertTrue("The commit file must be deleted", 
lastCommitFile.delete());
+      Files.deleteIfExists(Paths.get(checkpoint + "/commits/.1.crc"));

Review Comment:
   The .crc file will be renamed along with the main file since 
[HADOOP-16255](https://issues.apache.org/jira/browse/HADOOP-16255), deleting 
the main file without deleting the crc file will result in a failure when 
renaming to the main file again:
   
   ```
   org.apache.hadoop.fs.FileAlreadyExistsException: Rename destination 
file:/var/folders/jw/nz45tb550rbgjkndd37m8rrh0000gn/T/junit-12664551068658781194/parquet/checkpoint/commits/.1.crc
 already exists.
        at 
org.apache.hadoop.fs.AbstractFileSystem.renameInternal(AbstractFileSystem.java:876)
        at 
org.apache.hadoop.fs.AbstractFileSystem.rename(AbstractFileSystem.java:807)
        at org.apache.hadoop.fs.ChecksumFs.renameInternal(ChecksumFs.java:519)
        at 
org.apache.hadoop.fs.AbstractFileSystem.rename(AbstractFileSystem.java:807)
        at org.apache.hadoop.fs.FileContext.rename(FileContext.java:1044)
        at 
org.apache.spark.sql.execution.streaming.FileContextBasedCheckpointFileManager.renameTempFile(CheckpointFileManager.scala:372)
        at 
org.apache.spark.sql.execution.streaming.CheckpointFileManager$RenameBasedFSDataOutputStream.close(CheckpointFileManager.scala:154)
        at 
org.apache.spark.sql.execution.streaming.HDFSMetadataLog.write(HDFSMetadataLog.scala:204)
        at 
org.apache.spark.sql.execution.streaming.HDFSMetadataLog.addNewBatchByStream(HDFSMetadataLog.scala:237)
        at 
org.apache.spark.sql.execution.streaming.HDFSMetadataLog.add(HDFSMetadataLog.scala:130)
        at 
org.apache.spark.sql.execution.streaming.MicroBatchExecution.$anonfun$markMicroBatchEnd$1(MicroBatchExecution.scala:785)
   ```



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to