http://deprecated-apache-flink-mailing-list-archive.368.s1.nabble.com/DISCUSS-FLIP-169-DataStream-API-for-Fine-Grained-Resource-Requirements-tp51071p51213.html
default resource allocation strategy.
`ResourceAllocationStrategy`. Currently, with its default
the config option as well. So, in your case, you need to define the
"external-resources" and "external-resources.disk.amount". Then, all
there is no slot level isolation. If the disk space of a task manager
cannot fulfill the disk requirement, RM will allocate a new one.
which allocates heterogeneous TMs according to the requirements. Then,
needed.
resources it requires in the future. But that is out of the scope of
this PR.
>
> Thanks @Yangze for preparing this FLIP.
>
> I think this is a good start point for the community users to have a taste
> on the fine-grained
> resource management, which we all believe it could improve the Flink job
> stability and
> cluster utilization.
>
> I have a simple question about the extended resources. It is possible to
> combine extended resources
> with fine-grained resource management. Except for the GPU, FPGA and other
> new computing devices,
> maybe the disk resource is a more general use case. For example, different
> SSG may have various
> disk requirements based on the state. So we need to allocate enough
> ephemeral storage resource for every
> TaskManager pod in Kubernetes deployment. Otherwise, it might be evicted
> due to running out of limits.
>
>
> Best,
> Yang
>
>
> Xintong Song <
[hidden email]> 于2021年6月8日周二 下午1:47写道:
>
> > I think being able to specify fine grained resource requirements without
> > having to change the codes and recompile the job is indeed a good idea. It
> > definitely improves the usability.
> >
> > However, this requires more careful designs, which probably deserves a
> > separate thread. I'd be good to have that discussion, but maybe not block
> > this feature on that.
> >
> > One idea concerning the configuration approach: As Yangze said, flink
> > configuration options are supposed to take effect at cluster level. For
> > updating job level specifics that are not suitable to be introduced as a
> > config option, currently the only way is to pass them as program arguments.
> > Would it make sense to introduce a general approach for overwriting such
> > job specifics without re-compiling the job?
> >
> > Thank you~
> >
> > Xintong Song
> >
> >
> >
> > On Tue, Jun 8, 2021 at 1:23 PM Yangze Guo <
[hidden email]> wrote:
> >
> > > @Wenlong
> > > After another consideration, the config option approach I mentioned
> > > above might not be appropriate. The resource requirements for SSG
> > > should be a job level configuration and should no be set in the
> > > flink-conf.
> > >
> > > I think we can define a JSON format, which would be the ResourceSpecs
> > > mapped by the name of SSGs, for the resource requirements of a
> > > specific job. Then, we allow user to configure the file path of that
> > > JSON. The JSON will be only parsed in runtime, which allows user to
> > > tune it without re-compiling the job.
> > >
> > > We can add another #setSlotSharingGroupResources for configuring the
> > > file path of that JSON:
> > > ```
> > > /**
> > > * Specify fine-grained resource requirements for slot sharing groups
> > > with the given resource JSON file. The existing resource
> > > * requirement of the same slot sharing group will be replaced.
> > > */
> > > public StreamExecutionEnvironment setSlotSharingGroupResources(
> > > String pathToResourceJson);
> > > ```
> > >
> > > WDYT?
> > >
> > > Best,
> > > Yangze Guo
> > >
> > > On Tue, Jun 8, 2021 at 12:12 PM Yangze Guo <
[hidden email]> wrote:
> > > >
> > > > Thanks for the feedbacks, Xintong and Wenlong!
> > > >
> > > > @Wenlong
> > > > I think that is a good idea, adjust the resource without re-compiling
> > > > the job will facilitate the tuning process.
> > > > We can define a pattern "slot-sharing-group.resource.{ssg name}"
> > > > (welcome any proposal for the prefix naming) for the resource spec
> > > > config of a slot sharing group. Then, user can set the ResourceSpec of
> > > > SSG "ssg1" by adding "slot-sharing-group.resource.ssg1: {cpu: 1.0,
> > > > heap: 100m, off-heap: 100m....}". WDYT?
> > > >
> > > >
> > > > Best,
> > > > Yangze Guo
> > > >
> > > > On Tue, Jun 8, 2021 at 10:37 AM wenlong.lwl <
[hidden email]>
> > > wrote:
> > > > >
> > > > > Thanks Yangze for the flip, it is great for users to be able to
> > > declare the
> > > > > fine-grained resource requirements for the job.
> > > > >
> > > > > I have one minor suggestion: can we support setting resource
> > > requirements
> > > > > by configuration? Currently most of the config options in execution
> > > config
> > > > > can be configured by configuration, and it is very likely that users
> > > need
> > > > > to adjust the resource according to the performance of their job
> > during
> > > > > debugging, Providing a configuration way will make it more
> > convenient.
> > > > >
> > > > > Bests,
> > > > > Wenlong Lyu
> > > > >
> > > > > On Thu, 3 Jun 2021 at 15:59, Xintong Song <
[hidden email]>
> > > wrote:
> > > > >
> > > > > > Thanks Yangze for preparing the FLIP.
> > > > > >
> > > > > > The proposed changes look good to me.
> > > > > >
> > > > > > As you've mentioned in the implementation plan, I believe one of
> > the
> > > most
> > > > > > important tasks of this FLIP is to have the feature well
> > documented.
> > > It
> > > > > > would be really nice if we can keep that in mind and start drafting
> > > the
> > > > > > documentation early.
> > > > > >
> > > > > > Thank you~
> > > > > >
> > > > > > Xintong Song
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Thu, Jun 3, 2021 at 3:13 PM Yangze Guo <
[hidden email]>
> > > wrote:
> > > > > >
> > > > > > > Hi, there,
> > > > > > >
> > > > > > > We would like to start a discussion thread on "FLIP-169:
> > DataStream
> > > > > > > API for Fine-Grained Resource Requirements"[1], where we propose
> > > the
> > > > > > > DataStream API for specifying fine-grained resource requirements
> > in
> > > > > > > StreamExecutionEnvironment.
> > > > > > >
> > > > > > > Please find more details in the FLIP wiki document [1]. Looking
> > > > > > > forward to your feedback.
> > > > > > >
> > > > > > > [1]
> > > > > > >
> > > > > >
> > >
> >
https://cwiki.apache.org/confluence/display/FLINK/FLIP-169+DataStream+API+for+Fine-Grained+Resource+Requirements> > > > > > >
> > > > > > >
> > > > > > > Best,
> > > > > > > Yangze Guo
> > > > > > >
> > > > > >
> > >
> >