llllssss94 opened a new pull request, #27423:
URL: https://github.com/apache/flink/pull/27423

   ## What is the purpose of the change
   
   This pull request fixes a critical bug 
([FLINK-38909](https://issues.apache.org/jira/browse/FLINK-38909)) that causes 
checkpoint cleanup to fail with a `PathIsNotEmptyDirectoryException`. The root 
cause was an incorrect, non-recursive delete call on a checkpoint's storage 
location, which by design contains multiple files.
   
   A completed Flink checkpoint always consists of multiple data files and a 
metadata file, grouped under a common path (`exclusiveCheckpointDir`). This 
logical location is never empty. Attempting to delete it with a non-recursive 
`delete(path, false)` command is fundamentally incorrect and guaranteed to fail 
on any compliant file system. This bug leads to orphaned checkpoint data and 
storage leaks.
   
   This fix corrects the logic by using a recursive delete, ensuring that all 
files and objects associated with a checkpoint's location are properly removed, 
regardless of the underlying filesystem's architecture.
   
   ## Brief change log
   
   *   In `FsCompletedCheckpointStorageLocation.disposeStorageLocation()`, the 
filesystem call was changed to `fs.delete(exclusiveCheckpointDir, true)`. This 
enables recursive deletion, ensuring the entire directory tree of a checkpoint 
is properly removed.
   
   ## Verifying this change
   
   This change added tests and can be verified as follows:
   
   *   **Added a new test case to `FsCompletedCheckpointStorageLocationTest`** 
to specifically reproduce the bug and validate the fix. This test simulates a 
real, non-empty checkpoint by creating a storage location with subdirectories 
and files. It then calls the `disposeStorageLocation()` method and asserts that 
no exception is thrown and the location is completely removed.
   
   ## Does this pull request potentially affect one of the following parts:
   
   *   Dependencies (does it add or upgrade a dependency): **no**
   *   The public API, i.e., is any changed class annotated with 
`@Public(Evolving)`: **no**
   *   The serializers: **no**
   *   The runtime per-record code paths (performance sensitive): **no**
   *   Anything that affects deployment or recovery: JobManager (and its 
components), Checkpointing, Kubernetes/Yarn, ZooKeeper: **yes** (This directly 
impacts the correctness of the checkpoint cleanup lifecycle.)
   *   The S3 file system connector: **yes** (While the fix is in core Flink, 
the bug is most frequently observed on object storage systems, and this change 
ensures correct behavior on them.)
   
   ## Documentation
   
   *   Does this pull request introduce a new feature? **no**
   *   If yes, how is the feature documented? **not applicable**


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

Reply via email to