[ https://issues.apache.org/jira/browse/FLINK-31139?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17703787#comment-17703787 ]
Roman Khachatryan commented on FLINK-31139: ------------------------------------------- Yes [~Feifan Wang] , I think so. I've pushed to backport-branches to make sure the change doesn't break anything: 1.16: [https://dev.azure.com/khachatryanroman/flink/_build/results?buildId=1681&view=results] (PASS) 1.17: [https://dev.azure.com/khachatryanroman/flink/_build/results?buildId=1680&view=results] (FAIL) I'm going to re-run failed stages for 1.17 and merge if it passes. > not upload empty state changelog file > ------------------------------------- > > Key: FLINK-31139 > URL: https://issues.apache.org/jira/browse/FLINK-31139 > Project: Flink > Issue Type: Bug > Components: Runtime / State Backends > Affects Versions: 1.15.3, 1.16.1 > Reporter: Feifan Wang > Assignee: Feifan Wang > Priority: Major > Labels: pull-request-available > Fix For: 1.16.2, 1.17.1 > > Attachments: image-2023-02-20-19-51-34-397.png > > > h1. Problem > *_BatchingStateChangeUploadScheduler_* will upload many empty changelog files > (file size == 1 and only contains compressed flag). > !image-2023-02-20-19-51-34-397.png|width=1062,height=188! > These files are not referenced by any checkpoints, are not cleaned up, and > become more numerous as the job runs. Taking our big job as an example, 2292 > such files were generated within 7 hours. It only takes about 4 months and > the number of files in the changelog directory will exceed a million. > h1. Problem causes > This problem is caused by *_BatchingStateChangeUploadScheduler#drainAndSave_* > not checking whether the task collection is empty. The data in the scheduled > queue may have been uploaded when the > _*BatchingStateChangeUploadScheduler#drainAndSave*_ method is executed. > > So we should check whether the task collection is empty in > *_BatchingStateChangeUploadScheduler#drainAndSave_* . WDYT [~roman] , > [~Yanfei Lei] ? -- This message was sent by Atlassian Jira (v8.20.10#820010)