Re: [DISCUSS] Change the default restart-strategy to exponential-delay

2023-12-11 文章 Rui Fan
Hi Maximilian and Mason,

Thanks a lot for your feedback!

After an offline consultation with Max, I guess I understand your
concern for now: when flink job restarts, it will make a bunch of
calls to the Kubernetes API, e.g. read/write to config maps, create
task managers. Currently, the default restart strategy is fixed-delay
with 1s delay time, so flink will restart jobs with high frequency
even if flink jobs cannot be started. It will cause the Kubernetes
cluster became unstable.

That's why I propose changing the default restart strategy to
exponential-delay. It can achieve: restarts happen quickly
enough unless there are consecutive failures. It is helpful for
the stability of external components.

After discussing with Max and Zhu Zhu at the PR comment[1],
Max suggested using 1.5 as the default value of backoff-multiplier
instead of 1.2. The 1.2 is a little small(delay time is too short).
This picture[2] is the relationship between restart-attempts and
retry-delay-time when backoff-multiplier is 1.2 and 1.5:

- The delay-time will reach 1 min after 12 attempts when backoff-multiplier
is 1.5
- The delay-time will reach 1 min after 24 attempts when backoff-multiplier
is 1.2

Is there any other suggestion? Looking forward to more feedback, thanks~

BTW, as Zhu said in the comment[1], if we update the default value,
a new vote is needed for this default value. So I will pause
FLINK-33736[1] first, and the rest of the JIRAs of FLIP-364 will be
continued.

To Mason:

If I understand your concerns correctly, I still don't know how
to benchmark. The kubernetes cluster instability only happens
when one cluster has a lot of jobs. In general, the test cannot
reproduce the pressure. Could you elaborate on how to
benchmark for this?

After this FLIP, the default restart frequency will be reduced
significantly. Especially when a job fails consecutively.
Do you think the benchmark is necessary?

Looking forward to your feedback, thanks~

[1] https://github.com/apache/flink/pull/23247#discussion_r1422626734
[2]
https://github.com/apache/flink/assets/38427477/642c57e0-b415-4326-af05-8b506c5fbb3a
[3] https://issues.apache.org/jira/browse/FLINK-33736

Best,
Rui

On Thu, Dec 7, 2023 at 10:57 PM Maximilian Michels  wrote:

> Hey Rui,
>
> +1 for changing the default restart strategy to exponential-delay.
> This is something all users eventually run into. They end up changing
> the restart strategy to exponential-delay. I think the current
> defaults are quite balanced. Restarts happen quickly enough unless
> there are consecutive failures where I think it makes sense to double
> the waiting time up till the max.
>
> -Max
>
>
> On Wed, Dec 6, 2023 at 12:51 AM Mason Chen  wrote:
> >
> > Hi Rui,
> >
> > Sorry for the late reply. I was suggesting that perhaps we could do some
> > testing with Kubernetes wrt configuring values for the exponential
> restart
> > strategy. We've noticed that the default strategy in 1.17 caused a lot of
> > requests to the K8s API server for unstable deployments.
> >
> > However, people in different Kubernetes setups will have different limits
> > so it would be challenging to provide a general benchmark. Another thing
> I
> > found helpful in the past is to refer to Kubernetes--for example, the
> > default strategy is exponential for pod restarts and we could draw
> > inspiration from what they have set as a general purpose default config.
> >
> > Best,
> > Mason
> >
> > On Sun, Nov 19, 2023 at 9:43 PM Rui Fan <1996fan...@gmail.com> wrote:
> >
> > > Hi David and Mason,
> > >
> > > Thanks for your feedback!
> > >
> > > To David:
> > >
> > > > Given that the new default feels more complex than the current
> behavior,
> > > if we decide to do this I think it will be important to include the
> > > rationale you've shared in the documentation.
> > >
> > > Sounds make sense to me, I will add the related doc if we
> > > update the default strategy.
> > >
> > > To Mason:
> > >
> > > > I suppose we could do some benchmarking on what works well for the
> > > resource providers that Flink relies on e.g. Kubernetes. Based on
> > > conferences and blogs,
> > > > it seems most people are relying on Kubernetes to deploy Flink and
> the
> > > restart strategy has a large dependency on how well Kubernetes can
> scale to
> > > requests to redeploy the job.
> > >
> > > Sorry, I didn't understand what type of benchmarking
> > > we should do, could you elaborate on it? Thanks a lot.
> > >
> > > Best,
> > > Rui
> > >
> > > On Sat, Nov 18, 2023 at 3:32 AM Mason Chen 
> wrote:
> > >
> > >> Hi Rui,
> > >>
> > >> I suppose we could do some benchmarking on what works well for the
> > >> resource providers that Flink relies on e.g. Kubernetes. Based on
> > >> conferences and blogs, it seems most people are relying on Kubernetes
> to
> > >> deploy Flink and the restart strategy has a large dependency on how
> well
> > >> Kubernetes can scale to requests to redeploy the job.
> > >>
> > >> Best,
> > >> Mason
> > >>
> > >> On 

退订

2023-12-11 文章 RoyWilde
退订