Thank you for your time, Gyula, I have more question about Flink-33548, we can have more discussion about this and make progress:
1. I agree with you about declaring resources in FlinkDeployment resource sections. But Flink Operator will override the “jobmanager.memory.process.size” and "taskmanager.memory.process.size", despite I have set these configuration or not in flink configuration. If user had configured all memory attributes, the override will leads to error as the overall computation is error. the code of override is in FlinkConfigManager.class in buildFrom method, which apply to JobmanagerSpec and TaskManagerSpec. 2. If I modified the code of override, I will still encounter this issue of FLINK-24150, because I only modified the code of flink operator but not flink-kubernetes package, so I will make a pod resources like (cpu:1c memory:1g) and container resource to be (cpu:1c, memory 850m), because I already set jobmanager.memory.process.size to 850m. 3. because of there two point, we need to make the podTemplate have higher priority. Otherwise we can refactor the code of flink operator, which should import something new configuration to support the native mode. I think it will be better to import some configuration, which FlinkConfigManager.class can override it using the resource of JobmanagerSpec and TaskManagerSpec. When it deep into the code flink-kubernetes package, we using these new configuration as the final result of containers resources. Thanks for your time. Richard Su > 2023年12月5日 16:45,Gyula Fóra <gyula.f...@gmail.com> 写道: > > As you can see in the jira ticket there hasn't been any progress, nobody > started to work on this yet. > > I personally don't think it's confusing to declare resources in the > FlinkDeployment resource sections. It's well documented and worked very > well so far for most users. > This is pretty common practice for kubernetes. > > Cheers, > Gyula > > On Tue, Dec 5, 2023 at 9:35 AM richard.su <richardsuc...@gmail.com> wrote: > >> Hi, Gyula, is there had any progress in FLINK-33548? I would like to join >> the discussion but I haven't seen any discussion in the url. >> >> I also make flinkdeployment by flink operator, which indeed will override >> the process size by TaskmanagerSpec.resources or JobmanagerSpec.resources, >> which really confused, I had modified the code of flink operator to avoid >> the override. >> >> Looking for your response. >> >> Thank you. >> Richard Su >> >> >>> 2023年12月5日 16:22,Gyula Fóra <gyula.f...@gmail.com> 写道: >>> >>> Hi! >>> >>> Please see the discussion in >>> https://lists.apache.org/thread/6p5tk6obmk1qxf169so498z4vk8cg969 >>> and the ticket: https://issues.apache.org/jira/browse/FLINK-33548 >>> >>> We should follow the approach outlined there. If you are interested you >> are >>> welcome to pick up the operator ticket. >>> >>> Unfortunately your PR can be a large unexpected change to existing users >> so >>> we should not add it. >>> >>> Cheers, >>> Gyula >>> >>> On Tue, Dec 5, 2023 at 9:05 AM 苏超腾 <richardsuc...@gmail.com> wrote: >>> >>>> Hello everyone, >>>> >>>> I've encountered an issue while using flink kubernetes native, Despite >>>> setting resource limits in the pod template, it appears that these >> limits >>>> and requests are not considered during JobManager(JM) and TaskManager >> (TM) >>>> pod deployment. >>>> >>>> I find the a issue had opened in jira FLINK-24150, which introduced >>>> almost the same questions that I encountered. >>>> >>>> I agrees that if user had provided pod templates, we should put priority >>>> on it higher than flink calculated from configuration. >>>> >>>> But this need some discussion in our community, because it related some >>>> scenarios: >>>> If I want to create a pod with Graranted QoS and want the memory of the >>>> Flink main container to be larger than the process size of Flink, I >> cannot >>>> directly modify podTemplate (although we can use limit factor, this will >>>> cause the QoS to change from Graranted to Burstable) >>>> If I want to create a pod with Burstable QoS, I don't want to use limit >>>> actor and want to directly configure the request to be 50% of the limit, >>>> which cannot be modified. >>>> In order to meet these scenarios, I had committed a pull request >>>> https://github.com/apache/flink/pull/23872 >>>> >>>> This code is very simple and just need someone to review, this pr can be >>>> cherry pick to other old version, which will be helpful. >>>> >>>> >>>> I would appreciate any feedback on this. >>>> >>>> Thank you for your time and contributions to the Flink project. >>>> >>>> Thank you, >>>> chaoran.su >> >>