Rui Fan created FLINK-30461:
-------------------------------
Summary: Some rocksdb sst files will remain forever
Key: FLINK-30461
URL: https://issues.apache.org/jira/browse/FLINK-30461
Project: Flink
Issue Type: Bug
Components: Runtime / Checkpointing
Affects Versions: 1.15.3, 1.16.0, 1.17.0
Reporter: Rui Fan
Attachments: image-2022-12-20-18-45-32-948.png,
image-2022-12-20-18-47-42-385.png
In rocksdb incremental checkpoint mode, during file upload, if some files have
been uploaded and some files have not been uploaded, the checkpoint timeout is
canceled at this time, and the uploaded files will remain.
h2. Impact:
The shared directory of a flink job has more than 1 million files. It exceeded
the hdfs upper limit, causing new files not to be written.
However only 50k files are available, the other 950k files should be cleaned up.
!https://user-images.githubusercontent.com/38427477/207588272-dda7ba69-c84c-4372-aeb4-c54657b9b956.png|width=1962,height=364!
h2. Root cause:
If an exception is thrown during the checkpoint async phase, flink will clean
up metaStateHandle, miscFiles and sstFiles.
However, when all sst files are uploaded, they are added together to sstFiles.
If some sst files have been uploaded and some sst files are still being
uploaded, and the checkpoint timeout is canceled at this time, all sst files
will not be added to sstFiles. The uploaded sst will remain on hdfs.
[code
link|https://github.com/apache/flink/blob/49146cdec41467445de5fc81f100585142728bdf/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksIncrementalSnapshotStrategy.java#L328]
h2. Solution:
Using the CloseableRegistry as the tmpResourcesRegistry. If the async phase is
failed, the tmpResourcesRegistry will cleanup these temporary resources.
POC code:
https://github.com/1996fanrui/flink/commit/86a456b2bbdad6c032bf8e0bff71c4824abb3ce1
!image-2022-12-20-18-45-32-948.png|width=1114,height=442!
!image-2022-12-20-18-47-42-385.png|width=1332,height=552!
--
This message was sent by Atlassian Jira
(v8.20.10#820010)