Hi Rui,
Thanks for creating this FLIP and sorry for jumping in so late into the
discussion.

The improvements of exponential-delay strategy and making it the default
strategy looks good it me in general. I have some comments for it, as well
as for the failure counting.

1. default values of exponential-delay configuration
It is mentioned in the FLIP that "the default value of these options are
not changed, we just change them for default restart-strategy. The default
restart-strategy just takes effect if checkpointing is enabled and the
user doesn’t configure the restart-strategy". I'm a bit concerned with
the inconsistency which may cause confusion to users. So if the proposed
configuration values work better in most cases, I'm leaning towards to
change the default values.

2. the default tolerable failure count
Currently, the restart-strategy behaves like this by default(if not
configured):
1. job retries on failures indefinitely if checkpointing is enabled
2. job fails on any failure if checkpointing is disabled
Changing it to always restart indefinitely may result in unexpected
behaviors in production. Therefore, either we should do the same
thing to exponential-delay, or have an open discussion(also involving
the user ML) on changing this default behavior.

3. failure counting
Flink currently will try to recognize concurrent failures and group them
together, which can be seen in the web UI. So how about to align the
failure counting with the concurrent failures computing? This can make it
more consistent and easier for understanding. It will require changes to
the concurrent failures computing though, i.e. taking the backoff time
into consideration. So maybe we can open a seperate FLIP for this change.

Thanks,
Zhu

Rui Fan <1996fan...@gmail.com> 于2023年11月10日周五 18:22写道:

> I'll start voting next Monday if there isn't any other comment.
>
> Best,
> Rui
>
> On Thu, Oct 19, 2023 at 6:59 PM Rui Fan <1996fan...@gmail.com> wrote:
>
> > Hi Konstantin and Max,
> >
> > Thanks for your feedback!
> >
> > Sorry, I forgot to mention the default value of
> > `restart-strategy.exponential-delay.max-attempts-before-reset-backoff`.
> >
> > Retrying forever sounds good to me, I have added it to the FLIP:
> >
> > The default value of
> > `restart-strategy.exponential-delay.max-attempts-before-reset-backoff` is
> > Integer.MAX_VALUE.
> >
> > Best,
> > Rui
> >
> > On Thu, Oct 19, 2023 at 6:29 PM Maximilian Michels <m...@apache.org>
> wrote:
> >
> >> Hey Rui,
> >>
> >> +1 for making exponential backoff the default. I agree with Konstantin
> >> that retrying forever is a good default for exponential backoff
> >> because oftentimes the issue will resolve eventually. The purpose of
> >> exponential backoff is precisely to continue to retry without causing
> >> too much load. However, I'm not against adding an optional max number
> >> of retries.
> >>
> >> -Max
> >>
> >> On Thu, Oct 19, 2023 at 11:35 AM Konstantin Knauf <kna...@apache.org>
> >> wrote:
> >> >
> >> > Hi Rui,
> >> >
> >> > Thank you for this proposal and working on this. I also agree that
> >> > exponential back off makes sense as a new default in general. I think
> >> > restarting indefinitely (no max attempts) makes sense by default,
> >> though,
> >> > but of course allowing users to change is valuable.
> >> >
> >> > So, overall +1.
> >> >
> >> > Cheers,
> >> >
> >> > Konstantin
> >> >
> >> > Am Di., 17. Okt. 2023 um 07:11 Uhr schrieb Rui Fan <
> >> 1996fan...@gmail.com>:
> >> >
> >> > > Hi all,
> >> > >
> >> > > I would like to start a discussion on FLIP-364: Improve the
> >> > > restart-strategy[1]
> >> > >
> >> > > As we know, the restart-strategy is critical for flink jobs, it
> mainly
> >> > > has two functions:
> >> > > 1. When an exception occurs in the flink job, quickly restart the
> job
> >> > > so that the job can return to the running state.
> >> > > 2. When a job cannot be recovered after frequent restarts within
> >> > > a certain period of time, Flink will not retry but will fail the
> job.
> >> > >
> >> > > The current restart-strategy support for function 2 has some issues:
> >> > > 1. The exponential-delay doesn't have the max attempts mechanism,
> >> > > it means that flink will restart indefinitely even if it fails
> >> frequently.
> >> > > 2. For multi-region streaming jobs and all batch jobs, the failure
> of
> >> > > each region will increase the total number of job failures by +1,
> >> > > even if these failures occur at the same time. If the number of
> >> > > failures increases too quickly, it will be difficult to set a
> >> reasonable
> >> > > number of retries.
> >> > > If the maximum number of failures is set too low, the job can easily
> >> > > reach the retry limit, causing the job to fail. If set too high,
> some
> >> jobs
> >> > > will never fail.
> >> > >
> >> > > In addition, when the above two problems are solved, we can also
> >> > > discuss whether exponential-delay can replace fixed-delay as the
> >> > > default restart-strategy. In theory, exponential-delay is smarter
> and
> >> > > friendlier than fixed-delay.
> >> > >
> >> > > I also thank Zhu Zhu for his suggestions on the option name in
> >> > > FLINK-32895[2] in advance.
> >> > >
> >> > > Looking forward to and welcome everyone's feedback and suggestions,
> >> thank
> >> > > you.
> >> > >
> >> > > [1] https://cwiki.apache.org/confluence/x/uJqzDw
> >> > > [2] https://issues.apache.org/jira/browse/FLINK-32895
> >> > >
> >> > > Best,
> >> > > Rui
> >> > >
> >> >
> >> >
> >> > --
> >> > https://twitter.com/snntrable
> >> > https://github.com/knaufk
> >>
> >
>

Reply via email to