Chesnay, David:

Thank you guys for the extra information. We were clearly missing some
context here around the scheduler related efforts and the currently
available feature set.

As for the concrete suggestions regarding the docs.

1. If the adaptive scheduler provides a significantly different feature set
from the default scheduler we could have its own smaller doc page detailing
the differences and why people should switch to it for streaming. This will
also help us when we are making the transition and change the default
behaviour.
2. We could still have an elastic scaling page that links to the adaptive
scheduler (and vice versa) that focuses on elastic scaling + the Kubernetes
operator autoscaler for a complete picture on elastic scaling options +
detailing the limitations of the different approaches.

This way the Adaptive Scheduler docs will be decoupled from elastic scaling
and will result in a better understanding for the users (it sure would have
helped us here, and we are on the more advanced user side :))

What do you think?
Gyula

On Sat, Jan 28, 2023 at 4:20 AM ConradJam <jam.gz...@gmail.com> wrote:

> Sorry I'm late to join discuss, I've gleaned a lot of useful information
> from you guys
>
> *@max*
>
>    - when user repartition, we still need to restart the job, can we try to
>    do this part of the work internally instead of externally, as
>    *@konstantin* said only trigger rescaling when the checkpoint or
>    retain-checkpoint is completed operations to minimize reprocessing
>
> *@konstantin*
>
>    - I think you mentioned that 2 FLIPs are being drafted which I consider
>    to be the condition to achieve the *@max* goal, I would love to join
>    this discussion and contribute it. I've tried a native implementation of
>    this part myself, if I can help the community that's the best I can do
>
> *@chesnay*
>
>    - The docs section is confusion/misconceptions confusing like *@gyula
> *say,
>    I'll see if I can fix it
>
>
> *About Rescale Api*
>
>   Some limitations and differences between *default* and *reactive mode*
> were
> discussed earlier, and *@chesnay* explained some of their limitations and
> behaviors, essentially they are two different things. I agree that when
> reactive mode is ready, it should be used as the *reactive mode* for the
> default *stream processing* job.
>   As for the *[1] **Rescale API*, as we know now it seems to be unusable, I
> believe the goal of this api is to be able to do fast reparallelism. I
> would like to wait until the discussion is over and the 2 draft FILPs
> mentioned earlier are completed. It is not too late to make another
> decision on whether to modify the *[2] **Rescale Rest API *to support for
> parallelism modification of job vertices
>
>
>    1.
> *
> https://nightlies.apache.org/flink/flink-docs-release-1.16/docs/deployment/elastic_scaling/
>    <
> https://nightlies.apache.org/flink/flink-docs-release-1.16/docs/deployment/elastic_scaling/
> >
>    *
>    2.
> *
> https://nightlies.apache.org/flink/flink-docs-master/docs/ops/rest_api/#jobs-jobid-rescaling
>    <
> https://nightlies.apache.org/flink/flink-docs-master/docs/ops/rest_api/#jobs-jobid-rescaling
> >
>    *
>
>
> Best~
>
>
>
> Maximilian Michels <m...@apache.org> 于2023年1月24日周二 01:08写道:
>
> > Hi,
> >
> > The current rescale API appears to be a work in progress. A couple years
> > ago, we disabled access to the API [1].
> >
> > I'm looking into this problem as part of working on autoscaling [2] where
> > we currently require a full restart of the job to apply the parallelism
> > overrides. This adds additional delay and comes with the caveat that we
> > don't know whether sufficient resources are available prior to executing
> > the scaling decision. We obviously do not want to get stuck due to a lack
> > of resources. So a rescale API would have to ensure enough resources are
> > available prior to restarting the job.
> >
> > I've created an issue here:
> > https://issues.apache.org/jira/browse/FLINK-30773
> >
> > Any comments or interest in working on this?
> >
> > -Max
> >
> > [1] https://issues.apache.org/jira/browse/FLINK-12312
> > [2]
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-271%3A+Autoscaling
> >
>
>
> --
> Best
>
> ConradJam
>

Reply via email to