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

Reply via email to