Thank you Matthias and all for the feedback and suggestions.

Please let me make a brief summary based on the historical comments:
- It's agreeded to optimize/fix this issue in the 1.x TLS versions.
- The primary goal of this optimization/fix is to minimize the number of 
TaskManagers used in application mode.
- The optimized logic should be controlled via a parameter.

I'd like to introduce the following parameter to control whether the optimized 
logic should be enabled:
- Name: jobmanager.adaptive-scheduler.resource.minimal-taskmanagers-preferred
- Type: boolean
- Default value: false
- Description: This parameter defines whether the adaptive scheduler 
prioritizes 
using the minimum number of TaskManagers when scheduling tasks.
Note: This parameter is currently suitable for cases that 
execution.state-recovery.from-local is disabled.'

BTW, I'm uncertain whether the introduction of a parameter for this specific 
fix necessitates documentation via a FLIP.
If so, I'm willing to initiate a FLIP to aid in subsequent tasks.
If not, I will add this email address to the corresponding jira ticket's 
comments for tracking  and start the work on MR.

Any suggestion would be appreciated!

Thank you!

Best,
Yuepeng Pan

On 2025/01/05 18:41:11 Matthias Pohl wrote:
> Hi everyone and sorry for the late reply. I was mostly off in November and
> forgot about that topic in December last year.
> 
> Thanks for summarizing and bringing up user feedback. I see the problem and
> agree with your view that it's a topic that we might want to address in the
> 1.x LTS version. I see how this can be labeled as a bug or a feature
> depending on the perspective. I think adding this behavior while being
> guarded by a feature flag/configuration parameter in the 1.x LTS version is
> reasonable.
> 
> Best,
> Matthias
> 
> [1]
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158873338#FLIP138:DeclarativeResourcemanagement-Howtodistributeslotsacrossdifferentjobs
> 
> On Wed, Nov 6, 2024 at 9:21 AM Rui Fan <1996fan...@gmail.com> wrote:
> 
> > Thanks Yuepeng for the PR and starting this discussion!
> >
> > And thanks Gyula and Yuanfeng for the input!
> >
> > I also agree to fix this behaviour in the 1.x line.
> >
> > The adaptive scheduler and rescaling API provide powerful capabilities to
> > increase or decrease parallelism.
> >
> > The main benefit I understand of decreasing parallelism is saving
> > resources.
> > If decreasing parallelism can't save resources, why do users decrease it?
> > This is why I think releasing TM resources when decreasing parallelism is
> > a basic capability that the Adaptive Scheduler should have.
> >
> > Please correct me if I miss anything, thanks~
> >
> > Also, I believe it does not work as the user expects. Because this
> > behaviour
> > was reported multiple times in the flink community, such as:
> > FLINK-33977[1],
> > FLINK-35594[2], FLINK-35903[3] and Slack channel[4].
> > And 1.20.x is a LTS version, so I agree to fix it in the 1.x line.
> >
> > [1] https://issues.apache.org/jira/browse/FLINK-33977
> > [2] https://issues.apache.org/jira/browse/FLINK-35594
> > [3] https://issues.apache.org/jira/browse/FLINK-35903
> > [4] https://apache-flink.slack.com/archives/C03G7LJTS2G/p1729167222445569
> >
> > Best,
> > Rui
> >
> > On Wed, Nov 6, 2024 at 4:15 PM yuanfeng hu <yuanf...@apache.org> wrote:
> >
> >> > Is it considered an error if the adaptive scheduler fails to release the
> >> task manager during scaling?
> >>
> >> +1 . When we enable adaptive mode and perform scaling operations on tasks,
> >> a significant part of the goal is to reduce resource usage for the tasks.
> >> However, due to some logic in the adaptive scheduler's scheduling process,
> >> the task manager cannot be released, and the ultimate goal cannot be
> >> achieved. Therefore, I consider this to be a mistake.
> >>
> >> Additionally, many tasks are currently running in this mode and will
> >> continue to run for quite a long time (many users are in this situation).
> >> So whether or not it is considered a bug, I believe we need to fix it in
> >> the 1.x version.
> >>
> >> Yuepeng Pan <panyuep...@apache.org> 于2024年11月6日周三 14:32写道:
> >>
> >> > Hi, community.
> >> >
> >> >
> >> >
> >> >
> >> > When working on ticket[1] we have received some lively discussions and
> >> > valuable
> >> > feedback[2](thanks for Matthias, Rui, Gyula, Maximilian, Tison, etc.),
> >> the
> >> > main issues are that:
> >> >
> >> > When the job runs in an application cluster, could the default behavior
> >> of
> >> > AdaptiveScheduler not actively releasing Taskmanagers resources during
> >> > downscaling be considered a bug?
> >> >
> >> > If so,should we fix it in flink 1.x?
> >> >
> >> >
> >> >
> >> > I’d like to start a discussion to hear more comments about it to define
> >> > the next step and I have sorted out some information in the doc[3]
> >> > regarding this discussion for you.
> >> >
> >> >
> >> >
> >> > Looking forward to your comments and attention.
> >> >
> >> > Thank you.
> >> >
> >> > Best,
> >> > Yuepeng Pan
> >> >
> >> >
> >> >
> >> >
> >> > [1] https://issues.apache.org/jira/browse/FLINK-33977
> >> >
> >> > [2] https://github.com/apache/flink/pull/25218#issuecomment-2401913141
> >> >
> >> > [3]
> >> >
> >> https://docs.google.com/document/d/1Rwwl2aGVz9g5kUJFMP5GMlJwzEO_a-eo4gPf7gITpdw/edit?tab=t.0#heading=h.s4i4hehbbli5
> >> >
> >> >
> >> >
> >>
> >> --
> >> Best,
> >> Yuanfeng
> >>
> >
> 

Reply via email to