Jiabao-Sun commented on code in PR #23917:
URL: https://github.com/apache/flink/pull/23917#discussion_r1428188735


##########
flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingWithStateTestBase.scala:
##########
@@ -68,8 +69,8 @@ class StreamingWithStateTestBase(state: StateBackendMode) 
extends StreamingTestB
     super.before()
     // set state backend
 
-    // subfolder are managed here because the tests could fail during cleanup 
when concurrently executed (see FLINK-33820)
-    baseCheckpointPath = TempDirUtils.newFolder(tempFolder)
+    val baseCheckpointPath = 
Files.createTempDirectory(getClass.getCanonicalName)
+    Files.deleteIfExists(baseCheckpointPath);

Review Comment:
   ```suggestion
       baseCheckpointPath = Files.createTempDirectory("junit")
   ```
   
   
   I noticed that previous failed stack trace:
   ```
   Suppressed: java.nio.file.NoSuchFileException: 
/tmp/junit8233404746490819295/junit2880192188533757139/71dc52714210ccdbd137bbcffa7955b6/chk-3
                at 
sun.nio.fs.UnixException.translateToIOException(UnixException.java:86)
                at 
sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:102)
                at 
sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:107)
                at 
sun.nio.fs.UnixFileAttributeViews$Basic.readAttributes(UnixFileAttributeViews.java:55)
                at 
sun.nio.fs.UnixFileSystemProvider.readAttributes(UnixFileSystemProvider.java:144)
                at 
sun.nio.fs.LinuxFileSystemProvider.readAttributes(LinuxFileSystemProvider.java:99)
                at java.nio.file.Files.readAttributes(Files.java:1737)
                at 
java.nio.file.FileTreeWalker.getAttributes(FileTreeWalker.java:219)
                at java.nio.file.FileTreeWalker.visit(FileTreeWalker.java:276)
                at java.nio.file.FileTreeWalker.next(FileTreeWalker.java:372)
                at java.nio.file.Files.walkFileTree(Files.java:2706)
                at java.nio.file.Files.walkFileTree(Files.java:2742)
                at 
org.junit.jupiter.engine.extension.TempDirectory$CloseablePath.deleteAllFilesAndDirectories(TempDirectory.java:329)
                at 
org.junit.jupiter.engine.extension.TempDirectory$CloseablePath.close(TempDirectory.java:310)
                ... 96 more
                Suppressed: java.nio.file.NoSuchFileException: 
/tmp/junit8233404746490819295/junit2880192188533757139/71dc52714210ccdbd137bbcffa7955b6/chk-3
                        at 
sun.nio.fs.UnixException.translateToIOException(UnixException.java:86)
                        at 
sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:102)
                        at 
sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:107)
                        at 
sun.nio.fs.UnixFileSystemProvider.implDelete(UnixFileSystemProvider.java:244)
                        at 
sun.nio.fs.AbstractFileSystemProvider.delete(AbstractFileSystemProvider.java:103)
                        at java.nio.file.Files.delete(Files.java:1126)
                        at 
org.junit.jupiter.engine.extension.TempDirectory$CloseablePath$1.resetPermissionsAndTryToDeleteAgain(TempDirectory.java:382)
                        at 
org.junit.jupiter.engine.extension.TempDirectory$CloseablePath$1.visitFileFailed(TempDirectory.java:342)
                        at 
org.junit.jupiter.engine.extension.TempDirectory$CloseablePath$1.visitFileFailed(TempDirectory.java:329)
                        at java.nio.file.Files.walkFileTree(Files.java:2672)
                        ... 99 more
   ```
   
   The problem seems to have some clues. The failure to junit clean up 
`TempDir` may be caused by the attempt to delete it concurrently with the 
`CheckpointCoordinator` cleaning up completed Checkpoints.
   
   
https://github.com/apache/flink/blob/45f966e8c3c5e903b3843391874f7d2478122d8c/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java#L1387-L1390
   
   We can use `baseCheckpointPath = Files.createTempDirectory("junit")` to 
avoid using junit to clean up the `TempForlder`. This will be safe for 
upgrading junit to version 5.10.1.
   
   By the way, the root cause of the previous deletion failed is likely to be 
caused by the automatic cleaning complete checkpoint  by 
`Checkpointcoordinator`. I believe we can solve it soon. 🤔️



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

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

Reply via email to