Hi Mayue, Hangxiangyu and Piotr, Sorry for the late reply and thanks a lot for your feedback. After careful thoughts, I agree with your opinions in following ways:
1, Instead of introducing a specific retry mechanism for RocksDB data transfer, we should introduce a more fine-grained retry mechanism for checkpointing when interacting with external storage to improve the overall success rate; 2, This fine-grained retry mechanism should work with all kinds of state implementations (OperatorStateBackend、KeyedStateBackend); 3, Currently, there are several Retry Implementations in Flink Code base: - `RetryStrategy` and `FutureUtils` under `flink-core` module for common usage - `AsyncRetryStrategy` and `RetryableResultHandlerDelegator` under datastream-api and used for async operators - `RetryPolicy` and `RetryingExecutor` under `flink-dstl` module and used for ChangelogStateBackend IMHO, these implementations are almost interchangeable with few variations. Unifying all the Retry implementations might be another big topic to discuss, but I do agree that the retry tools used by all StateBackend should be consistent. 4,The fine-grained retry mechanism should also consider different external storage exceptions, the retry action should not be performed for a permanent failure. AFAIK, this can be achieved by a customized retry predicate. According to the suggestions mentioned above, I will rework FLIP-414[1] and continue this discussion after that. Thx again for your valuable options~ [1] https://cwiki.apache.org/confluence/display/FLINK/FLIP-414%3A+Support+Retry+Mechanism+in+RocksDBStateDataTransfer Best Regards, Xiangyu Feng Piotr Nowojski <pnowoj...@apache.org> 于2024年1月12日周五 01:39写道: > Hi, > > Thanks for the proposal. I second the Hangxiang's suggestions. > > I think this might be valuable. Instead of retrying the whole checkpoint, > it will be more resource efficient > to retry upload of a single file. > > Regarding re-using configuration options, a while back we introduced > `taskmanager.network.retries` > config option. It was hoped to eventually encompass things like this. > > My own concern is if we should retry regardless of the exception type, or > should we focus on things like > connection loss/host unreachable? All in all, it would be better to not > retry upload if the failure was: > - `FileSystem` for given schema not found > - authorisation failed > - lack of write rights > - ... > > Best, > Piotrek > > > > > czw., 11 sty 2024 o 10:35 Hangxiang Yu <master...@gmail.com> napisał(a): > > > Thanks for driving this. > > Retry mechanism is common when we want to get or put data by network. > > So I think it will help when checkpoint failure due to temporary network > > problems, of course it may increase a bit overhead for some other > reasons. > > > > Some comments and suggestions: > > 1. Since Flink has a checkpoint mechanism to retry failed checkpoint > > coarsely, I think it looks good to me if this fine-grained retry could be > > configurable and don't change the current default mechanism. > > 2. This should work with the checkpoint procedure of all state backends, > > Could we make this config unrelated to a specific state backend (maybe > > execution.checkpointing.xxx)? Then it could be supported by below state > > backends. > > 3. We may not need to re-implement it. There are some tools supporting > the > > Retry mechanism (see RetryingExecutor and RetryPolicy in changelog dstl > > module), it's better to make them become more common tools and reuse > them. > > > > On Thu, Jan 11, 2024 at 3:09 PM yue ma <mayuefi...@gmail.com> wrote: > > > > > Thanks for driving this effort, xiangyu! > > > The proposal overall LGTM. > > > I just have a small question. There are other places in Flink that > > interact > > > with external storage. Should we consider adding a general retry > > mechanism > > > to them? > > > > > > xiangyu feng <xiangyu...@gmail.com> 于2024年1月8日周一 11:31写道: > > > > > > > Hi devs, > > > > > > > > I'm opening this thread to discuss FLIP-414: Support Retry Mechanism > in > > > > RocksDBStateDataTransfer[1]. > > > > > > > > Currently, there is no retry mechanism for downloading and uploading > > > > RocksDB state files. Any jittering of remote filesystem might lead > to a > > > > checkpoint failure. By supporting retry mechanism in > > > > `RocksDBStateDataTransfer`, we can significantly reduce the failure > > rate > > > of > > > > checkpoint during asynchronous phrase. > > > > > > > > To make this retry mechanism configurable, we have introduced two > > options > > > > in this FLIP: `state.backend.rocksdb.checkpoint.transfer.retry.times` > > > and ` > > > > state.backend.rocksdb.checkpoint.transfer.retry.interval`. The > default > > > > behavior remains to be no retry will be performed in order to be > > > consistent > > > > with the original behavior. > > > > > > > > Looking forward to your feedback, thanks. > > > > > > > > [1] > > > > > > > > > > > > > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-414%3A+Support+Retry+Mechanism+in+RocksDBStateDataTransfer > > > > > > > > Best regards, > > > > Xiangyu Feng > > > > > > > > > > > > > -- > > > Best, > > > Yue > > > > > > > > > -- > > Best, > > Hangxiang. > > >