Re: Discussion: [FLINK-24150] Support to configure cpu resource request and limit in pod template

2023-12-05 Thread richard.su
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  写道:
> 
> 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 苏超腾  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



Re: Discussion: [FLINK-24150] Support to configure cpu resource request and limit in pod template

2023-12-05 Thread richard.su
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  写道:
> 
> 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  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  写道:
>>> 
>>> 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 苏超腾  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
>> 
>> 



Re: Discussion: [FLINK-24150] Support to configure cpu resource request and limit in pod template

2023-12-05 Thread richard.su
Sorry Gyula,  let me explain more about the point of 2, if I avoid the 
override, I will got a jobmanager pod still with resources consist with 
“jobmanager.memory.process.size”, but a flinkdeployment with a resource larger 
than that.

Thanks for your time.
Richard Su

> 2023年12月5日 17:13,richard.su  写道:
> 
> 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  写道:
>> 
>> 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  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  写道:
>>>> 
>>>> 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 苏超腾  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 w

Re: Discussion: [FLINK-24150] Support to configure cpu resource request and limit in pod template

2023-12-05 Thread richard.su
Hi, Gyula, from my opinion, this still will using flinkDeployment's resource 
filed to set jobManager.memory.process.size, and I have told an uncovered case 
that:

When user wants to define a flinkdeployment with jobmanager has 1G memory 
resources in container field but config jobmanager.memory.process.size as 850m, 
which this solution only improves user config and actually make sconfig more 
intuitive and easier but not make the container resource decoupling flink 
configuration.

So from my side, I think it need to add new configuration to support this 
proposal, and it need more discussion.

Thanks
Chaoran Su


> 2023年12月5日 18:28,Gyula Fóra  写道:
> 
> This is the proposal according to FLINK-33548:
> 
> spec:
>  taskManager:
>resources:
>  requests:
>memory: "64Mi"
>cpu: "250m"
>  limits:
>memory: "128Mi"
>cpu: "500m"
> 
> I honestly think this is much more intuitive and easier than using the
> podTemplate, which is very complex immediately.
> Please tell me what use-case/setup is not covered by this improved spec.
> 
> Unless there is a big limitation here I am still -1 for modifying the
> podTemplate logic and +1 for continuing with FLINK-33548
> 
> Gyula
> 
> 
> 
> On Tue, Dec 5, 2023 at 11:16 AM Surendra Singh Lilhore <
> surendralilh...@gmail.com> wrote:
> 
>> Hi Gyula,
>> 
>> FLINK-33548 proposes adding a new resource field to match with Kubernetes
>> pod resource configuration. Here's my suggestion: instead of adding a new
>> resource field, let's use a pod template for more advanced resource setup.
>> Adding a new resource field might confuse users. This change can also help
>> with issues when users use Flink Kubernetes commands directly, without the
>> operator.
>> 
>> Thanks
>> Surendra
>> 
>> 
>> On Tue, Dec 5, 2023 at 3:10 PM richard.su  wrote:
>> 
>>> Sorry Gyula,  let me explain more about the point of 2, if I avoid the
>>> override, I will got a jobmanager pod still with resources consist with
>>> “jobmanager.memory.process.size”, but a flinkdeployment with a resource
>>> larger than that.
>>> 
>>> Thanks for your time.
>>> Richard Su
>>> 
>>>> 2023年12月5日 17:13,richard.su  写道:
>>>> 
>>>> 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  写道:
>>>>> 
>>>>> 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

Re: Discussion: [FLINK-24150] Support to configure cpu resource request and limit in pod template

2023-12-05 Thread richard.su
I think the new configuration could be :

"kubernetes.taskmanager.memory.amount" and "kubernetes.jobmanager.memory.amout"

once we can calculate the limit-factor by the different of requests and limits.

when native mode, we no longer check the process.size as default memory, but 
using this configuration for decoupling logic.

Thanks

Richard Su

> 2023年12月5日 19:22,richard.su  写道:
> 
> Hi, Gyula, from my opinion, this still will using flinkDeployment's resource 
> filed to set jobManager.memory.process.size, and I have told an uncovered 
> case that:
> 
> When user wants to define a flinkdeployment with jobmanager has 1G memory 
> resources in container field but config jobmanager.memory.process.size as 
> 850m, which this solution only improves user config and actually make sconfig 
> more intuitive and easier but not make the container resource decoupling 
> flink configuration.
> 
> So from my side, I think it need to add new configuration to support this 
> proposal, and it need more discussion.
> 
> Thanks
> Chaoran Su
> 
> 
>> 2023年12月5日 18:28,Gyula Fóra  写道:
>> 
>> This is the proposal according to FLINK-33548:
>> 
>> spec:
>> taskManager:
>>   resources:
>> requests:
>>   memory: "64Mi"
>>   cpu: "250m"
>> limits:
>>   memory: "128Mi"
>>   cpu: "500m"
>> 
>> I honestly think this is much more intuitive and easier than using the
>> podTemplate, which is very complex immediately.
>> Please tell me what use-case/setup is not covered by this improved spec.
>> 
>> Unless there is a big limitation here I am still -1 for modifying the
>> podTemplate logic and +1 for continuing with FLINK-33548
>> 
>> Gyula
>> 
>> 
>> 
>> On Tue, Dec 5, 2023 at 11:16 AM Surendra Singh Lilhore <
>> surendralilh...@gmail.com> wrote:
>> 
>>> Hi Gyula,
>>> 
>>> FLINK-33548 proposes adding a new resource field to match with Kubernetes
>>> pod resource configuration. Here's my suggestion: instead of adding a new
>>> resource field, let's use a pod template for more advanced resource setup.
>>> Adding a new resource field might confuse users. This change can also help
>>> with issues when users use Flink Kubernetes commands directly, without the
>>> operator.
>>> 
>>> Thanks
>>> Surendra
>>> 
>>> 
>>> On Tue, Dec 5, 2023 at 3:10 PM richard.su  wrote:
>>> 
>>>> Sorry Gyula,  let me explain more about the point of 2, if I avoid the
>>>> override, I will got a jobmanager pod still with resources consist with
>>>> “jobmanager.memory.process.size”, but a flinkdeployment with a resource
>>>> larger than that.
>>>> 
>>>> Thanks for your time.
>>>> Richard Su
>>>> 
>>>>> 2023年12月5日 17:13,richard.su  写道:
>>>>> 
>>>>> 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 configur

Re: Discussion: [FLINK-24150] Support to configure cpu resource request and limit in pod template

2023-12-05 Thread richard.su
Hi, Gyula, yes, this is a special case in our scenarios, sorry about that it's 
hard to understand,  which we want to reserved some memory beyond the 
jobmanager or task manager's process.To be clear, we need a container has 
memory larger than request, and confirm this pod has Guarantee Qos.

This is because we encounter the glibc problem inside container with flink job 
using Rcoksdb, which reserved memory will help to ease this problem.

So I hope the container resources's request can be decoupling from flink 
configuration.

From flink's current implementation, this could not be done.

Thanks.

Richard Su

> 2023年12月5日 20:28,Gyula Fóra  写道:
> 
> Richard, I still don't understand why the current setup doesn't work for
> you. According to
> https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/deployment/memory/mem_setup/
> :
> 
> The process memory config (which is what we configure) translates directly
> into the container request size. With the new proposal you can set the
> limit independently.
> 
> What you write doesn't make sense to me:
> "user wants to define a flinkdeployment with jobmanager has 1G memory
> resources in container field but config jobmanager.memory.process.size as
> 850m"
> 
> If you want to have a 1G container you set the memory request
> (process.size) in the spec simply  to 1G. Then you have 1G, there are other
> configs on how this 1G will be split inside the container for various
> purposes but these are all covered in detail by the flink memory configs.
> 
> Cheers
> Gyula
> 
> On Tue, Dec 5, 2023 at 1:06 PM richard.su  wrote:
> 
>> I think the new configuration could be :
>> 
>> "kubernetes.taskmanager.memory.amount" and
>> "kubernetes.jobmanager.memory.amout"
>> 
>> once we can calculate the limit-factor by the different of requests and
>> limits.
>> 
>> when native mode, we no longer check the process.size as default memory,
>> but using this configuration for decoupling logic.
>> 
>> Thanks
>> 
>> Richard Su
>> 
>>> 2023年12月5日 19:22,richard.su  写道:
>>> 
>>> Hi, Gyula, from my opinion, this still will using flinkDeployment's
>> resource filed to set jobManager.memory.process.size, and I have told an
>> uncovered case that:
>>> 
>>> When user wants to define a flinkdeployment with jobmanager has 1G
>> memory resources in container field but config
>> jobmanager.memory.process.size as 850m, which this solution only improves
>> user config and actually make sconfig more intuitive and easier but not
>> make the container resource decoupling flink configuration.
>>> 
>>> So from my side, I think it need to add new configuration to support
>> this proposal, and it need more discussion.
>>> 
>>> Thanks
>>> Chaoran Su
>>> 
>>> 
>>>> 2023年12月5日 18:28,Gyula Fóra  写道:
>>>> 
>>>> This is the proposal according to FLINK-33548:
>>>> 
>>>> spec:
>>>> taskManager:
>>>>  resources:
>>>>requests:
>>>>  memory: "64Mi"
>>>>  cpu: "250m"
>>>>limits:
>>>>  memory: "128Mi"
>>>>  cpu: "500m"
>>>> 
>>>> I honestly think this is much more intuitive and easier than using the
>>>> podTemplate, which is very complex immediately.
>>>> Please tell me what use-case/setup is not covered by this improved spec.
>>>> 
>>>> Unless there is a big limitation here I am still -1 for modifying the
>>>> podTemplate logic and +1 for continuing with FLINK-33548
>>>> 
>>>> Gyula
>>>> 
>>>> 
>>>> 
>>>> On Tue, Dec 5, 2023 at 11:16 AM Surendra Singh Lilhore <
>>>> surendralilh...@gmail.com> wrote:
>>>> 
>>>>> Hi Gyula,
>>>>> 
>>>>> FLINK-33548 proposes adding a new resource field to match with
>> Kubernetes
>>>>> pod resource configuration. Here's my suggestion: instead of adding a
>> new
>>>>> resource field, let's use a pod template for more advanced resource
>> setup.
>>>>> Adding a new resource field might confuse users. This change can also
>> help
>>>>> with issues when users use Flink Kubernetes commands directly, without
>> the
>>>>> operator.
>>>>> 
>>>>> Thanks
>>>>> Surendra
>>>>> 
>>>>>

Re: Discussion: [FLINK-24150] Support to configure cpu resource request and limit in pod template

2023-12-05 Thread richard.su
Sorry, "To be clear, we need a container has memory larger than request, and 
confirm this pod has Guarantee Qos." which need to be "To be clear, we need a 
container has memory larger than process.size, and confirm this pod has 
Guarantee Qos."

Thanks.

Richard Su


> 2023年12月5日 20:47,richard.su  写道:
> 
> Hi, Gyula, yes, this is a special case in our scenarios, sorry about that 
> it's hard to understand,  which we want to reserved some memory beyond the 
> jobmanager or task manager's process.To be clear, we need a container has 
> memory larger than request, and confirm this pod has Guarantee Qos.
> 
> This is because we encounter the glibc problem inside container with flink 
> job using Rcoksdb, which reserved memory will help to ease this problem.
> 
> So I hope the container resources's request can be decoupling from flink 
> configuration.
> 
> From flink's current implementation, this could not be done.
> 
> Thanks.
> 
> Richard Su
> 
>> 2023年12月5日 20:28,Gyula Fóra  写道:
>> 
>> Richard, I still don't understand why the current setup doesn't work for
>> you. According to
>> https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/deployment/memory/mem_setup/
>> :
>> 
>> The process memory config (which is what we configure) translates directly
>> into the container request size. With the new proposal you can set the
>> limit independently.
>> 
>> What you write doesn't make sense to me:
>> "user wants to define a flinkdeployment with jobmanager has 1G memory
>> resources in container field but config jobmanager.memory.process.size as
>> 850m"
>> 
>> If you want to have a 1G container you set the memory request
>> (process.size) in the spec simply  to 1G. Then you have 1G, there are other
>> configs on how this 1G will be split inside the container for various
>> purposes but these are all covered in detail by the flink memory configs.
>> 
>> Cheers
>> Gyula
>> 
>> On Tue, Dec 5, 2023 at 1:06 PM richard.su  wrote:
>> 
>>> I think the new configuration could be :
>>> 
>>> "kubernetes.taskmanager.memory.amount" and
>>> "kubernetes.jobmanager.memory.amout"
>>> 
>>> once we can calculate the limit-factor by the different of requests and
>>> limits.
>>> 
>>> when native mode, we no longer check the process.size as default memory,
>>> but using this configuration for decoupling logic.
>>> 
>>> Thanks
>>> 
>>> Richard Su
>>> 
>>>> 2023年12月5日 19:22,richard.su  写道:
>>>> 
>>>> Hi, Gyula, from my opinion, this still will using flinkDeployment's
>>> resource filed to set jobManager.memory.process.size, and I have told an
>>> uncovered case that:
>>>> 
>>>> When user wants to define a flinkdeployment with jobmanager has 1G
>>> memory resources in container field but config
>>> jobmanager.memory.process.size as 850m, which this solution only improves
>>> user config and actually make sconfig more intuitive and easier but not
>>> make the container resource decoupling flink configuration.
>>>> 
>>>> So from my side, I think it need to add new configuration to support
>>> this proposal, and it need more discussion.
>>>> 
>>>> Thanks
>>>> Chaoran Su
>>>> 
>>>> 
>>>>> 2023年12月5日 18:28,Gyula Fóra  写道:
>>>>> 
>>>>> This is the proposal according to FLINK-33548:
>>>>> 
>>>>> spec:
>>>>> taskManager:
>>>>> resources:
>>>>>   requests:
>>>>> memory: "64Mi"
>>>>> cpu: "250m"
>>>>>   limits:
>>>>> memory: "128Mi"
>>>>> cpu: "500m"
>>>>> 
>>>>> I honestly think this is much more intuitive and easier than using the
>>>>> podTemplate, which is very complex immediately.
>>>>> Please tell me what use-case/setup is not covered by this improved spec.
>>>>> 
>>>>> Unless there is a big limitation here I am still -1 for modifying the
>>>>> podTemplate logic and +1 for continuing with FLINK-33548
>>>>> 
>>>>> Gyula
>>>>> 
>>>>> 
>>>>> 
>>>>> On Tue, Dec 5, 2023 at 11:16 AM Surendra Singh Lilhore <
>>>>> surendralilh...@gmail.com> wr

Re: Discussion: [FLINK-24150] Support to configure cpu resource request and limit in pod template

2023-12-05 Thread richard.su
Thank you, Gyula, we are working on validate setting larger  
taskmanager.memory.jvm-overhead.fraction to ease this problem, and on the other 
side, we try to find a way in deployment path to ease this problem.

I agree with you proposal, may be I could find sometime to make a pr for 
FLINK-33548 <https://issues.apache.org/jira/browse/FLINK-33548>.

Thank you for your time.

Richard Su

> 2023年12月5日 21:24,Gyula Fóra  写道:
> 
> I understand your problem but I think you are trying to find a solution in
> the wrong place.
> Have you tried setting taskmanager.memory.jvm-overhead.fraction ? That
> would reserve more memory from the total process memory for non-JVM use.
> 
> Gyula
> 
> On Tue, Dec 5, 2023 at 1:50 PM richard.su  wrote:
> 
>> Sorry, "To be clear, we need a container has memory larger than request,
>> and confirm this pod has Guarantee Qos." which need to be "To be clear, we
>> need a container has memory larger than process.size, and confirm this pod
>> has Guarantee Qos."
>> 
>> Thanks.
>> 
>> Richard Su
>> 
>> 
>>> 2023年12月5日 20:47,richard.su  写道:
>>> 
>>> Hi, Gyula, yes, this is a special case in our scenarios, sorry about
>> that it's hard to understand,  which we want to reserved some memory beyond
>> the jobmanager or task manager's process.To be clear, we need a container
>> has memory larger than request, and confirm this pod has Guarantee Qos.
>>> 
>>> This is because we encounter the glibc problem inside container with
>> flink job using Rcoksdb, which reserved memory will help to ease this
>> problem.
>>> 
>>> So I hope the container resources's request can be decoupling from flink
>> configuration.
>>> 
>>> From flink's current implementation, this could not be done.
>>> 
>>> Thanks.
>>> 
>>> Richard Su
>>> 
>>>> 2023年12月5日 20:28,Gyula Fóra  写道:
>>>> 
>>>> Richard, I still don't understand why the current setup doesn't work for
>>>> you. According to
>>>> 
>> https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/deployment/memory/mem_setup/
>>>> :
>>>> 
>>>> The process memory config (which is what we configure) translates
>> directly
>>>> into the container request size. With the new proposal you can set the
>>>> limit independently.
>>>> 
>>>> What you write doesn't make sense to me:
>>>> "user wants to define a flinkdeployment with jobmanager has 1G memory
>>>> resources in container field but config jobmanager.memory.process.size
>> as
>>>> 850m"
>>>> 
>>>> If you want to have a 1G container you set the memory request
>>>> (process.size) in the spec simply  to 1G. Then you have 1G, there are
>> other
>>>> configs on how this 1G will be split inside the container for various
>>>> purposes but these are all covered in detail by the flink memory
>> configs.
>>>> 
>>>> Cheers
>>>> Gyula
>>>> 
>>>> On Tue, Dec 5, 2023 at 1:06 PM richard.su 
>> wrote:
>>>> 
>>>>> I think the new configuration could be :
>>>>> 
>>>>> "kubernetes.taskmanager.memory.amount" and
>>>>> "kubernetes.jobmanager.memory.amout"
>>>>> 
>>>>> once we can calculate the limit-factor by the different of requests and
>>>>> limits.
>>>>> 
>>>>> when native mode, we no longer check the process.size as default
>> memory,
>>>>> but using this configuration for decoupling logic.
>>>>> 
>>>>> Thanks
>>>>> 
>>>>> Richard Su
>>>>> 
>>>>>> 2023年12月5日 19:22,richard.su  写道:
>>>>>> 
>>>>>> Hi, Gyula, from my opinion, this still will using flinkDeployment's
>>>>> resource filed to set jobManager.memory.process.size, and I have told
>> an
>>>>> uncovered case that:
>>>>>> 
>>>>>> When user wants to define a flinkdeployment with jobmanager has 1G
>>>>> memory resources in container field but config
>>>>> jobmanager.memory.process.size as 850m, which this solution only
>> improves
>>>>> user config and actually make sconfig more intuitive and easier but not
>>>>> make the container resource decoupling flink configuration.

[DISCUSSION] Consider Flink operator having a way to monitor the status of bounded streaming jobs after they finish or error?

2023-12-07 Thread richard.su
Hi, Community, I had found out this issue, but I'm not sure this issue have any 
solution. I have tried flink operator 1.6, which this issue is still exist.

If not, I think this could create a jira issue to following.

When we create a bounded streaming jobs which will finally to become Finished 
status, after this job's status from Running to Finished, flink will shut down 
kubernetes cluster, at code of flink-kubernetes package, class 
KubernetesResourceManagerDriver's method deregisterApplication, which will 
delete jm deployment directly in a second (in our env).
But our operator config, when jm deployment status is Ready and not in 
savepoint progress, this observer interval is 15s, which means operator will 
never observe the job status changing.
So if the job is failed not finished, we cannot distinguish this. All we known 
is Jm deployment is Missing and Job status is Reconciling.
We want to using flink operator integrating into our platform, but it cannot 
monitor job real status, which is wired.

May be it till related to the clean logic of flink native mode, from my side, 
operator side is hard to deal with such situation because we cannot directly 
get the exit code of container when pod is missing and jm deployment is missing.

Thanks to your time to read this issue. 
Richard Su
> 
> 2023年12月6日 13:34,richard.su  写道:
> 
> For more information to produce this problem,
> 
> version: flink operator 1.4
> mode: native
> job: wordcount
> language: java
> type: FlinkDeployment
> 
>> 2023年12月6日 10:52,richard.su  写道:
>> 
>> Hi Community, the default configuration of flink operator is:
>> 
>> kubernetes.operator.reconcile.interval: 15s
>> kubernetes.operator.observer.progress-check.interval: 5s
>> 
>> when a bounded streaming job already stays in stop or error status, jm 
>> deployment will stay to be missing, if I set configuration:
>> 
>> kubernetes.operator.jm-deployment-recover.enabled: false
>> 
>> then, flink operator can only observe the job status at Recociling and jm 
>> deployment status at Missing
>> 
>> we cannot check whether the flink job is  finished or error, because of in 
>> the interval of observer.progress-check, flink web ui is already down.
>> 
>> so, we hope someone in community could show a way to monitor bounded 
>> steaming job's status.
>> 
>> Thanks.
>> 
>> Richard Su
> 



Re: [DISCUSSION] Consider Flink operator having a way to monitor the status of bounded streaming jobs after they finish or error?

2023-12-07 Thread richard.su
Hi Gyula, Flink version is 1.14
Our flink version is hard to upgrade since we have some user in our platform.
sorry I have not noticed this configuration, it's confusing because flink 
operator announced support from 1.13 to 1.17/1.18

Has other solution will work in our situation?

Thanks 
Richard Su

> 2023年12月7日 16:22,Gyula Fóra  写道:
> 
> Hi!
> 
> What Flink version are you using?
> The operator always sets: execution.shutdown-on-application-finish to false
> so that finished / failed application clusters should not exit immediately
> and we can observe them.
> 
> This is however only available in Flink 1.15 and above.
> 
> Cheers,
> Gyula
> 
> On Thu, Dec 7, 2023 at 9:15 AM richard.su  wrote:
> 
>> Hi, Community, I had found out this issue, but I'm not sure this issue
>> have any solution. I have tried flink operator 1.6, which this issue is
>> still exist.
>> 
>> If not, I think this could create a jira issue to following.
>> 
>> When we create a bounded streaming jobs which will finally to become
>> Finished status, after this job's status from Running to Finished, flink
>> will shut down kubernetes cluster, at code of flink-kubernetes package,
>> class KubernetesResourceManagerDriver's method deregisterApplication, which
>> will delete jm deployment directly in a second (in our env).
>> But our operator config, when jm deployment status is Ready and not in
>> savepoint progress, this observer interval is 15s, which means operator
>> will never observe the job status changing.
>> So if the job is failed not finished, we cannot distinguish this. All we
>> known is Jm deployment is Missing and Job status is Reconciling.
>> We want to using flink operator integrating into our platform, but it
>> cannot monitor job real status, which is wired.
>> 
>> May be it till related to the clean logic of flink native mode, from my
>> side, operator side is hard to deal with such situation because we cannot
>> directly get the exit code of container when pod is missing and jm
>> deployment is missing.
>> 
>> Thanks to your time to read this issue.
>> Richard Su
>>> 
>>> 2023年12月6日 13:34,richard.su  写道:
>>> 
>>> For more information to produce this problem,
>>> 
>>> version: flink operator 1.4
>>> mode: native
>>> job: wordcount
>>> language: java
>>> type: FlinkDeployment
>>> 
>>>> 2023年12月6日 10:52,richard.su  写道:
>>>> 
>>>> Hi Community, the default configuration of flink operator is:
>>>> 
>>>> kubernetes.operator.reconcile.interval: 15s
>>>> kubernetes.operator.observer.progress-check.interval: 5s
>>>> 
>>>> when a bounded streaming job already stays in stop or error status, jm
>> deployment will stay to be missing, if I set configuration:
>>>> 
>>>> kubernetes.operator.jm-deployment-recover.enabled: false
>>>> 
>>>> then, flink operator can only observe the job status at Recociling and
>> jm deployment status at Missing
>>>> 
>>>> we cannot check whether the flink job is  finished or error, because of
>> in the interval of observer.progress-check, flink web ui is already down.
>>>> 
>>>> so, we hope someone in community could show a way to monitor bounded
>> steaming job's status.
>>>> 
>>>> Thanks.
>>>> 
>>>> Richard Su
>>> 
>> 
>> 



Re: [DISCUSSION] Consider Flink operator having a way to monitor the status of bounded streaming jobs after they finish or error?

2023-12-07 Thread richard.su
Hi Gyula, Flink version is 1.14
Our flink version is hard to upgrade since we have some user in our platform.
sorry I have not noticed this configuration, it's confusing because flink 
operator announced support from 1.13 to 1.17/1.18

Has other solution will work in our situation?

Thanks 
Richard Su

> 2023年12月7日 16:22,Gyula Fóra  写道:
> 
> Hi!
> 
> What Flink version are you using?
> The operator always sets: execution.shutdown-on-application-finish to false
> so that finished / failed application clusters should not exit immediately
> and we can observe them.
> 
> This is however only available in Flink 1.15 and above.
> 
> Cheers,
> Gyula
> 
> On Thu, Dec 7, 2023 at 9:15 AM richard.su  wrote:
> 
>> Hi, Community, I had found out this issue, but I'm not sure this issue
>> have any solution. I have tried flink operator 1.6, which this issue is
>> still exist.
>> 
>> If not, I think this could create a jira issue to following.
>> 
>> When we create a bounded streaming jobs which will finally to become
>> Finished status, after this job's status from Running to Finished, flink
>> will shut down kubernetes cluster, at code of flink-kubernetes package,
>> class KubernetesResourceManagerDriver's method deregisterApplication, which
>> will delete jm deployment directly in a second (in our env).
>> But our operator config, when jm deployment status is Ready and not in
>> savepoint progress, this observer interval is 15s, which means operator
>> will never observe the job status changing.
>> So if the job is failed not finished, we cannot distinguish this. All we
>> known is Jm deployment is Missing and Job status is Reconciling.
>> We want to using flink operator integrating into our platform, but it
>> cannot monitor job real status, which is wired.
>> 
>> May be it till related to the clean logic of flink native mode, from my
>> side, operator side is hard to deal with such situation because we cannot
>> directly get the exit code of container when pod is missing and jm
>> deployment is missing.
>> 
>> Thanks to your time to read this issue.
>> Richard Su
>>> 
>>> 2023年12月6日 13:34,richard.su  写道:
>>> 
>>> For more information to produce this problem,
>>> 
>>> version: flink operator 1.4
>>> mode: native
>>> job: wordcount
>>> language: java
>>> type: FlinkDeployment
>>> 
>>>> 2023年12月6日 10:52,richard.su  写道:
>>>> 
>>>> Hi Community, the default configuration of flink operator is:
>>>> 
>>>> kubernetes.operator.reconcile.interval: 15s
>>>> kubernetes.operator.observer.progress-check.interval: 5s
>>>> 
>>>> when a bounded streaming job already stays in stop or error status, jm
>> deployment will stay to be missing, if I set configuration:
>>>> 
>>>> kubernetes.operator.jm-deployment-recover.enabled: false
>>>> 
>>>> then, flink operator can only observe the job status at Recociling and
>> jm deployment status at Missing
>>>> 
>>>> we cannot check whether the flink job is  finished or error, because of
>> in the interval of observer.progress-check, flink web ui is already down.
>>>> 
>>>> so, we hope someone in community could show a way to monitor bounded
>> steaming job's status.
>>>> 
>>>> Thanks.
>>>> 
>>>> Richard Su
>>> 
>> 
>> 



Re: [DISCUSSION] Consider Flink operator having a way to monitor the status of bounded streaming jobs after they finish or error?

2023-12-07 Thread richard.su
Thanks, Gyula, this is one of our technical debt in our platform's develop 
progress, which is helpless for me.

If the operator's monitor will sometimesmissing bounded job, we may 
change our strategy to modified the docker-entrypoint.sh of our 
flink-custom-image, which capture the exit code of jm process and do what we 
want after job done, although this is wired but work.

I think this should add some tips in doc of flink operator, actually the 
start-up and shutdown process of flink job works well in flink 1.14, but only 
this situation cannot work.

I will checkout all code of version judgment  through flink operator to find 
out other potential issues. hope this will be helpful for other users.

Thanks again.

Richard Su

> 2023年12月7日 16:45,Gyula Fóra  写道:
> 
> This config has nothing to do with the operator (it's a core flink feature)
> and is not an issue after Flink 1.15.
> Newer operator versions (1.7+) drop support for Flink 1.13 and 1.14 as it's
> not feasible to maintain too many legacy codepaths.
> 
> The only solution for you is to update your Flink versions, you are missing
> out on so many improvements.
> 
> Gyula
> 
> On Thu, Dec 7, 2023 at 9:32 AM richard.su  wrote:
> 
>> Hi Gyula, Flink version is 1.14
>> Our flink version is hard to upgrade since we have some user in our
>> platform.
>> sorry I have not noticed this configuration, it's confusing because flink
>> operator announced support from 1.13 to 1.17/1.18
>> 
>> Has other solution will work in our situation?
>> 
>> Thanks
>> Richard Su
>> 
>>> 2023年12月7日 16:22,Gyula Fóra  写道:
>>> 
>>> Hi!
>>> 
>>> What Flink version are you using?
>>> The operator always sets: execution.shutdown-on-application-finish to
>> false
>>> so that finished / failed application clusters should not exit
>> immediately
>>> and we can observe them.
>>> 
>>> This is however only available in Flink 1.15 and above.
>>> 
>>> Cheers,
>>> Gyula
>>> 
>>> On Thu, Dec 7, 2023 at 9:15 AM richard.su 
>> wrote:
>>> 
>>>> Hi, Community, I had found out this issue, but I'm not sure this issue
>>>> have any solution. I have tried flink operator 1.6, which this issue is
>>>> still exist.
>>>> 
>>>> If not, I think this could create a jira issue to following.
>>>> 
>>>> When we create a bounded streaming jobs which will finally to become
>>>> Finished status, after this job's status from Running to Finished, flink
>>>> will shut down kubernetes cluster, at code of flink-kubernetes package,
>>>> class KubernetesResourceManagerDriver's method deregisterApplication,
>> which
>>>> will delete jm deployment directly in a second (in our env).
>>>> But our operator config, when jm deployment status is Ready and not in
>>>> savepoint progress, this observer interval is 15s, which means operator
>>>> will never observe the job status changing.
>>>> So if the job is failed not finished, we cannot distinguish this. All we
>>>> known is Jm deployment is Missing and Job status is Reconciling.
>>>> We want to using flink operator integrating into our platform, but it
>>>> cannot monitor job real status, which is wired.
>>>> 
>>>> May be it till related to the clean logic of flink native mode, from my
>>>> side, operator side is hard to deal with such situation because we
>> cannot
>>>> directly get the exit code of container when pod is missing and jm
>>>> deployment is missing.
>>>> 
>>>> Thanks to your time to read this issue.
>>>> Richard Su
>>>>> 
>>>>> 2023年12月6日 13:34,richard.su  写道:
>>>>> 
>>>>> For more information to produce this problem,
>>>>> 
>>>>> version: flink operator 1.4
>>>>> mode: native
>>>>> job: wordcount
>>>>> language: java
>>>>> type: FlinkDeployment
>>>>> 
>>>>>> 2023年12月6日 10:52,richard.su  写道:
>>>>>> 
>>>>>> Hi Community, the default configuration of flink operator is:
>>>>>> 
>>>>>> kubernetes.operator.reconcile.interval: 15s
>>>>>> kubernetes.operator.observer.progress-check.interval: 5s
>>>>>> 
>>>>>> when a bounded streaming job already stays in stop or error status, jm
>>>> deployment will stay to be missing, if I set configuration:
>>>>>> 
>>>>>> kubernetes.operator.jm-deployment-recover.enabled: false
>>>>>> 
>>>>>> then, flink operator can only observe the job status at Recociling and
>>>> jm deployment status at Missing
>>>>>> 
>>>>>> we cannot check whether the flink job is  finished or error, because
>> of
>>>> in the interval of observer.progress-check, flink web ui is already
>> down.
>>>>>> 
>>>>>> so, we hope someone in community could show a way to monitor bounded
>>>> steaming job's status.
>>>>>> 
>>>>>> Thanks.
>>>>>> 
>>>>>> Richard Su
>>>>> 
>>>> 
>>>> 
>> 
>>