Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-20 Thread Yang Wang
I also lean to persist the FlinkDeployment and its status via K8s resources.
Unless necessary, we should not introduce other external dependencies(e.g.
mysql).
It will make the k8s operator more complicated.


Best,
Yang

Gyula Fóra  于2022年2月21日周一 02:48写道:

> Hi!
>
> Thank you for your interest in contributing to the operator.
>
> The operator persists information in the status of the FlinkDeployment
> resource. We should not need any additional persistence layer on top of
> this in the current design.
>
> Could you please give me a concrete example of what is not working with the
> current design?
>
> Thanks,
> Gyula
>
> On Sat, Feb 19, 2022 at 7:06 AM zhengyu chen  wrote:
>
> > Hi, regarding the construction of k8s Flink Operator, I have already
> > completed some functions. I hope to contribute this part of the functions
> > and discuss with the community how to improve it. How should I start?
> >
> > So far I have seen that the component has no operation persistence.
> Should
> > we persist its operation? for example, when I have a SessionCluster
> > deployment, I need to write its metadata to an external storage system in
> > yaml mode,
> > such as use mysql for storage. This design idea is similar to etcd in
> > k8s.If our  k8s Flink Operator application is restarted, We can recover
> > metadata information about deployment jobs, clusters, and so on based on
> > the database
> >
> > Best
> > ConradJam
> >
> > On 2022/01/25 05:08:01 Thomas Weise wrote:
> > > Hi,
> > >
> > > As promised in [1] we would like to start the discussion on the
> > > addition of a Kubernetes operator to the Flink project as FLIP-212:
> > >
> >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-212%3A+Introduce+Flink+Kubernetes+Operator
> > >
> > > Please note that the FLIP is currently focussed on the overall
> > > direction; the intention is to fill in more details once we converge
> > > on the high level plan.
> > >
> > > Thanks and looking forward to a lively discussion!
> > >
> > > Thomas
> > >
> > > [1] https://lists.apache.org/thread/l1dkp8v4bhlcyb4tdts99g7w4wdglfy4
> > >
> >
>


Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-20 Thread Gyula Fóra
Hi!

Thank you for your interest in contributing to the operator.

The operator persists information in the status of the FlinkDeployment
resource. We should not need any additional persistence layer on top of
this in the current design.

Could you please give me a concrete example of what is not working with the
current design?

Thanks,
Gyula

On Sat, Feb 19, 2022 at 7:06 AM zhengyu chen  wrote:

> Hi, regarding the construction of k8s Flink Operator, I have already
> completed some functions. I hope to contribute this part of the functions
> and discuss with the community how to improve it. How should I start?
>
> So far I have seen that the component has no operation persistence. Should
> we persist its operation? for example, when I have a SessionCluster
> deployment, I need to write its metadata to an external storage system in
> yaml mode,
> such as use mysql for storage. This design idea is similar to etcd in
> k8s.If our  k8s Flink Operator application is restarted, We can recover
> metadata information about deployment jobs, clusters, and so on based on
> the database
>
> Best
> ConradJam
>
> On 2022/01/25 05:08:01 Thomas Weise wrote:
> > Hi,
> >
> > As promised in [1] we would like to start the discussion on the
> > addition of a Kubernetes operator to the Flink project as FLIP-212:
> >
>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-212%3A+Introduce+Flink+Kubernetes+Operator
> >
> > Please note that the FLIP is currently focussed on the overall
> > direction; the intention is to fill in more details once we converge
> > on the high level plan.
> >
> > Thanks and looking forward to a lively discussion!
> >
> > Thomas
> >
> > [1] https://lists.apache.org/thread/l1dkp8v4bhlcyb4tdts99g7w4wdglfy4
> >
>


RE: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-18 Thread zhengyu chen
Hi, regarding the construction of k8s Flink Operator, I have already
completed some functions. I hope to contribute this part of the functions
and discuss with the community how to improve it. How should I start?

So far I have seen that the component has no operation persistence. Should
we persist its operation? for example, when I have a SessionCluster
deployment, I need to write its metadata to an external storage system in
yaml mode,
such as use mysql for storage. This design idea is similar to etcd in
k8s.If our  k8s Flink Operator application is restarted, We can recover
metadata information about deployment jobs, clusters, and so on based on
the database

Best
ConradJam

On 2022/01/25 05:08:01 Thomas Weise wrote:
> Hi,
>
> As promised in [1] we would like to start the discussion on the
> addition of a Kubernetes operator to the Flink project as FLIP-212:
>
https://cwiki.apache.org/confluence/display/FLINK/FLIP-212%3A+Introduce+Flink+Kubernetes+Operator
>
> Please note that the FLIP is currently focussed on the overall
> direction; the intention is to fill in more details once we converge
> on the high level plan.
>
> Thanks and looking forward to a lively discussion!
>
> Thomas
>
> [1] https://lists.apache.org/thread/l1dkp8v4bhlcyb4tdts99g7w4wdglfy4
>


Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-16 Thread Gyula Fóra
Hi Konstantin,

I completely agree with the general philosophy that if a resource exists it
should be "running" or I would rather say "do it's thing" whatever that
means for a particular resource.

We followed this design principle when we decided to have only 2 "desired
states" running and suspended and not have states like canceled etc.

Based on our own use cases and the feedback that we received from others
temporarily suspending a streaming job deployment is part of it's regular
lifecycle. The job still exist and will continue after suspension but it
signifies a state where data processing should be paused for whatever
reason.

The current suspend mechanism is also difficult to do manually if we remove
it from the operator:
 1. We need to implement a cancel-with-savepoint operation and expose this
to the user
 2. The user needs to manually look up the savepoint
 3. Create a new resource later

Adding 1.) is basically equivalent to the current implementation but would
actually expose an operation that feels much more unnatural compared to a
supend.

Cheers,
Gyula

On Wed, Feb 16, 2022 at 11:16 AM Konstantin Knauf  wrote:

> Hi Gyula,
>
> sorry for joining late. One comment on the API design for consideration: we
> are using the job.state as kind of a "desired state", right? This is quite
> uncommon in Kubernetes to my knowledge. In Kubernetes almost always the
> fact that a resource exists means that it should be "running". The only API
> that I am aware of that has something like "suspended" is a Kubernetes Job
> (
>
> https://kubernetes.io/docs/concepts/workloads/controllers/job/#suspending-a-job
> ),
> which looks retrofitted to me.
>
> Cheers,
>
> Konstantin
>
> On Wed, Feb 16, 2022 at 10:52 AM Gyula Fóra  wrote:
>
> > Hi All!
> >
> > Thank you all for reviewing the PR and already helping to make it
> better. I
> > have opened a bunch of jira tickets under
> > https://issues.apache.org/jira/browse/FLINK-25963 based on some comments
> > and incomplete features in general.
> >
> > Given that there were no major objections about the prototype, I will
> merge
> > it now so we can start collaborating together.
> >
> > Cheers,
> > Gyula
> >
> > On Wed, Feb 16, 2022 at 3:52 AM Yang Wang  wrote:
> >
> > > Thanks for the explanation.
> > > Given that it is unrelated with java version in Flink.
> > > Starting with java11 for the flink-kubernetes-operator makes sense to
> me.
> > >
> > >
> > > Best,
> > > Yang
> > >
> > > Thomas Weise  于2022年2月15日周二 23:57写道:
> > >
> > > > Hi,
> > > >
> > > > At this point I see no reason to support Java 8 for a new project.
> > > > Java 8 is being phased out, we should start with 11.
> > > >
> > > > Also, since the operator isn't a library but effectively just a
> docker
> > > > image, the ability to change the Java version isn't as critical as it
> > > > is for Flink core, which needs to run in many different environments.
> > > >
> > > > Cheers,
> > > > Thomas
> > > >
> > > > On Tue, Feb 15, 2022 at 4:50 AM Gyula Fóra 
> > wrote:
> > > > >
> > > > > Hi Devs,
> > > > >
> > > > > Yang Wang discovered that the current prototype is not compatible
> > with
> > > > Java
> > > > > 8 but only 11 and upwards.
> > > > >
> > > > > The reason for this is that the java operator SDK itself is not
> java
> > 8
> > > > > compatible unfortunately.
> > > > >
> > > > > Given that Java 8 is on the road to deprecation and that the
> operator
> > > > runs
> > > > > as a containerized deployment, are there any concerns regarding
> > making
> > > > the
> > > > > target java version 11?
> > > > > This should not affect deployed flink clusters and jobs, those
> should
> > > > still
> > > > > work with Java 8, but only the kubernetes operator itself.
> > > > >
> > > > > Cheers,
> > > > > Gyula
> > > > >
> > > > >
> > > > > On Tue, Feb 15, 2022 at 1:06 PM Yang Wang 
> > > wrote:
> > > > >
> > > > > > I also lean to not introduce the savepoint/checkpoint related
> > fields
> > > > to the
> > > > > > job spec, especially in the very beginning of
> > > > flink-kubernetes-operator.
> > > > > >
> > > > > >
> > > > > > Best,
> > > > > > Yang
> > > > > >
> > > > > > Gyula Fóra  于2022年2月15日周二 19:02写道:
> > > > > >
> > > > > > > Hi Peng Yuan!
> > > > > > >
> > > > > > > While I do agree that savepoint path is a very important
> > production
> > > > > > > configuration there are a lot of other things that come to my
> > mind:
> > > > > > >  - savepoint dir
> > > > > > >  - checkpoint dir
> > > > > > >  - checkpoint interval/timeout
> > > > > > >  - high availability settings (provider/storagedir etc)
> > > > > > >
> > > > > > > just to name a few...
> > > > > > >
> > > > > > > While these are all production critical, they have nice clean
> > Flink
> > > > > > config
> > > > > > > settings to go with them. If we stand introducing these to
> > jobspec
> > > we
> > > > > > only
> > > > > > > get confusion about priority order etc and it is going to be
> hard
> > > to
> > > > > > change
> > > > > > > or remove them 

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-16 Thread Konstantin Knauf
Hi Gyula,

sorry for joining late. One comment on the API design for consideration: we
are using the job.state as kind of a "desired state", right? This is quite
uncommon in Kubernetes to my knowledge. In Kubernetes almost always the
fact that a resource exists means that it should be "running". The only API
that I am aware of that has something like "suspended" is a Kubernetes Job (
https://kubernetes.io/docs/concepts/workloads/controllers/job/#suspending-a-job),
which looks retrofitted to me.

Cheers,

Konstantin

On Wed, Feb 16, 2022 at 10:52 AM Gyula Fóra  wrote:

> Hi All!
>
> Thank you all for reviewing the PR and already helping to make it better. I
> have opened a bunch of jira tickets under
> https://issues.apache.org/jira/browse/FLINK-25963 based on some comments
> and incomplete features in general.
>
> Given that there were no major objections about the prototype, I will merge
> it now so we can start collaborating together.
>
> Cheers,
> Gyula
>
> On Wed, Feb 16, 2022 at 3:52 AM Yang Wang  wrote:
>
> > Thanks for the explanation.
> > Given that it is unrelated with java version in Flink.
> > Starting with java11 for the flink-kubernetes-operator makes sense to me.
> >
> >
> > Best,
> > Yang
> >
> > Thomas Weise  于2022年2月15日周二 23:57写道:
> >
> > > Hi,
> > >
> > > At this point I see no reason to support Java 8 for a new project.
> > > Java 8 is being phased out, we should start with 11.
> > >
> > > Also, since the operator isn't a library but effectively just a docker
> > > image, the ability to change the Java version isn't as critical as it
> > > is for Flink core, which needs to run in many different environments.
> > >
> > > Cheers,
> > > Thomas
> > >
> > > On Tue, Feb 15, 2022 at 4:50 AM Gyula Fóra 
> wrote:
> > > >
> > > > Hi Devs,
> > > >
> > > > Yang Wang discovered that the current prototype is not compatible
> with
> > > Java
> > > > 8 but only 11 and upwards.
> > > >
> > > > The reason for this is that the java operator SDK itself is not java
> 8
> > > > compatible unfortunately.
> > > >
> > > > Given that Java 8 is on the road to deprecation and that the operator
> > > runs
> > > > as a containerized deployment, are there any concerns regarding
> making
> > > the
> > > > target java version 11?
> > > > This should not affect deployed flink clusters and jobs, those should
> > > still
> > > > work with Java 8, but only the kubernetes operator itself.
> > > >
> > > > Cheers,
> > > > Gyula
> > > >
> > > >
> > > > On Tue, Feb 15, 2022 at 1:06 PM Yang Wang 
> > wrote:
> > > >
> > > > > I also lean to not introduce the savepoint/checkpoint related
> fields
> > > to the
> > > > > job spec, especially in the very beginning of
> > > flink-kubernetes-operator.
> > > > >
> > > > >
> > > > > Best,
> > > > > Yang
> > > > >
> > > > > Gyula Fóra  于2022年2月15日周二 19:02写道:
> > > > >
> > > > > > Hi Peng Yuan!
> > > > > >
> > > > > > While I do agree that savepoint path is a very important
> production
> > > > > > configuration there are a lot of other things that come to my
> mind:
> > > > > >  - savepoint dir
> > > > > >  - checkpoint dir
> > > > > >  - checkpoint interval/timeout
> > > > > >  - high availability settings (provider/storagedir etc)
> > > > > >
> > > > > > just to name a few...
> > > > > >
> > > > > > While these are all production critical, they have nice clean
> Flink
> > > > > config
> > > > > > settings to go with them. If we stand introducing these to
> jobspec
> > we
> > > > > only
> > > > > > get confusion about priority order etc and it is going to be hard
> > to
> > > > > change
> > > > > > or remove them in the future. In any case we should validate that
> > > these
> > > > > > configs exist in cases where users use a stateful upgrade mode
> for
> > > > > example.
> > > > > > This is something we need to add for sure.
> > > > > >
> > > > > > As for the other options you mentioned like automatic savepoint
> > > > > generation
> > > > > > for instance, those deserve an independent discussion of their
> own
> > I
> > > > > > believe :)
> > > > > >
> > > > > > Cheers,
> > > > > > Gyula
> > > > > >
> > > > > > On Tue, Feb 15, 2022 at 11:23 AM K Fred 
> > > wrote:
> > > > > >
> > > > > > > Hi Matyas!
> > > > > > >
> > > > > > > Thanks for your reply!
> > > > > > > For 1. and 3. scenarios,I couldn't agree more with the
> > podTemplate
> > > > > > solution
> > > > > > > , i missed this part.
> > > > > > > For savepoint related configuration, I think it's very
> important
> > > to be
> > > > > > > specified in JobSpec, Because savepoint is a very common
> > > configuration
> > > > > > for
> > > > > > > upgrading a job, if it has been placed in JobSpec can be
> > obviously
> > > > > > > configured by the user. In addition, other advanced properties
> > can
> > > be
> > > > > put
> > > > > > > into flinkConfiguration customized by expert users.
> > > > > > > A bunch of savepoint configuration as follows:
> > > > > > >
> > > > > > > > fromSavepoint——Job restart from
> > > > > > >
> > > > > 

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-16 Thread Gyula Fóra
Hi All!

Thank you all for reviewing the PR and already helping to make it better. I
have opened a bunch of jira tickets under
https://issues.apache.org/jira/browse/FLINK-25963 based on some comments
and incomplete features in general.

Given that there were no major objections about the prototype, I will merge
it now so we can start collaborating together.

Cheers,
Gyula

On Wed, Feb 16, 2022 at 3:52 AM Yang Wang  wrote:

> Thanks for the explanation.
> Given that it is unrelated with java version in Flink.
> Starting with java11 for the flink-kubernetes-operator makes sense to me.
>
>
> Best,
> Yang
>
> Thomas Weise  于2022年2月15日周二 23:57写道:
>
> > Hi,
> >
> > At this point I see no reason to support Java 8 for a new project.
> > Java 8 is being phased out, we should start with 11.
> >
> > Also, since the operator isn't a library but effectively just a docker
> > image, the ability to change the Java version isn't as critical as it
> > is for Flink core, which needs to run in many different environments.
> >
> > Cheers,
> > Thomas
> >
> > On Tue, Feb 15, 2022 at 4:50 AM Gyula Fóra  wrote:
> > >
> > > Hi Devs,
> > >
> > > Yang Wang discovered that the current prototype is not compatible with
> > Java
> > > 8 but only 11 and upwards.
> > >
> > > The reason for this is that the java operator SDK itself is not java 8
> > > compatible unfortunately.
> > >
> > > Given that Java 8 is on the road to deprecation and that the operator
> > runs
> > > as a containerized deployment, are there any concerns regarding making
> > the
> > > target java version 11?
> > > This should not affect deployed flink clusters and jobs, those should
> > still
> > > work with Java 8, but only the kubernetes operator itself.
> > >
> > > Cheers,
> > > Gyula
> > >
> > >
> > > On Tue, Feb 15, 2022 at 1:06 PM Yang Wang 
> wrote:
> > >
> > > > I also lean to not introduce the savepoint/checkpoint related fields
> > to the
> > > > job spec, especially in the very beginning of
> > flink-kubernetes-operator.
> > > >
> > > >
> > > > Best,
> > > > Yang
> > > >
> > > > Gyula Fóra  于2022年2月15日周二 19:02写道:
> > > >
> > > > > Hi Peng Yuan!
> > > > >
> > > > > While I do agree that savepoint path is a very important production
> > > > > configuration there are a lot of other things that come to my mind:
> > > > >  - savepoint dir
> > > > >  - checkpoint dir
> > > > >  - checkpoint interval/timeout
> > > > >  - high availability settings (provider/storagedir etc)
> > > > >
> > > > > just to name a few...
> > > > >
> > > > > While these are all production critical, they have nice clean Flink
> > > > config
> > > > > settings to go with them. If we stand introducing these to jobspec
> we
> > > > only
> > > > > get confusion about priority order etc and it is going to be hard
> to
> > > > change
> > > > > or remove them in the future. In any case we should validate that
> > these
> > > > > configs exist in cases where users use a stateful upgrade mode for
> > > > example.
> > > > > This is something we need to add for sure.
> > > > >
> > > > > As for the other options you mentioned like automatic savepoint
> > > > generation
> > > > > for instance, those deserve an independent discussion of their own
> I
> > > > > believe :)
> > > > >
> > > > > Cheers,
> > > > > Gyula
> > > > >
> > > > > On Tue, Feb 15, 2022 at 11:23 AM K Fred 
> > wrote:
> > > > >
> > > > > > Hi Matyas!
> > > > > >
> > > > > > Thanks for your reply!
> > > > > > For 1. and 3. scenarios,I couldn't agree more with the
> podTemplate
> > > > > solution
> > > > > > , i missed this part.
> > > > > > For savepoint related configuration, I think it's very important
> > to be
> > > > > > specified in JobSpec, Because savepoint is a very common
> > configuration
> > > > > for
> > > > > > upgrading a job, if it has been placed in JobSpec can be
> obviously
> > > > > > configured by the user. In addition, other advanced properties
> can
> > be
> > > > put
> > > > > > into flinkConfiguration customized by expert users.
> > > > > > A bunch of savepoint configuration as follows:
> > > > > >
> > > > > > > fromSavepoint——Job restart from
> > > > > >
> > > > > > autoSavepointSecond—— Automatically take a savepoint to the
> > > > > `savepointsDir`
> > > > > > > every n seconds.
> > > > > >
> > > > > > savepointsDir—— Savepoints dir where to store automatically taken
> > > > > > > savepoints
> > > > > >
> > > > > > savepointGeneration—— Update savepoint generation of job status
> > for a
> > > > > > > running job (should be defined in JobStatus)
> > > > > >
> > > > > >
> > > > > > Best wishes,
> > > > > > Peng Yuan.
> > > > > >
> > > > > > On Tue, Feb 15, 2022 at 4:41 PM Őrhidi Mátyás <
> > matyas.orh...@gmail.com
> > > > >
> > > > > > wrote:
> > > > > >
> > > > > > > Hi Peng,
> > > > > > >
> > > > > > > Thanks for your feedback. Regarding 1. and 3. scenarios, the
> > > > > podTemplate
> > > > > > > functionality in the operator could cover both. We also need to
> > be
> > > > > > careful
> > > > > > > about 

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-15 Thread Yang Wang
Thanks for the explanation.
Given that it is unrelated with java version in Flink.
Starting with java11 for the flink-kubernetes-operator makes sense to me.


Best,
Yang

Thomas Weise  于2022年2月15日周二 23:57写道:

> Hi,
>
> At this point I see no reason to support Java 8 for a new project.
> Java 8 is being phased out, we should start with 11.
>
> Also, since the operator isn't a library but effectively just a docker
> image, the ability to change the Java version isn't as critical as it
> is for Flink core, which needs to run in many different environments.
>
> Cheers,
> Thomas
>
> On Tue, Feb 15, 2022 at 4:50 AM Gyula Fóra  wrote:
> >
> > Hi Devs,
> >
> > Yang Wang discovered that the current prototype is not compatible with
> Java
> > 8 but only 11 and upwards.
> >
> > The reason for this is that the java operator SDK itself is not java 8
> > compatible unfortunately.
> >
> > Given that Java 8 is on the road to deprecation and that the operator
> runs
> > as a containerized deployment, are there any concerns regarding making
> the
> > target java version 11?
> > This should not affect deployed flink clusters and jobs, those should
> still
> > work with Java 8, but only the kubernetes operator itself.
> >
> > Cheers,
> > Gyula
> >
> >
> > On Tue, Feb 15, 2022 at 1:06 PM Yang Wang  wrote:
> >
> > > I also lean to not introduce the savepoint/checkpoint related fields
> to the
> > > job spec, especially in the very beginning of
> flink-kubernetes-operator.
> > >
> > >
> > > Best,
> > > Yang
> > >
> > > Gyula Fóra  于2022年2月15日周二 19:02写道:
> > >
> > > > Hi Peng Yuan!
> > > >
> > > > While I do agree that savepoint path is a very important production
> > > > configuration there are a lot of other things that come to my mind:
> > > >  - savepoint dir
> > > >  - checkpoint dir
> > > >  - checkpoint interval/timeout
> > > >  - high availability settings (provider/storagedir etc)
> > > >
> > > > just to name a few...
> > > >
> > > > While these are all production critical, they have nice clean Flink
> > > config
> > > > settings to go with them. If we stand introducing these to jobspec we
> > > only
> > > > get confusion about priority order etc and it is going to be hard to
> > > change
> > > > or remove them in the future. In any case we should validate that
> these
> > > > configs exist in cases where users use a stateful upgrade mode for
> > > example.
> > > > This is something we need to add for sure.
> > > >
> > > > As for the other options you mentioned like automatic savepoint
> > > generation
> > > > for instance, those deserve an independent discussion of their own I
> > > > believe :)
> > > >
> > > > Cheers,
> > > > Gyula
> > > >
> > > > On Tue, Feb 15, 2022 at 11:23 AM K Fred 
> wrote:
> > > >
> > > > > Hi Matyas!
> > > > >
> > > > > Thanks for your reply!
> > > > > For 1. and 3. scenarios,I couldn't agree more with the podTemplate
> > > > solution
> > > > > , i missed this part.
> > > > > For savepoint related configuration, I think it's very important
> to be
> > > > > specified in JobSpec, Because savepoint is a very common
> configuration
> > > > for
> > > > > upgrading a job, if it has been placed in JobSpec can be obviously
> > > > > configured by the user. In addition, other advanced properties can
> be
> > > put
> > > > > into flinkConfiguration customized by expert users.
> > > > > A bunch of savepoint configuration as follows:
> > > > >
> > > > > > fromSavepoint——Job restart from
> > > > >
> > > > > autoSavepointSecond—— Automatically take a savepoint to the
> > > > `savepointsDir`
> > > > > > every n seconds.
> > > > >
> > > > > savepointsDir—— Savepoints dir where to store automatically taken
> > > > > > savepoints
> > > > >
> > > > > savepointGeneration—— Update savepoint generation of job status
> for a
> > > > > > running job (should be defined in JobStatus)
> > > > >
> > > > >
> > > > > Best wishes,
> > > > > Peng Yuan.
> > > > >
> > > > > On Tue, Feb 15, 2022 at 4:41 PM Őrhidi Mátyás <
> matyas.orh...@gmail.com
> > > >
> > > > > wrote:
> > > > >
> > > > > > Hi Peng,
> > > > > >
> > > > > > Thanks for your feedback. Regarding 1. and 3. scenarios, the
> > > > podTemplate
> > > > > > functionality in the operator could cover both. We also need to
> be
> > > > > careful
> > > > > > about introducing proxy parameters in the CRD spec. The savepoint
> > > path
> > > > is
> > > > > > usually accompanied with a bunch of other configurations for
> example,
> > > > so
> > > > > > users need to use configuration params anyway. What do you think?
> > > > > >
> > > > > > Best,
> > > > > > Matyas
> > > > > >
> > > > > > On Tue, Feb 15, 2022 at 8:58 AM K Fred 
> > > wrote:
> > > > > >
> > > > > > > Hi Gyula!
> > > > > > >
> > > > > > > I have reviewed the prototype design of
> flink-kubernetes-operator
> > > you
> > > > > > > submitted, and I have the following questions:
> > > > > > >
> > > > > > > 1.Can a Flink Jar package that supports pulling from the
> sidecar be
> > > > > added
> > > > > > > to the 

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-15 Thread K Fred
Hi Gyula!

Alright! For clarity, the savepoint path and other savepoint related
configuration can be put into flinkConfiguration.
On the side, I think the automatic savepoint generation for instance should
be put into JobSpec along with other job options, and FlinkConfiguration
only configures what is contained in the Flink-conf.yaml file.

Best Wishes,
Peng Yuan

On Tue, Feb 15, 2022 at 7:02 PM Gyula Fóra  wrote:

> Hi Peng Yuan!
>
> While I do agree that savepoint path is a very important production
> configuration there are a lot of other things that come to my mind:
>  - savepoint dir
>  - checkpoint dir
>  - checkpoint interval/timeout
>  - high availability settings (provider/storagedir etc)
>
> just to name a few...
>
> While these are all production critical, they have nice clean Flink config
> settings to go with them. If we stand introducing these to jobspec we only
> get confusion about priority order etc and it is going to be hard to change
> or remove them in the future. In any case we should validate that these
> configs exist in cases where users use a stateful upgrade mode for example.
> This is something we need to add for sure.
>
> As for the other options you mentioned like automatic savepoint generation
> for instance, those deserve an independent discussion of their own I
> believe :)
>
> Cheers,
> Gyula
>
> On Tue, Feb 15, 2022 at 11:23 AM K Fred  wrote:
>
> > Hi Matyas!
> >
> > Thanks for your reply!
> > For 1. and 3. scenarios,I couldn't agree more with the podTemplate
> solution
> > , i missed this part.
> > For savepoint related configuration, I think it's very important to be
> > specified in JobSpec, Because savepoint is a very common configuration
> for
> > upgrading a job, if it has been placed in JobSpec can be obviously
> > configured by the user. In addition, other advanced properties can be put
> > into flinkConfiguration customized by expert users.
> > A bunch of savepoint configuration as follows:
> >
> > > fromSavepoint——Job restart from
> >
> > autoSavepointSecond—— Automatically take a savepoint to the
> `savepointsDir`
> > > every n seconds.
> >
> > savepointsDir—— Savepoints dir where to store automatically taken
> > > savepoints
> >
> > savepointGeneration—— Update savepoint generation of job status for a
> > > running job (should be defined in JobStatus)
> >
> >
> > Best wishes,
> > Peng Yuan.
> >
> > On Tue, Feb 15, 2022 at 4:41 PM Őrhidi Mátyás 
> > wrote:
> >
> > > Hi Peng,
> > >
> > > Thanks for your feedback. Regarding 1. and 3. scenarios, the
> podTemplate
> > > functionality in the operator could cover both. We also need to be
> > careful
> > > about introducing proxy parameters in the CRD spec. The savepoint path
> is
> > > usually accompanied with a bunch of other configurations for example,
> so
> > > users need to use configuration params anyway. What do you think?
> > >
> > > Best,
> > > Matyas
> > >
> > > On Tue, Feb 15, 2022 at 8:58 AM K Fred  wrote:
> > >
> > > > Hi Gyula!
> > > >
> > > > I have reviewed the prototype design of flink-kubernetes-operator you
> > > > submitted, and I have the following questions:
> > > >
> > > > 1.Can a Flink Jar package that supports pulling from the sidecar be
> > added
> > > > to the JobSpec? just like this:
> > > >
> > > > > initContainers:
> > > > >   - name: downloader
> > > > > image: curlimages/curl
> > > > > env:
> > > > >   - name: JAR_URL
> > > > > value:
> > > > >
> > > >
> > >
> >
> https://repo1.maven.org/maven2/org/apache/flink/flink-examples-streaming_2.12/1.14.3/flink-examples-streaming_2.12-1.14.3-WordCount.jar
> > > > >   - name: DEST_PATH
> > > > > value: /cache/flink-app.jar
> > > > > command: ['sh', '-c', 'curl -o ${DEST_PATH} ${JAR_URL}']
> > > >
> > > > 2.Can we add savepoint path property to job specification?
> > > > 3.Can we add an extra port to the JobManagerSpec and TaskManagerSpec
> to
> > > > expose some service ,such as prometheus?The property can be this:
> > > >
> > > > > extraPorts:
> > > > >   - name: prom
> > > > > containerPort: 9249
> > > >
> > > >
> > > >
> > > > Best wishes,
> > > > Peng Yuan
> > > >
> > > > On Tue, Feb 15, 2022 at 12:23 AM Gyula Fóra 
> wrote:
> > > >
> > > > > Hi Flink Devs!
> > > > >
> > > > > We would like to present to you the first prototype of the
> > > > > flink-kubernetes-operator that was built based on the FLIP and the
> > > > > discussion on this mail thread. We would also like to call out some
> > > > design
> > > > > decisions that we have made regarding architecture components that
> > were
> > > > not
> > > > > explicitly mentioned in the FLIP document/thread and give you the
> > > > > opportunity to raise any concerns here.
> > > > >
> > > > > You can find the initial prototype here:
> > > > > https://github.com/apache/flink-kubernetes-operator/pull/1
> > > > >
> > > > > We will leave the PR open for 1-2 days before merging to let people
> > > > comment
> > > > > 

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-15 Thread Thomas Weise
Hi,

At this point I see no reason to support Java 8 for a new project.
Java 8 is being phased out, we should start with 11.

Also, since the operator isn't a library but effectively just a docker
image, the ability to change the Java version isn't as critical as it
is for Flink core, which needs to run in many different environments.

Cheers,
Thomas

On Tue, Feb 15, 2022 at 4:50 AM Gyula Fóra  wrote:
>
> Hi Devs,
>
> Yang Wang discovered that the current prototype is not compatible with Java
> 8 but only 11 and upwards.
>
> The reason for this is that the java operator SDK itself is not java 8
> compatible unfortunately.
>
> Given that Java 8 is on the road to deprecation and that the operator runs
> as a containerized deployment, are there any concerns regarding making the
> target java version 11?
> This should not affect deployed flink clusters and jobs, those should still
> work with Java 8, but only the kubernetes operator itself.
>
> Cheers,
> Gyula
>
>
> On Tue, Feb 15, 2022 at 1:06 PM Yang Wang  wrote:
>
> > I also lean to not introduce the savepoint/checkpoint related fields to the
> > job spec, especially in the very beginning of flink-kubernetes-operator.
> >
> >
> > Best,
> > Yang
> >
> > Gyula Fóra  于2022年2月15日周二 19:02写道:
> >
> > > Hi Peng Yuan!
> > >
> > > While I do agree that savepoint path is a very important production
> > > configuration there are a lot of other things that come to my mind:
> > >  - savepoint dir
> > >  - checkpoint dir
> > >  - checkpoint interval/timeout
> > >  - high availability settings (provider/storagedir etc)
> > >
> > > just to name a few...
> > >
> > > While these are all production critical, they have nice clean Flink
> > config
> > > settings to go with them. If we stand introducing these to jobspec we
> > only
> > > get confusion about priority order etc and it is going to be hard to
> > change
> > > or remove them in the future. In any case we should validate that these
> > > configs exist in cases where users use a stateful upgrade mode for
> > example.
> > > This is something we need to add for sure.
> > >
> > > As for the other options you mentioned like automatic savepoint
> > generation
> > > for instance, those deserve an independent discussion of their own I
> > > believe :)
> > >
> > > Cheers,
> > > Gyula
> > >
> > > On Tue, Feb 15, 2022 at 11:23 AM K Fred  wrote:
> > >
> > > > Hi Matyas!
> > > >
> > > > Thanks for your reply!
> > > > For 1. and 3. scenarios,I couldn't agree more with the podTemplate
> > > solution
> > > > , i missed this part.
> > > > For savepoint related configuration, I think it's very important to be
> > > > specified in JobSpec, Because savepoint is a very common configuration
> > > for
> > > > upgrading a job, if it has been placed in JobSpec can be obviously
> > > > configured by the user. In addition, other advanced properties can be
> > put
> > > > into flinkConfiguration customized by expert users.
> > > > A bunch of savepoint configuration as follows:
> > > >
> > > > > fromSavepoint——Job restart from
> > > >
> > > > autoSavepointSecond—— Automatically take a savepoint to the
> > > `savepointsDir`
> > > > > every n seconds.
> > > >
> > > > savepointsDir—— Savepoints dir where to store automatically taken
> > > > > savepoints
> > > >
> > > > savepointGeneration—— Update savepoint generation of job status for a
> > > > > running job (should be defined in JobStatus)
> > > >
> > > >
> > > > Best wishes,
> > > > Peng Yuan.
> > > >
> > > > On Tue, Feb 15, 2022 at 4:41 PM Őrhidi Mátyás  > >
> > > > wrote:
> > > >
> > > > > Hi Peng,
> > > > >
> > > > > Thanks for your feedback. Regarding 1. and 3. scenarios, the
> > > podTemplate
> > > > > functionality in the operator could cover both. We also need to be
> > > > careful
> > > > > about introducing proxy parameters in the CRD spec. The savepoint
> > path
> > > is
> > > > > usually accompanied with a bunch of other configurations for example,
> > > so
> > > > > users need to use configuration params anyway. What do you think?
> > > > >
> > > > > Best,
> > > > > Matyas
> > > > >
> > > > > On Tue, Feb 15, 2022 at 8:58 AM K Fred 
> > wrote:
> > > > >
> > > > > > Hi Gyula!
> > > > > >
> > > > > > I have reviewed the prototype design of flink-kubernetes-operator
> > you
> > > > > > submitted, and I have the following questions:
> > > > > >
> > > > > > 1.Can a Flink Jar package that supports pulling from the sidecar be
> > > > added
> > > > > > to the JobSpec? just like this:
> > > > > >
> > > > > > > initContainers:
> > > > > > >   - name: downloader
> > > > > > > image: curlimages/curl
> > > > > > > env:
> > > > > > >   - name: JAR_URL
> > > > > > > value:
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> > https://repo1.maven.org/maven2/org/apache/flink/flink-examples-streaming_2.12/1.14.3/flink-examples-streaming_2.12-1.14.3-WordCount.jar
> > > > > > >   - name: DEST_PATH
> > > > > > > value: /cache/flink-app.jar
> > > 

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-15 Thread Márton Balassi
Good catch, Yang Wang and Gyula on the Java version. I personally prefer
that we simply can not support Java 8 for the operator, since it is a net
new project we are better off starting support at Java 11 right away.

As Gyula outlined above, it is important to note that it only affects the
operator (and the operator container image), not existing or new Flink jobs.

On Tue, Feb 15, 2022 at 1:50 PM Gyula Fóra  wrote:

> Hi Devs,
>
> Yang Wang discovered that the current prototype is not compatible with Java
> 8 but only 11 and upwards.
>
> The reason for this is that the java operator SDK itself is not java 8
> compatible unfortunately.
>
> Given that Java 8 is on the road to deprecation and that the operator runs
> as a containerized deployment, are there any concerns regarding making the
> target java version 11?
> This should not affect deployed flink clusters and jobs, those should still
> work with Java 8, but only the kubernetes operator itself.
>
> Cheers,
> Gyula
>
>
> On Tue, Feb 15, 2022 at 1:06 PM Yang Wang  wrote:
>
> > I also lean to not introduce the savepoint/checkpoint related fields to
> the
> > job spec, especially in the very beginning of flink-kubernetes-operator.
> >
> >
> > Best,
> > Yang
> >
> > Gyula Fóra  于2022年2月15日周二 19:02写道:
> >
> > > Hi Peng Yuan!
> > >
> > > While I do agree that savepoint path is a very important production
> > > configuration there are a lot of other things that come to my mind:
> > >  - savepoint dir
> > >  - checkpoint dir
> > >  - checkpoint interval/timeout
> > >  - high availability settings (provider/storagedir etc)
> > >
> > > just to name a few...
> > >
> > > While these are all production critical, they have nice clean Flink
> > config
> > > settings to go with them. If we stand introducing these to jobspec we
> > only
> > > get confusion about priority order etc and it is going to be hard to
> > change
> > > or remove them in the future. In any case we should validate that these
> > > configs exist in cases where users use a stateful upgrade mode for
> > example.
> > > This is something we need to add for sure.
> > >
> > > As for the other options you mentioned like automatic savepoint
> > generation
> > > for instance, those deserve an independent discussion of their own I
> > > believe :)
> > >
> > > Cheers,
> > > Gyula
> > >
> > > On Tue, Feb 15, 2022 at 11:23 AM K Fred 
> wrote:
> > >
> > > > Hi Matyas!
> > > >
> > > > Thanks for your reply!
> > > > For 1. and 3. scenarios,I couldn't agree more with the podTemplate
> > > solution
> > > > , i missed this part.
> > > > For savepoint related configuration, I think it's very important to
> be
> > > > specified in JobSpec, Because savepoint is a very common
> configuration
> > > for
> > > > upgrading a job, if it has been placed in JobSpec can be obviously
> > > > configured by the user. In addition, other advanced properties can be
> > put
> > > > into flinkConfiguration customized by expert users.
> > > > A bunch of savepoint configuration as follows:
> > > >
> > > > > fromSavepoint——Job restart from
> > > >
> > > > autoSavepointSecond—— Automatically take a savepoint to the
> > > `savepointsDir`
> > > > > every n seconds.
> > > >
> > > > savepointsDir—— Savepoints dir where to store automatically taken
> > > > > savepoints
> > > >
> > > > savepointGeneration—— Update savepoint generation of job status for a
> > > > > running job (should be defined in JobStatus)
> > > >
> > > >
> > > > Best wishes,
> > > > Peng Yuan.
> > > >
> > > > On Tue, Feb 15, 2022 at 4:41 PM Őrhidi Mátyás <
> matyas.orh...@gmail.com
> > >
> > > > wrote:
> > > >
> > > > > Hi Peng,
> > > > >
> > > > > Thanks for your feedback. Regarding 1. and 3. scenarios, the
> > > podTemplate
> > > > > functionality in the operator could cover both. We also need to be
> > > > careful
> > > > > about introducing proxy parameters in the CRD spec. The savepoint
> > path
> > > is
> > > > > usually accompanied with a bunch of other configurations for
> example,
> > > so
> > > > > users need to use configuration params anyway. What do you think?
> > > > >
> > > > > Best,
> > > > > Matyas
> > > > >
> > > > > On Tue, Feb 15, 2022 at 8:58 AM K Fred 
> > wrote:
> > > > >
> > > > > > Hi Gyula!
> > > > > >
> > > > > > I have reviewed the prototype design of flink-kubernetes-operator
> > you
> > > > > > submitted, and I have the following questions:
> > > > > >
> > > > > > 1.Can a Flink Jar package that supports pulling from the sidecar
> be
> > > > added
> > > > > > to the JobSpec? just like this:
> > > > > >
> > > > > > > initContainers:
> > > > > > >   - name: downloader
> > > > > > > image: curlimages/curl
> > > > > > > env:
> > > > > > >   - name: JAR_URL
> > > > > > > value:
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://repo1.maven.org/maven2/org/apache/flink/flink-examples-streaming_2.12/1.14.3/flink-examples-streaming_2.12-1.14.3-WordCount.jar
> > > > > > >   - name: DEST_PATH
> 

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-15 Thread Gyula Fóra
Hi Devs,

Yang Wang discovered that the current prototype is not compatible with Java
8 but only 11 and upwards.

The reason for this is that the java operator SDK itself is not java 8
compatible unfortunately.

Given that Java 8 is on the road to deprecation and that the operator runs
as a containerized deployment, are there any concerns regarding making the
target java version 11?
This should not affect deployed flink clusters and jobs, those should still
work with Java 8, but only the kubernetes operator itself.

Cheers,
Gyula


On Tue, Feb 15, 2022 at 1:06 PM Yang Wang  wrote:

> I also lean to not introduce the savepoint/checkpoint related fields to the
> job spec, especially in the very beginning of flink-kubernetes-operator.
>
>
> Best,
> Yang
>
> Gyula Fóra  于2022年2月15日周二 19:02写道:
>
> > Hi Peng Yuan!
> >
> > While I do agree that savepoint path is a very important production
> > configuration there are a lot of other things that come to my mind:
> >  - savepoint dir
> >  - checkpoint dir
> >  - checkpoint interval/timeout
> >  - high availability settings (provider/storagedir etc)
> >
> > just to name a few...
> >
> > While these are all production critical, they have nice clean Flink
> config
> > settings to go with them. If we stand introducing these to jobspec we
> only
> > get confusion about priority order etc and it is going to be hard to
> change
> > or remove them in the future. In any case we should validate that these
> > configs exist in cases where users use a stateful upgrade mode for
> example.
> > This is something we need to add for sure.
> >
> > As for the other options you mentioned like automatic savepoint
> generation
> > for instance, those deserve an independent discussion of their own I
> > believe :)
> >
> > Cheers,
> > Gyula
> >
> > On Tue, Feb 15, 2022 at 11:23 AM K Fred  wrote:
> >
> > > Hi Matyas!
> > >
> > > Thanks for your reply!
> > > For 1. and 3. scenarios,I couldn't agree more with the podTemplate
> > solution
> > > , i missed this part.
> > > For savepoint related configuration, I think it's very important to be
> > > specified in JobSpec, Because savepoint is a very common configuration
> > for
> > > upgrading a job, if it has been placed in JobSpec can be obviously
> > > configured by the user. In addition, other advanced properties can be
> put
> > > into flinkConfiguration customized by expert users.
> > > A bunch of savepoint configuration as follows:
> > >
> > > > fromSavepoint——Job restart from
> > >
> > > autoSavepointSecond—— Automatically take a savepoint to the
> > `savepointsDir`
> > > > every n seconds.
> > >
> > > savepointsDir—— Savepoints dir where to store automatically taken
> > > > savepoints
> > >
> > > savepointGeneration—— Update savepoint generation of job status for a
> > > > running job (should be defined in JobStatus)
> > >
> > >
> > > Best wishes,
> > > Peng Yuan.
> > >
> > > On Tue, Feb 15, 2022 at 4:41 PM Őrhidi Mátyás  >
> > > wrote:
> > >
> > > > Hi Peng,
> > > >
> > > > Thanks for your feedback. Regarding 1. and 3. scenarios, the
> > podTemplate
> > > > functionality in the operator could cover both. We also need to be
> > > careful
> > > > about introducing proxy parameters in the CRD spec. The savepoint
> path
> > is
> > > > usually accompanied with a bunch of other configurations for example,
> > so
> > > > users need to use configuration params anyway. What do you think?
> > > >
> > > > Best,
> > > > Matyas
> > > >
> > > > On Tue, Feb 15, 2022 at 8:58 AM K Fred 
> wrote:
> > > >
> > > > > Hi Gyula!
> > > > >
> > > > > I have reviewed the prototype design of flink-kubernetes-operator
> you
> > > > > submitted, and I have the following questions:
> > > > >
> > > > > 1.Can a Flink Jar package that supports pulling from the sidecar be
> > > added
> > > > > to the JobSpec? just like this:
> > > > >
> > > > > > initContainers:
> > > > > >   - name: downloader
> > > > > > image: curlimages/curl
> > > > > > env:
> > > > > >   - name: JAR_URL
> > > > > > value:
> > > > > >
> > > > >
> > > >
> > >
> >
> https://repo1.maven.org/maven2/org/apache/flink/flink-examples-streaming_2.12/1.14.3/flink-examples-streaming_2.12-1.14.3-WordCount.jar
> > > > > >   - name: DEST_PATH
> > > > > > value: /cache/flink-app.jar
> > > > > > command: ['sh', '-c', 'curl -o ${DEST_PATH} ${JAR_URL}']
> > > > >
> > > > > 2.Can we add savepoint path property to job specification?
> > > > > 3.Can we add an extra port to the JobManagerSpec and
> TaskManagerSpec
> > to
> > > > > expose some service ,such as prometheus?The property can be this:
> > > > >
> > > > > > extraPorts:
> > > > > >   - name: prom
> > > > > > containerPort: 9249
> > > > >
> > > > >
> > > > >
> > > > > Best wishes,
> > > > > Peng Yuan
> > > > >
> > > > > On Tue, Feb 15, 2022 at 12:23 AM Gyula Fóra 
> > wrote:
> > > > >
> > > > > > Hi Flink Devs!
> > > > > >
> > > > > > We would like to present to you the first 

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-15 Thread Yang Wang
I also lean to not introduce the savepoint/checkpoint related fields to the
job spec, especially in the very beginning of flink-kubernetes-operator.


Best,
Yang

Gyula Fóra  于2022年2月15日周二 19:02写道:

> Hi Peng Yuan!
>
> While I do agree that savepoint path is a very important production
> configuration there are a lot of other things that come to my mind:
>  - savepoint dir
>  - checkpoint dir
>  - checkpoint interval/timeout
>  - high availability settings (provider/storagedir etc)
>
> just to name a few...
>
> While these are all production critical, they have nice clean Flink config
> settings to go with them. If we stand introducing these to jobspec we only
> get confusion about priority order etc and it is going to be hard to change
> or remove them in the future. In any case we should validate that these
> configs exist in cases where users use a stateful upgrade mode for example.
> This is something we need to add for sure.
>
> As for the other options you mentioned like automatic savepoint generation
> for instance, those deserve an independent discussion of their own I
> believe :)
>
> Cheers,
> Gyula
>
> On Tue, Feb 15, 2022 at 11:23 AM K Fred  wrote:
>
> > Hi Matyas!
> >
> > Thanks for your reply!
> > For 1. and 3. scenarios,I couldn't agree more with the podTemplate
> solution
> > , i missed this part.
> > For savepoint related configuration, I think it's very important to be
> > specified in JobSpec, Because savepoint is a very common configuration
> for
> > upgrading a job, if it has been placed in JobSpec can be obviously
> > configured by the user. In addition, other advanced properties can be put
> > into flinkConfiguration customized by expert users.
> > A bunch of savepoint configuration as follows:
> >
> > > fromSavepoint——Job restart from
> >
> > autoSavepointSecond—— Automatically take a savepoint to the
> `savepointsDir`
> > > every n seconds.
> >
> > savepointsDir—— Savepoints dir where to store automatically taken
> > > savepoints
> >
> > savepointGeneration—— Update savepoint generation of job status for a
> > > running job (should be defined in JobStatus)
> >
> >
> > Best wishes,
> > Peng Yuan.
> >
> > On Tue, Feb 15, 2022 at 4:41 PM Őrhidi Mátyás 
> > wrote:
> >
> > > Hi Peng,
> > >
> > > Thanks for your feedback. Regarding 1. and 3. scenarios, the
> podTemplate
> > > functionality in the operator could cover both. We also need to be
> > careful
> > > about introducing proxy parameters in the CRD spec. The savepoint path
> is
> > > usually accompanied with a bunch of other configurations for example,
> so
> > > users need to use configuration params anyway. What do you think?
> > >
> > > Best,
> > > Matyas
> > >
> > > On Tue, Feb 15, 2022 at 8:58 AM K Fred  wrote:
> > >
> > > > Hi Gyula!
> > > >
> > > > I have reviewed the prototype design of flink-kubernetes-operator you
> > > > submitted, and I have the following questions:
> > > >
> > > > 1.Can a Flink Jar package that supports pulling from the sidecar be
> > added
> > > > to the JobSpec? just like this:
> > > >
> > > > > initContainers:
> > > > >   - name: downloader
> > > > > image: curlimages/curl
> > > > > env:
> > > > >   - name: JAR_URL
> > > > > value:
> > > > >
> > > >
> > >
> >
> https://repo1.maven.org/maven2/org/apache/flink/flink-examples-streaming_2.12/1.14.3/flink-examples-streaming_2.12-1.14.3-WordCount.jar
> > > > >   - name: DEST_PATH
> > > > > value: /cache/flink-app.jar
> > > > > command: ['sh', '-c', 'curl -o ${DEST_PATH} ${JAR_URL}']
> > > >
> > > > 2.Can we add savepoint path property to job specification?
> > > > 3.Can we add an extra port to the JobManagerSpec and TaskManagerSpec
> to
> > > > expose some service ,such as prometheus?The property can be this:
> > > >
> > > > > extraPorts:
> > > > >   - name: prom
> > > > > containerPort: 9249
> > > >
> > > >
> > > >
> > > > Best wishes,
> > > > Peng Yuan
> > > >
> > > > On Tue, Feb 15, 2022 at 12:23 AM Gyula Fóra 
> wrote:
> > > >
> > > > > Hi Flink Devs!
> > > > >
> > > > > We would like to present to you the first prototype of the
> > > > > flink-kubernetes-operator that was built based on the FLIP and the
> > > > > discussion on this mail thread. We would also like to call out some
> > > > design
> > > > > decisions that we have made regarding architecture components that
> > were
> > > > not
> > > > > explicitly mentioned in the FLIP document/thread and give you the
> > > > > opportunity to raise any concerns here.
> > > > >
> > > > > You can find the initial prototype here:
> > > > > https://github.com/apache/flink-kubernetes-operator/pull/1
> > > > >
> > > > > We will leave the PR open for 1-2 days before merging to let people
> > > > comment
> > > > > on it, but please be mindful that this is an initial prototype with
> > > many
> > > > > rough edges. It is not intended to be a complete implementation of
> > the
> > > > FLIP
> > > > > specs as that will take some more 

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-15 Thread Gyula Fóra
Hi Peng Yuan!

While I do agree that savepoint path is a very important production
configuration there are a lot of other things that come to my mind:
 - savepoint dir
 - checkpoint dir
 - checkpoint interval/timeout
 - high availability settings (provider/storagedir etc)

just to name a few...

While these are all production critical, they have nice clean Flink config
settings to go with them. If we stand introducing these to jobspec we only
get confusion about priority order etc and it is going to be hard to change
or remove them in the future. In any case we should validate that these
configs exist in cases where users use a stateful upgrade mode for example.
This is something we need to add for sure.

As for the other options you mentioned like automatic savepoint generation
for instance, those deserve an independent discussion of their own I
believe :)

Cheers,
Gyula

On Tue, Feb 15, 2022 at 11:23 AM K Fred  wrote:

> Hi Matyas!
>
> Thanks for your reply!
> For 1. and 3. scenarios,I couldn't agree more with the podTemplate solution
> , i missed this part.
> For savepoint related configuration, I think it's very important to be
> specified in JobSpec, Because savepoint is a very common configuration for
> upgrading a job, if it has been placed in JobSpec can be obviously
> configured by the user. In addition, other advanced properties can be put
> into flinkConfiguration customized by expert users.
> A bunch of savepoint configuration as follows:
>
> > fromSavepoint——Job restart from
>
> autoSavepointSecond—— Automatically take a savepoint to the `savepointsDir`
> > every n seconds.
>
> savepointsDir—— Savepoints dir where to store automatically taken
> > savepoints
>
> savepointGeneration—— Update savepoint generation of job status for a
> > running job (should be defined in JobStatus)
>
>
> Best wishes,
> Peng Yuan.
>
> On Tue, Feb 15, 2022 at 4:41 PM Őrhidi Mátyás 
> wrote:
>
> > Hi Peng,
> >
> > Thanks for your feedback. Regarding 1. and 3. scenarios, the podTemplate
> > functionality in the operator could cover both. We also need to be
> careful
> > about introducing proxy parameters in the CRD spec. The savepoint path is
> > usually accompanied with a bunch of other configurations for example, so
> > users need to use configuration params anyway. What do you think?
> >
> > Best,
> > Matyas
> >
> > On Tue, Feb 15, 2022 at 8:58 AM K Fred  wrote:
> >
> > > Hi Gyula!
> > >
> > > I have reviewed the prototype design of flink-kubernetes-operator you
> > > submitted, and I have the following questions:
> > >
> > > 1.Can a Flink Jar package that supports pulling from the sidecar be
> added
> > > to the JobSpec? just like this:
> > >
> > > > initContainers:
> > > >   - name: downloader
> > > > image: curlimages/curl
> > > > env:
> > > >   - name: JAR_URL
> > > > value:
> > > >
> > >
> >
> https://repo1.maven.org/maven2/org/apache/flink/flink-examples-streaming_2.12/1.14.3/flink-examples-streaming_2.12-1.14.3-WordCount.jar
> > > >   - name: DEST_PATH
> > > > value: /cache/flink-app.jar
> > > > command: ['sh', '-c', 'curl -o ${DEST_PATH} ${JAR_URL}']
> > >
> > > 2.Can we add savepoint path property to job specification?
> > > 3.Can we add an extra port to the JobManagerSpec and TaskManagerSpec to
> > > expose some service ,such as prometheus?The property can be this:
> > >
> > > > extraPorts:
> > > >   - name: prom
> > > > containerPort: 9249
> > >
> > >
> > >
> > > Best wishes,
> > > Peng Yuan
> > >
> > > On Tue, Feb 15, 2022 at 12:23 AM Gyula Fóra  wrote:
> > >
> > > > Hi Flink Devs!
> > > >
> > > > We would like to present to you the first prototype of the
> > > > flink-kubernetes-operator that was built based on the FLIP and the
> > > > discussion on this mail thread. We would also like to call out some
> > > design
> > > > decisions that we have made regarding architecture components that
> were
> > > not
> > > > explicitly mentioned in the FLIP document/thread and give you the
> > > > opportunity to raise any concerns here.
> > > >
> > > > You can find the initial prototype here:
> > > > https://github.com/apache/flink-kubernetes-operator/pull/1
> > > >
> > > > We will leave the PR open for 1-2 days before merging to let people
> > > comment
> > > > on it, but please be mindful that this is an initial prototype with
> > many
> > > > rough edges. It is not intended to be a complete implementation of
> the
> > > FLIP
> > > > specs as that will take some more work from all of us :)
> > > >
> > > >
> > > > *Prototype feature set:*The prototype contains a basic working
> version
> > of
> > > > the flink-kubernetes-operator that supports deployment and lifecycle
> > > > management of a stateful native flink application. We have basic
> > support
> > > > for stateful and stateless upgrades, UI ingress, pod templates etc.
> > Error
> > > > handling at this point is largely missing.
> > > >
> > > >
> > > > *Features / design 

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-15 Thread K Fred
Hi Matyas!

Thanks for your reply!
For 1. and 3. scenarios,I couldn't agree more with the podTemplate solution
, i missed this part.
For savepoint related configuration, I think it's very important to be
specified in JobSpec, Because savepoint is a very common configuration for
upgrading a job, if it has been placed in JobSpec can be obviously
configured by the user. In addition, other advanced properties can be put
into flinkConfiguration customized by expert users.
A bunch of savepoint configuration as follows:

> fromSavepoint——Job restart from

autoSavepointSecond—— Automatically take a savepoint to the `savepointsDir`
> every n seconds.

savepointsDir—— Savepoints dir where to store automatically taken
> savepoints

savepointGeneration—— Update savepoint generation of job status for a
> running job (should be defined in JobStatus)


Best wishes,
Peng Yuan.

On Tue, Feb 15, 2022 at 4:41 PM Őrhidi Mátyás 
wrote:

> Hi Peng,
>
> Thanks for your feedback. Regarding 1. and 3. scenarios, the podTemplate
> functionality in the operator could cover both. We also need to be careful
> about introducing proxy parameters in the CRD spec. The savepoint path is
> usually accompanied with a bunch of other configurations for example, so
> users need to use configuration params anyway. What do you think?
>
> Best,
> Matyas
>
> On Tue, Feb 15, 2022 at 8:58 AM K Fred  wrote:
>
> > Hi Gyula!
> >
> > I have reviewed the prototype design of flink-kubernetes-operator you
> > submitted, and I have the following questions:
> >
> > 1.Can a Flink Jar package that supports pulling from the sidecar be added
> > to the JobSpec? just like this:
> >
> > > initContainers:
> > >   - name: downloader
> > > image: curlimages/curl
> > > env:
> > >   - name: JAR_URL
> > > value:
> > >
> >
> https://repo1.maven.org/maven2/org/apache/flink/flink-examples-streaming_2.12/1.14.3/flink-examples-streaming_2.12-1.14.3-WordCount.jar
> > >   - name: DEST_PATH
> > > value: /cache/flink-app.jar
> > > command: ['sh', '-c', 'curl -o ${DEST_PATH} ${JAR_URL}']
> >
> > 2.Can we add savepoint path property to job specification?
> > 3.Can we add an extra port to the JobManagerSpec and TaskManagerSpec to
> > expose some service ,such as prometheus?The property can be this:
> >
> > > extraPorts:
> > >   - name: prom
> > > containerPort: 9249
> >
> >
> >
> > Best wishes,
> > Peng Yuan
> >
> > On Tue, Feb 15, 2022 at 12:23 AM Gyula Fóra  wrote:
> >
> > > Hi Flink Devs!
> > >
> > > We would like to present to you the first prototype of the
> > > flink-kubernetes-operator that was built based on the FLIP and the
> > > discussion on this mail thread. We would also like to call out some
> > design
> > > decisions that we have made regarding architecture components that were
> > not
> > > explicitly mentioned in the FLIP document/thread and give you the
> > > opportunity to raise any concerns here.
> > >
> > > You can find the initial prototype here:
> > > https://github.com/apache/flink-kubernetes-operator/pull/1
> > >
> > > We will leave the PR open for 1-2 days before merging to let people
> > comment
> > > on it, but please be mindful that this is an initial prototype with
> many
> > > rough edges. It is not intended to be a complete implementation of the
> > FLIP
> > > specs as that will take some more work from all of us :)
> > >
> > >
> > > *Prototype feature set:*The prototype contains a basic working version
> of
> > > the flink-kubernetes-operator that supports deployment and lifecycle
> > > management of a stateful native flink application. We have basic
> support
> > > for stateful and stateless upgrades, UI ingress, pod templates etc.
> Error
> > > handling at this point is largely missing.
> > >
> > >
> > > *Features / design decisions that were not explicitly discussed in this
> > > thread*
> > >
> > > *Basic Admission control using a Webhook*Standard resource admission
> > > control in Kubernetes to validate and potentially reject resources is
> > done
> > > through Webhooks.
> > >
> > >
> >
> https://kubernetes.io/docs/reference/access-authn-authz/extensible-admission-controllers/
> > > This is a necessary mechanism to give the user an upfront error when an
> > > incorrect resource was submitted. In the Flink operator's case we need
> to
> > > validate that the FlinkDeployment yaml actually makes sense and does
> not
> > > contain erroneous config options that would inevitably lead to
> > > deployment/job failures.
> > >
> > > We have implemented a simple webhook that we can use for this type of
> > > validation, as a separate maven module (flink-kubernetes-webhook). The
> > > webhook is an optional component and can be enabled or disabled during
> > > deployment. To avoid pulling in new external dependencies we have used
> > the
> > > Flink Shaded Netty module to build the simple rest endpoint required.
> If
> > > the community feels that Netty adds unnecessary 

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-15 Thread Őrhidi Mátyás
Hi Peng,

Thanks for your feedback. Regarding 1. and 3. scenarios, the podTemplate
functionality in the operator could cover both. We also need to be careful
about introducing proxy parameters in the CRD spec. The savepoint path is
usually accompanied with a bunch of other configurations for example, so
users need to use configuration params anyway. What do you think?

Best,
Matyas

On Tue, Feb 15, 2022 at 8:58 AM K Fred  wrote:

> Hi Gyula!
>
> I have reviewed the prototype design of flink-kubernetes-operator you
> submitted, and I have the following questions:
>
> 1.Can a Flink Jar package that supports pulling from the sidecar be added
> to the JobSpec? just like this:
>
> > initContainers:
> >   - name: downloader
> > image: curlimages/curl
> > env:
> >   - name: JAR_URL
> > value:
> >
> https://repo1.maven.org/maven2/org/apache/flink/flink-examples-streaming_2.12/1.14.3/flink-examples-streaming_2.12-1.14.3-WordCount.jar
> >   - name: DEST_PATH
> > value: /cache/flink-app.jar
> > command: ['sh', '-c', 'curl -o ${DEST_PATH} ${JAR_URL}']
>
> 2.Can we add savepoint path property to job specification?
> 3.Can we add an extra port to the JobManagerSpec and TaskManagerSpec to
> expose some service ,such as prometheus?The property can be this:
>
> > extraPorts:
> >   - name: prom
> > containerPort: 9249
>
>
>
> Best wishes,
> Peng Yuan
>
> On Tue, Feb 15, 2022 at 12:23 AM Gyula Fóra  wrote:
>
> > Hi Flink Devs!
> >
> > We would like to present to you the first prototype of the
> > flink-kubernetes-operator that was built based on the FLIP and the
> > discussion on this mail thread. We would also like to call out some
> design
> > decisions that we have made regarding architecture components that were
> not
> > explicitly mentioned in the FLIP document/thread and give you the
> > opportunity to raise any concerns here.
> >
> > You can find the initial prototype here:
> > https://github.com/apache/flink-kubernetes-operator/pull/1
> >
> > We will leave the PR open for 1-2 days before merging to let people
> comment
> > on it, but please be mindful that this is an initial prototype with many
> > rough edges. It is not intended to be a complete implementation of the
> FLIP
> > specs as that will take some more work from all of us :)
> >
> >
> > *Prototype feature set:*The prototype contains a basic working version of
> > the flink-kubernetes-operator that supports deployment and lifecycle
> > management of a stateful native flink application. We have basic support
> > for stateful and stateless upgrades, UI ingress, pod templates etc. Error
> > handling at this point is largely missing.
> >
> >
> > *Features / design decisions that were not explicitly discussed in this
> > thread*
> >
> > *Basic Admission control using a Webhook*Standard resource admission
> > control in Kubernetes to validate and potentially reject resources is
> done
> > through Webhooks.
> >
> >
> https://kubernetes.io/docs/reference/access-authn-authz/extensible-admission-controllers/
> > This is a necessary mechanism to give the user an upfront error when an
> > incorrect resource was submitted. In the Flink operator's case we need to
> > validate that the FlinkDeployment yaml actually makes sense and does not
> > contain erroneous config options that would inevitably lead to
> > deployment/job failures.
> >
> > We have implemented a simple webhook that we can use for this type of
> > validation, as a separate maven module (flink-kubernetes-webhook). The
> > webhook is an optional component and can be enabled or disabled during
> > deployment. To avoid pulling in new external dependencies we have used
> the
> > Flink Shaded Netty module to build the simple rest endpoint required. If
> > the community feels that Netty adds unnecessary complexity to the webhook
> > implementation we are open to alternative backends such as Springboot for
> > instance which would practically eliminate all the boilerplate.
> >
> >
> > *Helm Chart for deployment*Helm charts provide an industry standard way
> of
> > managing kubernetes deployments. We have created a helm chart prototype
> > that can be used to deploy the operator together with all required
> > resources. The helm chart allows easy configuration for things like
> images,
> > namespaces etc and flags to control specific parts of the deployment such
> > as RBAC or the webhook.
> >
> > The helm chart provided is intended to be a first version that worked for
> > us during development but we expect to have a lot of iterations on it
> based
> > on the feedback from the community.
> >
> > *Acknowledgment*
> > We would like to thank everyone who has provided support and valuable
> > feedback on this FLIP.
> > We would also like to thank Yang Wang & Alexis Sarda-Espinosa
> specifically
> > for making their operators open source and available to us which had a
> big
> > impact on the FLIP and the prototype.
> >
> > We are 

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-14 Thread K Fred
Hi Gyula!

I have reviewed the prototype design of flink-kubernetes-operator you
submitted, and I have the following questions:

1.Can a Flink Jar package that supports pulling from the sidecar be added
to the JobSpec? just like this:

> initContainers:
>   - name: downloader
> image: curlimages/curl
> env:
>   - name: JAR_URL
> value:
> https://repo1.maven.org/maven2/org/apache/flink/flink-examples-streaming_2.12/1.14.3/flink-examples-streaming_2.12-1.14.3-WordCount.jar
>   - name: DEST_PATH
> value: /cache/flink-app.jar
> command: ['sh', '-c', 'curl -o ${DEST_PATH} ${JAR_URL}']

2.Can we add savepoint path property to job specification?
3.Can we add an extra port to the JobManagerSpec and TaskManagerSpec to
expose some service ,such as prometheus?The property can be this:

> extraPorts:
>   - name: prom
> containerPort: 9249



Best wishes,
Peng Yuan

On Tue, Feb 15, 2022 at 12:23 AM Gyula Fóra  wrote:

> Hi Flink Devs!
>
> We would like to present to you the first prototype of the
> flink-kubernetes-operator that was built based on the FLIP and the
> discussion on this mail thread. We would also like to call out some design
> decisions that we have made regarding architecture components that were not
> explicitly mentioned in the FLIP document/thread and give you the
> opportunity to raise any concerns here.
>
> You can find the initial prototype here:
> https://github.com/apache/flink-kubernetes-operator/pull/1
>
> We will leave the PR open for 1-2 days before merging to let people comment
> on it, but please be mindful that this is an initial prototype with many
> rough edges. It is not intended to be a complete implementation of the FLIP
> specs as that will take some more work from all of us :)
>
>
> *Prototype feature set:*The prototype contains a basic working version of
> the flink-kubernetes-operator that supports deployment and lifecycle
> management of a stateful native flink application. We have basic support
> for stateful and stateless upgrades, UI ingress, pod templates etc. Error
> handling at this point is largely missing.
>
>
> *Features / design decisions that were not explicitly discussed in this
> thread*
>
> *Basic Admission control using a Webhook*Standard resource admission
> control in Kubernetes to validate and potentially reject resources is done
> through Webhooks.
>
> https://kubernetes.io/docs/reference/access-authn-authz/extensible-admission-controllers/
> This is a necessary mechanism to give the user an upfront error when an
> incorrect resource was submitted. In the Flink operator's case we need to
> validate that the FlinkDeployment yaml actually makes sense and does not
> contain erroneous config options that would inevitably lead to
> deployment/job failures.
>
> We have implemented a simple webhook that we can use for this type of
> validation, as a separate maven module (flink-kubernetes-webhook). The
> webhook is an optional component and can be enabled or disabled during
> deployment. To avoid pulling in new external dependencies we have used the
> Flink Shaded Netty module to build the simple rest endpoint required. If
> the community feels that Netty adds unnecessary complexity to the webhook
> implementation we are open to alternative backends such as Springboot for
> instance which would practically eliminate all the boilerplate.
>
>
> *Helm Chart for deployment*Helm charts provide an industry standard way of
> managing kubernetes deployments. We have created a helm chart prototype
> that can be used to deploy the operator together with all required
> resources. The helm chart allows easy configuration for things like images,
> namespaces etc and flags to control specific parts of the deployment such
> as RBAC or the webhook.
>
> The helm chart provided is intended to be a first version that worked for
> us during development but we expect to have a lot of iterations on it based
> on the feedback from the community.
>
> *Acknowledgment*
> We would like to thank everyone who has provided support and valuable
> feedback on this FLIP.
> We would also like to thank Yang Wang & Alexis Sarda-Espinosa specifically
> for making their operators open source and available to us which had a big
> impact on the FLIP and the prototype.
>
> We are looking forward to continuing development on the operator together
> with the broader community.
> All work will be tracked using the ASF Jira from now on.
>
> Cheers,
> Gyula
>
> On Mon, Feb 14, 2022 at 9:21 AM K Fred  wrote:
>
> > Hi Gyula,
> >
> > Thanks!
> > It's great to see the project getting started and I can't wait to see the
> > PR and start contributing code.
> >
> > Best Wishes!
> > Peng Yuan
> >
> > On Mon, Feb 14, 2022 at 4:14 PM Gyula Fóra  wrote:
> >
> > > Hi Peng Yuan!
> > >
> > > The repo is already created:
> > > https://github.com/apache/flink-kubernetes-operator
> > >
> > > We will open the PR with the initial prototype later 

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-14 Thread Gyula Fóra
Hi Flink Devs!

We would like to present to you the first prototype of the
flink-kubernetes-operator that was built based on the FLIP and the
discussion on this mail thread. We would also like to call out some design
decisions that we have made regarding architecture components that were not
explicitly mentioned in the FLIP document/thread and give you the
opportunity to raise any concerns here.

You can find the initial prototype here:
https://github.com/apache/flink-kubernetes-operator/pull/1

We will leave the PR open for 1-2 days before merging to let people comment
on it, but please be mindful that this is an initial prototype with many
rough edges. It is not intended to be a complete implementation of the FLIP
specs as that will take some more work from all of us :)


*Prototype feature set:*The prototype contains a basic working version of
the flink-kubernetes-operator that supports deployment and lifecycle
management of a stateful native flink application. We have basic support
for stateful and stateless upgrades, UI ingress, pod templates etc. Error
handling at this point is largely missing.


*Features / design decisions that were not explicitly discussed in this
thread*

*Basic Admission control using a Webhook*Standard resource admission
control in Kubernetes to validate and potentially reject resources is done
through Webhooks.
https://kubernetes.io/docs/reference/access-authn-authz/extensible-admission-controllers/
This is a necessary mechanism to give the user an upfront error when an
incorrect resource was submitted. In the Flink operator's case we need to
validate that the FlinkDeployment yaml actually makes sense and does not
contain erroneous config options that would inevitably lead to
deployment/job failures.

We have implemented a simple webhook that we can use for this type of
validation, as a separate maven module (flink-kubernetes-webhook). The
webhook is an optional component and can be enabled or disabled during
deployment. To avoid pulling in new external dependencies we have used the
Flink Shaded Netty module to build the simple rest endpoint required. If
the community feels that Netty adds unnecessary complexity to the webhook
implementation we are open to alternative backends such as Springboot for
instance which would practically eliminate all the boilerplate.


*Helm Chart for deployment*Helm charts provide an industry standard way of
managing kubernetes deployments. We have created a helm chart prototype
that can be used to deploy the operator together with all required
resources. The helm chart allows easy configuration for things like images,
namespaces etc and flags to control specific parts of the deployment such
as RBAC or the webhook.

The helm chart provided is intended to be a first version that worked for
us during development but we expect to have a lot of iterations on it based
on the feedback from the community.

*Acknowledgment*
We would like to thank everyone who has provided support and valuable
feedback on this FLIP.
We would also like to thank Yang Wang & Alexis Sarda-Espinosa specifically
for making their operators open source and available to us which had a big
impact on the FLIP and the prototype.

We are looking forward to continuing development on the operator together
with the broader community.
All work will be tracked using the ASF Jira from now on.

Cheers,
Gyula

On Mon, Feb 14, 2022 at 9:21 AM K Fred  wrote:

> Hi Gyula,
>
> Thanks!
> It's great to see the project getting started and I can't wait to see the
> PR and start contributing code.
>
> Best Wishes!
> Peng Yuan
>
> On Mon, Feb 14, 2022 at 4:14 PM Gyula Fóra  wrote:
>
> > Hi Peng Yuan!
> >
> > The repo is already created:
> > https://github.com/apache/flink-kubernetes-operator
> >
> > We will open the PR with the initial prototype later today, stay tuned in
> > this thread! :)
> >
> > Cheers,
> > Gyula
> >
> > On Mon, Feb 14, 2022 at 9:09 AM K Fred  wrote:
> >
> > > Hi All,
> > >
> > > Has the project of flink-kubernetes-operator been created in github?
> > >
> > > Peng Yuan
> > >
> > > On Wed, Feb 9, 2022 at 1:23 AM Gyula Fóra 
> wrote:
> > >
> > > > I agree with flink-kubernetes-operator as the repo name :)
> > > > Don't have any better idea
> > > >
> > > > Gyula
> > > >
> > > > On Sat, Feb 5, 2022 at 2:41 AM Thomas Weise  wrote:
> > > >
> > > > > Hi,
> > > > >
> > > > > Thanks for the continued feedback and discussion. Looks like we are
> > > > > ready to start a VOTE, I will initiate it shortly.
> > > > >
> > > > > In parallel it would be good to find the repository name.
> > > > >
> > > > > My suggestion would be: flink-kubernetes-operator
> > > > >
> > > > > I thought "flink-operator" could be a bit misleading since the term
> > > > > operator already has a meaning in Flink.
> > > > >
> > > > > I also considered "flink-k8s-operator" but that would be almost
> > > > > identical to existing operator implementations and could lead to
> > > > > confusion in the future.
> > > > >
> > 

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-14 Thread K Fred
Hi Gyula,

Thanks!
It's great to see the project getting started and I can't wait to see the
PR and start contributing code.

Best Wishes!
Peng Yuan

On Mon, Feb 14, 2022 at 4:14 PM Gyula Fóra  wrote:

> Hi Peng Yuan!
>
> The repo is already created:
> https://github.com/apache/flink-kubernetes-operator
>
> We will open the PR with the initial prototype later today, stay tuned in
> this thread! :)
>
> Cheers,
> Gyula
>
> On Mon, Feb 14, 2022 at 9:09 AM K Fred  wrote:
>
> > Hi All,
> >
> > Has the project of flink-kubernetes-operator been created in github?
> >
> > Peng Yuan
> >
> > On Wed, Feb 9, 2022 at 1:23 AM Gyula Fóra  wrote:
> >
> > > I agree with flink-kubernetes-operator as the repo name :)
> > > Don't have any better idea
> > >
> > > Gyula
> > >
> > > On Sat, Feb 5, 2022 at 2:41 AM Thomas Weise  wrote:
> > >
> > > > Hi,
> > > >
> > > > Thanks for the continued feedback and discussion. Looks like we are
> > > > ready to start a VOTE, I will initiate it shortly.
> > > >
> > > > In parallel it would be good to find the repository name.
> > > >
> > > > My suggestion would be: flink-kubernetes-operator
> > > >
> > > > I thought "flink-operator" could be a bit misleading since the term
> > > > operator already has a meaning in Flink.
> > > >
> > > > I also considered "flink-k8s-operator" but that would be almost
> > > > identical to existing operator implementations and could lead to
> > > > confusion in the future.
> > > >
> > > > Thoughts?
> > > >
> > > > Thanks,
> > > > Thomas
> > > >
> > > >
> > > >
> > > > On Fri, Feb 4, 2022 at 5:15 AM Gyula Fóra 
> > wrote:
> > > > >
> > > > > Hi Danny,
> > > > >
> > > > > So far we have been focusing our dev efforts on the initial native
> > > > > implementation with the team.
> > > > > If the discussion and vote goes well for this FLIP we are looking
> > > forward
> > > > > to contributing the initial version sometime next week (fingers
> > > crossed).
> > > > >
> > > > > At that point I think we can already start the dev work to support
> > the
> > > > > standalone mode as well, especially if you can dedicate some effort
> > to
> > > > > pushing that side.
> > > > > Working together on this sounds like a great idea and we should
> start
> > > as
> > > > > soon as possible! :)
> > > > >
> > > > > Cheers,
> > > > > Gyula
> > > > >
> > > > > On Fri, Feb 4, 2022 at 2:07 PM Danny Cranmer <
> > dannycran...@apache.org>
> > > > > wrote:
> > > > >
> > > > > > I have been discussing this one with my team. We are interested
> in
> > > the
> > > > > > Standalone mode, and are willing to contribute towards the
> > > > implementation.
> > > > > > Potentially we can work together to support both modes in
> parallel?
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > On Wed, Feb 2, 2022 at 4:02 PM Gyula Fóra 
> > > > wrote:
> > > > > >
> > > > > > > Hi Danny!
> > > > > > >
> > > > > > > Thanks for the feedback :)
> > > > > > >
> > > > > > > Versioning:
> > > > > > > Versioning will be independent from Flink and the operator will
> > > > depend
> > > > > > on a
> > > > > > > fixed flink version (in every given operator version).
> > > > > > > This should be the exact same setup as with Stateful Functions
> (
> > > > > > > https://github.com/apache/flink-statefun). So independent
> > release
> > > > cycle
> > > > > > > but
> > > > > > > still within the Flink umbrella.
> > > > > > >
> > > > > > > Deployment error handling:
> > > > > > > I think that's a very good point, as general exception handling
> > for
> > > > the
> > > > > > > different failure scenarios is a tricky problem. I think the
> > > > exception
> > > > > > > classifiers and retry strategies could avoid a lot of manual
> > > > intervention
> > > > > > > from the user. We will definitely need to add something like
> > this.
> > > > Once
> > > > > > we
> > > > > > > have the repo created with the initial operator code we should
> > open
> > > > some
> > > > > > > tickets for this and put it on the short term roadmap!
> > > > > > >
> > > > > > > Cheers,
> > > > > > > Gyula
> > > > > > >
> > > > > > > On Wed, Feb 2, 2022 at 4:50 PM Danny Cranmer <
> > > > dannycran...@apache.org>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Hey team,
> > > > > > > >
> > > > > > > > Great work on the FLIP, I am looking forward to this one. I
> > agree
> > > > that
> > > > > > we
> > > > > > > > can move forward to the voting stage.
> > > > > > > >
> > > > > > > > I have general feedback around how we will handle job
> > submission
> > > > > > failure
> > > > > > > > and retry. As discussed in the Rejected Alternatives section,
> > we
> > > > can
> > > > > > use
> > > > > > > > Java to handle job submission failures from the Flink client.
> > It
> > > > would
> > > > > > be
> > > > > > > > useful to have the ability to configure exception classifiers
> > and
> > > > retry
> > > > > > > > strategy as part of operator configuration.
> > > > > > > >
> > > > > > > > Given this will be in a separate Github repository I am
> 

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-14 Thread Gyula Fóra
Hi Peng Yuan!

The repo is already created:
https://github.com/apache/flink-kubernetes-operator

We will open the PR with the initial prototype later today, stay tuned in
this thread! :)

Cheers,
Gyula

On Mon, Feb 14, 2022 at 9:09 AM K Fred  wrote:

> Hi All,
>
> Has the project of flink-kubernetes-operator been created in github?
>
> Peng Yuan
>
> On Wed, Feb 9, 2022 at 1:23 AM Gyula Fóra  wrote:
>
> > I agree with flink-kubernetes-operator as the repo name :)
> > Don't have any better idea
> >
> > Gyula
> >
> > On Sat, Feb 5, 2022 at 2:41 AM Thomas Weise  wrote:
> >
> > > Hi,
> > >
> > > Thanks for the continued feedback and discussion. Looks like we are
> > > ready to start a VOTE, I will initiate it shortly.
> > >
> > > In parallel it would be good to find the repository name.
> > >
> > > My suggestion would be: flink-kubernetes-operator
> > >
> > > I thought "flink-operator" could be a bit misleading since the term
> > > operator already has a meaning in Flink.
> > >
> > > I also considered "flink-k8s-operator" but that would be almost
> > > identical to existing operator implementations and could lead to
> > > confusion in the future.
> > >
> > > Thoughts?
> > >
> > > Thanks,
> > > Thomas
> > >
> > >
> > >
> > > On Fri, Feb 4, 2022 at 5:15 AM Gyula Fóra 
> wrote:
> > > >
> > > > Hi Danny,
> > > >
> > > > So far we have been focusing our dev efforts on the initial native
> > > > implementation with the team.
> > > > If the discussion and vote goes well for this FLIP we are looking
> > forward
> > > > to contributing the initial version sometime next week (fingers
> > crossed).
> > > >
> > > > At that point I think we can already start the dev work to support
> the
> > > > standalone mode as well, especially if you can dedicate some effort
> to
> > > > pushing that side.
> > > > Working together on this sounds like a great idea and we should start
> > as
> > > > soon as possible! :)
> > > >
> > > > Cheers,
> > > > Gyula
> > > >
> > > > On Fri, Feb 4, 2022 at 2:07 PM Danny Cranmer <
> dannycran...@apache.org>
> > > > wrote:
> > > >
> > > > > I have been discussing this one with my team. We are interested in
> > the
> > > > > Standalone mode, and are willing to contribute towards the
> > > implementation.
> > > > > Potentially we can work together to support both modes in parallel?
> > > > >
> > > > > Thanks,
> > > > >
> > > > > On Wed, Feb 2, 2022 at 4:02 PM Gyula Fóra 
> > > wrote:
> > > > >
> > > > > > Hi Danny!
> > > > > >
> > > > > > Thanks for the feedback :)
> > > > > >
> > > > > > Versioning:
> > > > > > Versioning will be independent from Flink and the operator will
> > > depend
> > > > > on a
> > > > > > fixed flink version (in every given operator version).
> > > > > > This should be the exact same setup as with Stateful Functions (
> > > > > > https://github.com/apache/flink-statefun). So independent
> release
> > > cycle
> > > > > > but
> > > > > > still within the Flink umbrella.
> > > > > >
> > > > > > Deployment error handling:
> > > > > > I think that's a very good point, as general exception handling
> for
> > > the
> > > > > > different failure scenarios is a tricky problem. I think the
> > > exception
> > > > > > classifiers and retry strategies could avoid a lot of manual
> > > intervention
> > > > > > from the user. We will definitely need to add something like
> this.
> > > Once
> > > > > we
> > > > > > have the repo created with the initial operator code we should
> open
> > > some
> > > > > > tickets for this and put it on the short term roadmap!
> > > > > >
> > > > > > Cheers,
> > > > > > Gyula
> > > > > >
> > > > > > On Wed, Feb 2, 2022 at 4:50 PM Danny Cranmer <
> > > dannycran...@apache.org>
> > > > > > wrote:
> > > > > >
> > > > > > > Hey team,
> > > > > > >
> > > > > > > Great work on the FLIP, I am looking forward to this one. I
> agree
> > > that
> > > > > we
> > > > > > > can move forward to the voting stage.
> > > > > > >
> > > > > > > I have general feedback around how we will handle job
> submission
> > > > > failure
> > > > > > > and retry. As discussed in the Rejected Alternatives section,
> we
> > > can
> > > > > use
> > > > > > > Java to handle job submission failures from the Flink client.
> It
> > > would
> > > > > be
> > > > > > > useful to have the ability to configure exception classifiers
> and
> > > retry
> > > > > > > strategy as part of operator configuration.
> > > > > > >
> > > > > > > Given this will be in a separate Github repository I am curious
> > how
> > > > > ther
> > > > > > > versioning strategy will work in relation to the Flink version?
> > Do
> > > we
> > > > > > have
> > > > > > > any other components with a similar setup I can look at? Will
> the
> > > > > > operator
> > > > > > > version track Flink or will it use its own versioning strategy
> > > with a
> > > > > > Flink
> > > > > > > version support matrix, or similar?
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On Tue, Feb 1, 2022 at 

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-14 Thread K Fred
Hi All,

Has the project of flink-kubernetes-operator been created in github?

Peng Yuan

On Wed, Feb 9, 2022 at 1:23 AM Gyula Fóra  wrote:

> I agree with flink-kubernetes-operator as the repo name :)
> Don't have any better idea
>
> Gyula
>
> On Sat, Feb 5, 2022 at 2:41 AM Thomas Weise  wrote:
>
> > Hi,
> >
> > Thanks for the continued feedback and discussion. Looks like we are
> > ready to start a VOTE, I will initiate it shortly.
> >
> > In parallel it would be good to find the repository name.
> >
> > My suggestion would be: flink-kubernetes-operator
> >
> > I thought "flink-operator" could be a bit misleading since the term
> > operator already has a meaning in Flink.
> >
> > I also considered "flink-k8s-operator" but that would be almost
> > identical to existing operator implementations and could lead to
> > confusion in the future.
> >
> > Thoughts?
> >
> > Thanks,
> > Thomas
> >
> >
> >
> > On Fri, Feb 4, 2022 at 5:15 AM Gyula Fóra  wrote:
> > >
> > > Hi Danny,
> > >
> > > So far we have been focusing our dev efforts on the initial native
> > > implementation with the team.
> > > If the discussion and vote goes well for this FLIP we are looking
> forward
> > > to contributing the initial version sometime next week (fingers
> crossed).
> > >
> > > At that point I think we can already start the dev work to support the
> > > standalone mode as well, especially if you can dedicate some effort to
> > > pushing that side.
> > > Working together on this sounds like a great idea and we should start
> as
> > > soon as possible! :)
> > >
> > > Cheers,
> > > Gyula
> > >
> > > On Fri, Feb 4, 2022 at 2:07 PM Danny Cranmer 
> > > wrote:
> > >
> > > > I have been discussing this one with my team. We are interested in
> the
> > > > Standalone mode, and are willing to contribute towards the
> > implementation.
> > > > Potentially we can work together to support both modes in parallel?
> > > >
> > > > Thanks,
> > > >
> > > > On Wed, Feb 2, 2022 at 4:02 PM Gyula Fóra 
> > wrote:
> > > >
> > > > > Hi Danny!
> > > > >
> > > > > Thanks for the feedback :)
> > > > >
> > > > > Versioning:
> > > > > Versioning will be independent from Flink and the operator will
> > depend
> > > > on a
> > > > > fixed flink version (in every given operator version).
> > > > > This should be the exact same setup as with Stateful Functions (
> > > > > https://github.com/apache/flink-statefun). So independent release
> > cycle
> > > > > but
> > > > > still within the Flink umbrella.
> > > > >
> > > > > Deployment error handling:
> > > > > I think that's a very good point, as general exception handling for
> > the
> > > > > different failure scenarios is a tricky problem. I think the
> > exception
> > > > > classifiers and retry strategies could avoid a lot of manual
> > intervention
> > > > > from the user. We will definitely need to add something like this.
> > Once
> > > > we
> > > > > have the repo created with the initial operator code we should open
> > some
> > > > > tickets for this and put it on the short term roadmap!
> > > > >
> > > > > Cheers,
> > > > > Gyula
> > > > >
> > > > > On Wed, Feb 2, 2022 at 4:50 PM Danny Cranmer <
> > dannycran...@apache.org>
> > > > > wrote:
> > > > >
> > > > > > Hey team,
> > > > > >
> > > > > > Great work on the FLIP, I am looking forward to this one. I agree
> > that
> > > > we
> > > > > > can move forward to the voting stage.
> > > > > >
> > > > > > I have general feedback around how we will handle job submission
> > > > failure
> > > > > > and retry. As discussed in the Rejected Alternatives section, we
> > can
> > > > use
> > > > > > Java to handle job submission failures from the Flink client. It
> > would
> > > > be
> > > > > > useful to have the ability to configure exception classifiers and
> > retry
> > > > > > strategy as part of operator configuration.
> > > > > >
> > > > > > Given this will be in a separate Github repository I am curious
> how
> > > > ther
> > > > > > versioning strategy will work in relation to the Flink version?
> Do
> > we
> > > > > have
> > > > > > any other components with a similar setup I can look at? Will the
> > > > > operator
> > > > > > version track Flink or will it use its own versioning strategy
> > with a
> > > > > Flink
> > > > > > version support matrix, or similar?
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Tue, Feb 1, 2022 at 2:33 PM Márton Balassi <
> > > > balassi.mar...@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Hi team,
> > > > > > >
> > > > > > > Thank you for the great feedback, Thomas has updated the FLIP
> > page
> > > > > > > accordingly. If you are comfortable with the currently existing
> > > > design
> > > > > > and
> > > > > > > depth in the FLIP [1] I suggest moving forward to the voting
> > stage -
> > > > > once
> > > > > > > that reaches a positive conclusion it lets us create the
> separate
> > > > code
> > > > > > > repository under the flink project for the operator.
> > > > > > 

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-08 Thread Gyula Fóra
I agree with flink-kubernetes-operator as the repo name :)
Don't have any better idea

Gyula

On Sat, Feb 5, 2022 at 2:41 AM Thomas Weise  wrote:

> Hi,
>
> Thanks for the continued feedback and discussion. Looks like we are
> ready to start a VOTE, I will initiate it shortly.
>
> In parallel it would be good to find the repository name.
>
> My suggestion would be: flink-kubernetes-operator
>
> I thought "flink-operator" could be a bit misleading since the term
> operator already has a meaning in Flink.
>
> I also considered "flink-k8s-operator" but that would be almost
> identical to existing operator implementations and could lead to
> confusion in the future.
>
> Thoughts?
>
> Thanks,
> Thomas
>
>
>
> On Fri, Feb 4, 2022 at 5:15 AM Gyula Fóra  wrote:
> >
> > Hi Danny,
> >
> > So far we have been focusing our dev efforts on the initial native
> > implementation with the team.
> > If the discussion and vote goes well for this FLIP we are looking forward
> > to contributing the initial version sometime next week (fingers crossed).
> >
> > At that point I think we can already start the dev work to support the
> > standalone mode as well, especially if you can dedicate some effort to
> > pushing that side.
> > Working together on this sounds like a great idea and we should start as
> > soon as possible! :)
> >
> > Cheers,
> > Gyula
> >
> > On Fri, Feb 4, 2022 at 2:07 PM Danny Cranmer 
> > wrote:
> >
> > > I have been discussing this one with my team. We are interested in the
> > > Standalone mode, and are willing to contribute towards the
> implementation.
> > > Potentially we can work together to support both modes in parallel?
> > >
> > > Thanks,
> > >
> > > On Wed, Feb 2, 2022 at 4:02 PM Gyula Fóra 
> wrote:
> > >
> > > > Hi Danny!
> > > >
> > > > Thanks for the feedback :)
> > > >
> > > > Versioning:
> > > > Versioning will be independent from Flink and the operator will
> depend
> > > on a
> > > > fixed flink version (in every given operator version).
> > > > This should be the exact same setup as with Stateful Functions (
> > > > https://github.com/apache/flink-statefun). So independent release
> cycle
> > > > but
> > > > still within the Flink umbrella.
> > > >
> > > > Deployment error handling:
> > > > I think that's a very good point, as general exception handling for
> the
> > > > different failure scenarios is a tricky problem. I think the
> exception
> > > > classifiers and retry strategies could avoid a lot of manual
> intervention
> > > > from the user. We will definitely need to add something like this.
> Once
> > > we
> > > > have the repo created with the initial operator code we should open
> some
> > > > tickets for this and put it on the short term roadmap!
> > > >
> > > > Cheers,
> > > > Gyula
> > > >
> > > > On Wed, Feb 2, 2022 at 4:50 PM Danny Cranmer <
> dannycran...@apache.org>
> > > > wrote:
> > > >
> > > > > Hey team,
> > > > >
> > > > > Great work on the FLIP, I am looking forward to this one. I agree
> that
> > > we
> > > > > can move forward to the voting stage.
> > > > >
> > > > > I have general feedback around how we will handle job submission
> > > failure
> > > > > and retry. As discussed in the Rejected Alternatives section, we
> can
> > > use
> > > > > Java to handle job submission failures from the Flink client. It
> would
> > > be
> > > > > useful to have the ability to configure exception classifiers and
> retry
> > > > > strategy as part of operator configuration.
> > > > >
> > > > > Given this will be in a separate Github repository I am curious how
> > > ther
> > > > > versioning strategy will work in relation to the Flink version? Do
> we
> > > > have
> > > > > any other components with a similar setup I can look at? Will the
> > > > operator
> > > > > version track Flink or will it use its own versioning strategy
> with a
> > > > Flink
> > > > > version support matrix, or similar?
> > > > >
> > > > > Thanks,
> > > > >
> > > > >
> > > > >
> > > > > On Tue, Feb 1, 2022 at 2:33 PM Márton Balassi <
> > > balassi.mar...@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hi team,
> > > > > >
> > > > > > Thank you for the great feedback, Thomas has updated the FLIP
> page
> > > > > > accordingly. If you are comfortable with the currently existing
> > > design
> > > > > and
> > > > > > depth in the FLIP [1] I suggest moving forward to the voting
> stage -
> > > > once
> > > > > > that reaches a positive conclusion it lets us create the separate
> > > code
> > > > > > repository under the flink project for the operator.
> > > > > >
> > > > > > I encourage everyone to keep improving the details in the
> meantime,
> > > > > however
> > > > > > I believe given the existing design and the general sentiment on
> this
> > > > > > thread that the most efficient path from here is starting the
> > > > > > implementation so that we can collectively iterate over it.
> > > > > >
> > > > > > [1]
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> 

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-07 Thread Yangze Guo
Thanks @Thomas and @Gyula.
+1 to only introduce necessary and reasonable shorthand proxy parameters.

Best,
Yangze Guo

On Tue, Feb 8, 2022 at 12:47 PM Thomas Weise  wrote:
>
> @Yangze thanks for bringing up the configuration priority. This is
> quite important indeed and should be mentioned in the FLIP.
>
> I agree with the sentiment that whenever possible we should use the
> native configuration directly (either Flink native settings or k8s pod
> template), rather than introducing proxy parameters in the CRD. That
> certainly applies to taskManager.taskSlots which can be specified
> directly under flinkConfiguration.
>
> Thanks @Alexis for the pointers!
>
> Regarding memory: I'm leaning towards starting from total memory at
> the k8s resource level and let Flink derive components by default. For
> many users that would be a more intuitive approach than specifying the
> components. So container memory -> taskmanager.memory.process.size ->
>  [1]
>
> With that approach we could also extract the resource spec from the
> pod template. Although setting memory is something necessary pretty
> much always and defining the pod template not necessarily. Having the
> shorthand proxy parameter may be a good compromise.
>
> Cheers,
> Thomas
>
> [1] 
> https://nightlies.apache.org/flink/flink-docs-master/docs/deployment/memory/mem_setup/
>
> On Mon, Feb 7, 2022 at 4:33 AM Alexis Sarda-Espinosa
>  wrote:
> >
> > Danny Cranmer mentioned they are interested in standalone mode, and I am 
> > too, so I just wanted to say that if that development starts in parallel, I 
> > might be able to contribute a little.
> >
> > Regarding the CRD, I agree it would be nice to avoid as many "duplications" 
> > as possible if pod templates are to be used. In my PoC I even tried to make 
> > use of existing configuration options like kubernetes.container.image & 
> > pipeline.jars [1]. For CPU/Memory resources, the discussion in [2] might be 
> > relevant.
> >
> > [1] 
> > https://github.com/MicroFocus/opsb-flink-k8s-operator/blob/main/kubernetes/sample_batch_job.yaml
> > [2] https://issues.apache.org/jira/browse/FLINK-24150
> >
> > Regards,
> > Alexis.
> >
> > -Original Message-
> > From: K Fred 
> > Sent: Montag, 7. Februar 2022 09:36
> > To: dev@flink.apache.org
> > Subject: Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator
> >
> > Hi Gyula!
> >
> > You are right. I think some common flink config options can be put in the 
> > CR, other expert settings continue to be overwritten by flink, and then the 
> > user can choose to customize the configuration.
> >
> > Best Wishes,
> > Peng Yuan
> >
> > On Mon, Feb 7, 2022 at 4:16 PM Gyula Fóra  wrote:
> >
> > > Hi Yangze!
> > >
> > > This is not set in stone at the moment but the way I think it should
> > > work is that first class config options in the CR should always take
> > > precedence over the Flink config.
> > >
> > > In general we should not introduce too many arbitrary config options
> > > that duplicate the flink configs without good reasons but the ones we
> > > introduce should overwrite flink configs.
> > >
> > > We should discuss and decide together what config options to keep in
> > > the flink conf and what to bring on the CR level. Resource related
> > > ones are difficult because on one hand they are integral to every
> > > application, on the other hand there are many expert settings that we
> > > should probably leave in the conf.
> > >
> > > Cheers,
> > > Gyula
> > >
> > >
> > >
> > > On Mon, Feb 7, 2022 at 8:28 AM Yangze Guo  wrote:
> > >
> > > > Thanks everyone for the great effort. The FLIP looks really good.
> > > >
> > > > I just want to make sure the configuration priority in the CR example.
> > > > It seems the requests resources or "taskManager. taskSlots" will be
> > > > transferred to Flink internal config, e.g.
> > > > "taskmanager.memory.process.size" and
> > > > "taskmanager.numberOfTaskSlots", and override the one in
> > > > "flinkConfiguration". Am I understanding this correctly?
> > > >
> > > > Best,
> > > > Yangze Guo
> > > >
> > > > On Mon, Feb 7, 2022 at 10:22 AM Xintong Song 
> > > > wrote:
> > > > >
> > > > > Sorry for the late reply. We were out due to the public holidays
&

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-07 Thread Thomas Weise
@Yangze thanks for bringing up the configuration priority. This is
quite important indeed and should be mentioned in the FLIP.

I agree with the sentiment that whenever possible we should use the
native configuration directly (either Flink native settings or k8s pod
template), rather than introducing proxy parameters in the CRD. That
certainly applies to taskManager.taskSlots which can be specified
directly under flinkConfiguration.

Thanks @Alexis for the pointers!

Regarding memory: I'm leaning towards starting from total memory at
the k8s resource level and let Flink derive components by default. For
many users that would be a more intuitive approach than specifying the
components. So container memory -> taskmanager.memory.process.size ->
 [1]

With that approach we could also extract the resource spec from the
pod template. Although setting memory is something necessary pretty
much always and defining the pod template not necessarily. Having the
shorthand proxy parameter may be a good compromise.

Cheers,
Thomas

[1] 
https://nightlies.apache.org/flink/flink-docs-master/docs/deployment/memory/mem_setup/

On Mon, Feb 7, 2022 at 4:33 AM Alexis Sarda-Espinosa
 wrote:
>
> Danny Cranmer mentioned they are interested in standalone mode, and I am too, 
> so I just wanted to say that if that development starts in parallel, I might 
> be able to contribute a little.
>
> Regarding the CRD, I agree it would be nice to avoid as many "duplications" 
> as possible if pod templates are to be used. In my PoC I even tried to make 
> use of existing configuration options like kubernetes.container.image & 
> pipeline.jars [1]. For CPU/Memory resources, the discussion in [2] might be 
> relevant.
>
> [1] 
> https://github.com/MicroFocus/opsb-flink-k8s-operator/blob/main/kubernetes/sample_batch_job.yaml
> [2] https://issues.apache.org/jira/browse/FLINK-24150
>
> Regards,
> Alexis.
>
> -Original Message-
> From: K Fred 
> Sent: Montag, 7. Februar 2022 09:36
> To: dev@flink.apache.org
> Subject: Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator
>
> Hi Gyula!
>
> You are right. I think some common flink config options can be put in the CR, 
> other expert settings continue to be overwritten by flink, and then the user 
> can choose to customize the configuration.
>
> Best Wishes,
> Peng Yuan
>
> On Mon, Feb 7, 2022 at 4:16 PM Gyula Fóra  wrote:
>
> > Hi Yangze!
> >
> > This is not set in stone at the moment but the way I think it should
> > work is that first class config options in the CR should always take
> > precedence over the Flink config.
> >
> > In general we should not introduce too many arbitrary config options
> > that duplicate the flink configs without good reasons but the ones we
> > introduce should overwrite flink configs.
> >
> > We should discuss and decide together what config options to keep in
> > the flink conf and what to bring on the CR level. Resource related
> > ones are difficult because on one hand they are integral to every
> > application, on the other hand there are many expert settings that we
> > should probably leave in the conf.
> >
> > Cheers,
> > Gyula
> >
> >
> >
> > On Mon, Feb 7, 2022 at 8:28 AM Yangze Guo  wrote:
> >
> > > Thanks everyone for the great effort. The FLIP looks really good.
> > >
> > > I just want to make sure the configuration priority in the CR example.
> > > It seems the requests resources or "taskManager. taskSlots" will be
> > > transferred to Flink internal config, e.g.
> > > "taskmanager.memory.process.size" and
> > > "taskmanager.numberOfTaskSlots", and override the one in
> > > "flinkConfiguration". Am I understanding this correctly?
> > >
> > > Best,
> > > Yangze Guo
> > >
> > > On Mon, Feb 7, 2022 at 10:22 AM Xintong Song 
> > > wrote:
> > > >
> > > > Sorry for the late reply. We were out due to the public holidays
> > > > in
> > > China.
> > > >
> > > > @Thomas,
> > > >
> > > > The intention is to support application management through
> > > > operator and
> > > CR,
> > > > > which means there won't be any 2 step submission process, which
> > > > > as
> > you
> > > > > allude to would defeat the purpose of this project. The CR
> > > > > example
> > > shows
> > > > > the application part. Please note that the bare cluster support
> > > > > is an
> > > > > *additional* feature for scenari

RE: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-07 Thread Alexis Sarda-Espinosa
Danny Cranmer mentioned they are interested in standalone mode, and I am too, 
so I just wanted to say that if that development starts in parallel, I might be 
able to contribute a little.

Regarding the CRD, I agree it would be nice to avoid as many "duplications" as 
possible if pod templates are to be used. In my PoC I even tried to make use of 
existing configuration options like kubernetes.container.image & pipeline.jars 
[1]. For CPU/Memory resources, the discussion in [2] might be relevant.

[1] 
https://github.com/MicroFocus/opsb-flink-k8s-operator/blob/main/kubernetes/sample_batch_job.yaml
[2] https://issues.apache.org/jira/browse/FLINK-24150

Regards,
Alexis.

-Original Message-
From: K Fred  
Sent: Montag, 7. Februar 2022 09:36
To: dev@flink.apache.org
Subject: Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

Hi Gyula!

You are right. I think some common flink config options can be put in the CR, 
other expert settings continue to be overwritten by flink, and then the user 
can choose to customize the configuration.

Best Wishes,
Peng Yuan

On Mon, Feb 7, 2022 at 4:16 PM Gyula Fóra  wrote:

> Hi Yangze!
>
> This is not set in stone at the moment but the way I think it should 
> work is that first class config options in the CR should always take 
> precedence over the Flink config.
>
> In general we should not introduce too many arbitrary config options 
> that duplicate the flink configs without good reasons but the ones we 
> introduce should overwrite flink configs.
>
> We should discuss and decide together what config options to keep in 
> the flink conf and what to bring on the CR level. Resource related 
> ones are difficult because on one hand they are integral to every 
> application, on the other hand there are many expert settings that we 
> should probably leave in the conf.
>
> Cheers,
> Gyula
>
>
>
> On Mon, Feb 7, 2022 at 8:28 AM Yangze Guo  wrote:
>
> > Thanks everyone for the great effort. The FLIP looks really good.
> >
> > I just want to make sure the configuration priority in the CR example.
> > It seems the requests resources or "taskManager. taskSlots" will be 
> > transferred to Flink internal config, e.g.
> > "taskmanager.memory.process.size" and 
> > "taskmanager.numberOfTaskSlots", and override the one in 
> > "flinkConfiguration". Am I understanding this correctly?
> >
> > Best,
> > Yangze Guo
> >
> > On Mon, Feb 7, 2022 at 10:22 AM Xintong Song 
> > wrote:
> > >
> > > Sorry for the late reply. We were out due to the public holidays 
> > > in
> > China.
> > >
> > > @Thomas,
> > >
> > > The intention is to support application management through 
> > > operator and
> > CR,
> > > > which means there won't be any 2 step submission process, which 
> > > > as
> you
> > > > allude to would defeat the purpose of this project. The CR 
> > > > example
> > shows
> > > > the application part. Please note that the bare cluster support 
> > > > is an
> > > > *additional* feature for scenarios that require external job
> > management. Is
> > > > there anything on the FLIP page that creates a different impression?
> > > >
> > >
> > > Sounds good to me. I don't remember what created the impression of 
> > > 2
> step
> > > submission back then. I revisited the latest version of this FLIP 
> > > and
> it
> > > looks good to me.
> > >
> > > @Gyula,
> > >
> > > Versioning:
> > > > Versioning will be independent from Flink and the operator will
> depend
> > on a
> > > > fixed flink version (in every given operator version).
> > > > This should be the exact same setup as with Stateful Functions ( 
> > > > https://github.com/apache/flink-statefun). So independent 
> > > > release
> > cycle
> > > > but
> > > > still within the Flink umbrella.
> > > >
> > >
> > > Does this mean if someone wants to upgrade Flink to a version that 
> > > is released after the operator version that is being used, he/she 
> > > would
> need
> > > to upgrade the operator version first?
> > > I'm not questioning this, just trying to make sure I'm 
> > > understanding
> this
> > > correctly.
> > >
> > > Thank you~
> > >
> > > Xintong Song
> > >
> > >
> > >
> > > On Mon, Feb 7, 2022 at 3:14 AM Gyula Fóra 
> wrote:
> > >
> >

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-07 Thread K Fred
Hi Gyula!

You are right. I think some common flink config options can be put in the
CR, other expert settings continue to be overwritten by flink, and then the
user can choose to customize the configuration.

Best Wishes,
Peng Yuan

On Mon, Feb 7, 2022 at 4:16 PM Gyula Fóra  wrote:

> Hi Yangze!
>
> This is not set in stone at the moment but the way I think it should work
> is that first class config options in the CR should always take precedence
> over the Flink config.
>
> In general we should not introduce too many arbitrary config options that
> duplicate the flink configs without good reasons but the ones we introduce
> should overwrite flink configs.
>
> We should discuss and decide together what config options to keep in the
> flink conf and what to bring on the CR level. Resource related ones are
> difficult because on one hand they are integral to every application, on
> the other hand there are many expert settings that we should probably leave
> in the conf.
>
> Cheers,
> Gyula
>
>
>
> On Mon, Feb 7, 2022 at 8:28 AM Yangze Guo  wrote:
>
> > Thanks everyone for the great effort. The FLIP looks really good.
> >
> > I just want to make sure the configuration priority in the CR example.
> > It seems the requests resources or "taskManager. taskSlots" will be
> > transferred to Flink internal config, e.g.
> > "taskmanager.memory.process.size" and "taskmanager.numberOfTaskSlots",
> > and override the one in "flinkConfiguration". Am I understanding this
> > correctly?
> >
> > Best,
> > Yangze Guo
> >
> > On Mon, Feb 7, 2022 at 10:22 AM Xintong Song 
> > wrote:
> > >
> > > Sorry for the late reply. We were out due to the public holidays in
> > China.
> > >
> > > @Thomas,
> > >
> > > The intention is to support application management through operator and
> > CR,
> > > > which means there won't be any 2 step submission process, which as
> you
> > > > allude to would defeat the purpose of this project. The CR example
> > shows
> > > > the application part. Please note that the bare cluster support is an
> > > > *additional* feature for scenarios that require external job
> > management. Is
> > > > there anything on the FLIP page that creates a different impression?
> > > >
> > >
> > > Sounds good to me. I don't remember what created the impression of 2
> step
> > > submission back then. I revisited the latest version of this FLIP and
> it
> > > looks good to me.
> > >
> > > @Gyula,
> > >
> > > Versioning:
> > > > Versioning will be independent from Flink and the operator will
> depend
> > on a
> > > > fixed flink version (in every given operator version).
> > > > This should be the exact same setup as with Stateful Functions (
> > > > https://github.com/apache/flink-statefun). So independent release
> > cycle
> > > > but
> > > > still within the Flink umbrella.
> > > >
> > >
> > > Does this mean if someone wants to upgrade Flink to a version that is
> > > released after the operator version that is being used, he/she would
> need
> > > to upgrade the operator version first?
> > > I'm not questioning this, just trying to make sure I'm understanding
> this
> > > correctly.
> > >
> > > Thank you~
> > >
> > > Xintong Song
> > >
> > >
> > >
> > > On Mon, Feb 7, 2022 at 3:14 AM Gyula Fóra 
> wrote:
> > >
> > > > Thank you Alexis,
> > > >
> > > > Will definitely check this out. You are right, Kotlin makes it
> > difficult to
> > > > adopt pieces of this code directly but I think it will be good to get
> > > > inspiration for the architecture and look at how particular problems
> > have
> > > > been solved. It will be a great help for us I am sure.
> > > >
> > > > Cheers,
> > > > Gyula
> > > >
> > > > On Sat, Feb 5, 2022 at 12:28 PM Alexis Sarda-Espinosa <
> > > > alexis.sarda-espin...@microfocus.com> wrote:
> > > >
> > > > > Hi everyone,
> > > > >
> > > > > just wanted to mention that my employer agreed to open source the
> > PoC I
> > > > > developed: https://github.com/MicroFocus/opsb-flink-k8s-operator
> > > > >
> > > > > I understand the concern for maintainability, so Gradle & Kotlin

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-07 Thread Gyula Fóra
Hi Yangze!

This is not set in stone at the moment but the way I think it should work
is that first class config options in the CR should always take precedence
over the Flink config.

In general we should not introduce too many arbitrary config options that
duplicate the flink configs without good reasons but the ones we introduce
should overwrite flink configs.

We should discuss and decide together what config options to keep in the
flink conf and what to bring on the CR level. Resource related ones are
difficult because on one hand they are integral to every application, on
the other hand there are many expert settings that we should probably leave
in the conf.

Cheers,
Gyula



On Mon, Feb 7, 2022 at 8:28 AM Yangze Guo  wrote:

> Thanks everyone for the great effort. The FLIP looks really good.
>
> I just want to make sure the configuration priority in the CR example.
> It seems the requests resources or "taskManager. taskSlots" will be
> transferred to Flink internal config, e.g.
> "taskmanager.memory.process.size" and "taskmanager.numberOfTaskSlots",
> and override the one in "flinkConfiguration". Am I understanding this
> correctly?
>
> Best,
> Yangze Guo
>
> On Mon, Feb 7, 2022 at 10:22 AM Xintong Song 
> wrote:
> >
> > Sorry for the late reply. We were out due to the public holidays in
> China.
> >
> > @Thomas,
> >
> > The intention is to support application management through operator and
> CR,
> > > which means there won't be any 2 step submission process, which as you
> > > allude to would defeat the purpose of this project. The CR example
> shows
> > > the application part. Please note that the bare cluster support is an
> > > *additional* feature for scenarios that require external job
> management. Is
> > > there anything on the FLIP page that creates a different impression?
> > >
> >
> > Sounds good to me. I don't remember what created the impression of 2 step
> > submission back then. I revisited the latest version of this FLIP and it
> > looks good to me.
> >
> > @Gyula,
> >
> > Versioning:
> > > Versioning will be independent from Flink and the operator will depend
> on a
> > > fixed flink version (in every given operator version).
> > > This should be the exact same setup as with Stateful Functions (
> > > https://github.com/apache/flink-statefun). So independent release
> cycle
> > > but
> > > still within the Flink umbrella.
> > >
> >
> > Does this mean if someone wants to upgrade Flink to a version that is
> > released after the operator version that is being used, he/she would need
> > to upgrade the operator version first?
> > I'm not questioning this, just trying to make sure I'm understanding this
> > correctly.
> >
> > Thank you~
> >
> > Xintong Song
> >
> >
> >
> > On Mon, Feb 7, 2022 at 3:14 AM Gyula Fóra  wrote:
> >
> > > Thank you Alexis,
> > >
> > > Will definitely check this out. You are right, Kotlin makes it
> difficult to
> > > adopt pieces of this code directly but I think it will be good to get
> > > inspiration for the architecture and look at how particular problems
> have
> > > been solved. It will be a great help for us I am sure.
> > >
> > > Cheers,
> > > Gyula
> > >
> > > On Sat, Feb 5, 2022 at 12:28 PM Alexis Sarda-Espinosa <
> > > alexis.sarda-espin...@microfocus.com> wrote:
> > >
> > > > Hi everyone,
> > > >
> > > > just wanted to mention that my employer agreed to open source the
> PoC I
> > > > developed: https://github.com/MicroFocus/opsb-flink-k8s-operator
> > > >
> > > > I understand the concern for maintainability, so Gradle & Kotlin
> might
> > > not
> > > > be appealing to you, but at least it gives you another reference. The
> > > Helm
> > > > resources in particular might be useful.
> > > >
> > > > There are bits and pieces there referring to Flink sessions, but
> those
> > > are
> > > > just placeholders, the functioning parts use application mode with
> native
> > > > integration.
> > > >
> > > > Regards,
> > > > Alexis.
> > > >
> > > > 
> > > > From: Thomas Weise 
> > > > Sent: Saturday, February 5, 2022 2:41 AM
> > > > To: dev 
> > > > Subject: Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator
> >

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-06 Thread Yangze Guo
Thanks everyone for the great effort. The FLIP looks really good.

I just want to make sure the configuration priority in the CR example.
It seems the requests resources or "taskManager. taskSlots" will be
transferred to Flink internal config, e.g.
"taskmanager.memory.process.size" and "taskmanager.numberOfTaskSlots",
and override the one in "flinkConfiguration". Am I understanding this
correctly?

Best,
Yangze Guo

On Mon, Feb 7, 2022 at 10:22 AM Xintong Song  wrote:
>
> Sorry for the late reply. We were out due to the public holidays in China.
>
> @Thomas,
>
> The intention is to support application management through operator and CR,
> > which means there won't be any 2 step submission process, which as you
> > allude to would defeat the purpose of this project. The CR example shows
> > the application part. Please note that the bare cluster support is an
> > *additional* feature for scenarios that require external job management. Is
> > there anything on the FLIP page that creates a different impression?
> >
>
> Sounds good to me. I don't remember what created the impression of 2 step
> submission back then. I revisited the latest version of this FLIP and it
> looks good to me.
>
> @Gyula,
>
> Versioning:
> > Versioning will be independent from Flink and the operator will depend on a
> > fixed flink version (in every given operator version).
> > This should be the exact same setup as with Stateful Functions (
> > https://github.com/apache/flink-statefun). So independent release cycle
> > but
> > still within the Flink umbrella.
> >
>
> Does this mean if someone wants to upgrade Flink to a version that is
> released after the operator version that is being used, he/she would need
> to upgrade the operator version first?
> I'm not questioning this, just trying to make sure I'm understanding this
> correctly.
>
> Thank you~
>
> Xintong Song
>
>
>
> On Mon, Feb 7, 2022 at 3:14 AM Gyula Fóra  wrote:
>
> > Thank you Alexis,
> >
> > Will definitely check this out. You are right, Kotlin makes it difficult to
> > adopt pieces of this code directly but I think it will be good to get
> > inspiration for the architecture and look at how particular problems have
> > been solved. It will be a great help for us I am sure.
> >
> > Cheers,
> > Gyula
> >
> > On Sat, Feb 5, 2022 at 12:28 PM Alexis Sarda-Espinosa <
> > alexis.sarda-espin...@microfocus.com> wrote:
> >
> > > Hi everyone,
> > >
> > > just wanted to mention that my employer agreed to open source the PoC I
> > > developed: https://github.com/MicroFocus/opsb-flink-k8s-operator
> > >
> > > I understand the concern for maintainability, so Gradle & Kotlin might
> > not
> > > be appealing to you, but at least it gives you another reference. The
> > Helm
> > > resources in particular might be useful.
> > >
> > > There are bits and pieces there referring to Flink sessions, but those
> > are
> > > just placeholders, the functioning parts use application mode with native
> > > integration.
> > >
> > > Regards,
> > > Alexis.
> > >
> > > 
> > > From: Thomas Weise 
> > > Sent: Saturday, February 5, 2022 2:41 AM
> > > To: dev 
> > > Subject: Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator
> > >
> > > Hi,
> > >
> > > Thanks for the continued feedback and discussion. Looks like we are
> > > ready to start a VOTE, I will initiate it shortly.
> > >
> > > In parallel it would be good to find the repository name.
> > >
> > > My suggestion would be: flink-kubernetes-operator
> > >
> > > I thought "flink-operator" could be a bit misleading since the term
> > > operator already has a meaning in Flink.
> > >
> > > I also considered "flink-k8s-operator" but that would be almost
> > > identical to existing operator implementations and could lead to
> > > confusion in the future.
> > >
> > > Thoughts?
> > >
> > > Thanks,
> > > Thomas
> > >
> > >
> > >
> > > On Fri, Feb 4, 2022 at 5:15 AM Gyula Fóra  wrote:
> > > >
> > > > Hi Danny,
> > > >
> > > > So far we have been focusing our dev efforts on the initial native
> > > > implementation with the team.
> > > > If the discussion and vote goes well for this FLIP we are

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-06 Thread Xintong Song
Sorry for the late reply. We were out due to the public holidays in China.

@Thomas,

The intention is to support application management through operator and CR,
> which means there won't be any 2 step submission process, which as you
> allude to would defeat the purpose of this project. The CR example shows
> the application part. Please note that the bare cluster support is an
> *additional* feature for scenarios that require external job management. Is
> there anything on the FLIP page that creates a different impression?
>

Sounds good to me. I don't remember what created the impression of 2 step
submission back then. I revisited the latest version of this FLIP and it
looks good to me.

@Gyula,

Versioning:
> Versioning will be independent from Flink and the operator will depend on a
> fixed flink version (in every given operator version).
> This should be the exact same setup as with Stateful Functions (
> https://github.com/apache/flink-statefun). So independent release cycle
> but
> still within the Flink umbrella.
>

Does this mean if someone wants to upgrade Flink to a version that is
released after the operator version that is being used, he/she would need
to upgrade the operator version first?
I'm not questioning this, just trying to make sure I'm understanding this
correctly.

Thank you~

Xintong Song



On Mon, Feb 7, 2022 at 3:14 AM Gyula Fóra  wrote:

> Thank you Alexis,
>
> Will definitely check this out. You are right, Kotlin makes it difficult to
> adopt pieces of this code directly but I think it will be good to get
> inspiration for the architecture and look at how particular problems have
> been solved. It will be a great help for us I am sure.
>
> Cheers,
> Gyula
>
> On Sat, Feb 5, 2022 at 12:28 PM Alexis Sarda-Espinosa <
> alexis.sarda-espin...@microfocus.com> wrote:
>
> > Hi everyone,
> >
> > just wanted to mention that my employer agreed to open source the PoC I
> > developed: https://github.com/MicroFocus/opsb-flink-k8s-operator
> >
> > I understand the concern for maintainability, so Gradle & Kotlin might
> not
> > be appealing to you, but at least it gives you another reference. The
> Helm
> > resources in particular might be useful.
> >
> > There are bits and pieces there referring to Flink sessions, but those
> are
> > just placeholders, the functioning parts use application mode with native
> > integration.
> >
> > Regards,
> > Alexis.
> >
> > 
> > From: Thomas Weise 
> > Sent: Saturday, February 5, 2022 2:41 AM
> > To: dev 
> > Subject: Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator
> >
> > Hi,
> >
> > Thanks for the continued feedback and discussion. Looks like we are
> > ready to start a VOTE, I will initiate it shortly.
> >
> > In parallel it would be good to find the repository name.
> >
> > My suggestion would be: flink-kubernetes-operator
> >
> > I thought "flink-operator" could be a bit misleading since the term
> > operator already has a meaning in Flink.
> >
> > I also considered "flink-k8s-operator" but that would be almost
> > identical to existing operator implementations and could lead to
> > confusion in the future.
> >
> > Thoughts?
> >
> > Thanks,
> > Thomas
> >
> >
> >
> > On Fri, Feb 4, 2022 at 5:15 AM Gyula Fóra  wrote:
> > >
> > > Hi Danny,
> > >
> > > So far we have been focusing our dev efforts on the initial native
> > > implementation with the team.
> > > If the discussion and vote goes well for this FLIP we are looking
> forward
> > > to contributing the initial version sometime next week (fingers
> crossed).
> > >
> > > At that point I think we can already start the dev work to support the
> > > standalone mode as well, especially if you can dedicate some effort to
> > > pushing that side.
> > > Working together on this sounds like a great idea and we should start
> as
> > > soon as possible! :)
> > >
> > > Cheers,
> > > Gyula
> > >
> > > On Fri, Feb 4, 2022 at 2:07 PM Danny Cranmer 
> > > wrote:
> > >
> > > > I have been discussing this one with my team. We are interested in
> the
> > > > Standalone mode, and are willing to contribute towards the
> > implementation.
> > > > Potentially we can work together to support both modes in parallel?
> > > >
> > > > Thanks,
> > > >
> > > > On Wed, Feb 2, 2022 at 4:02 PM Gyula Fóra 
> > wrote:

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-06 Thread Gyula Fóra
Thank you Alexis,

Will definitely check this out. You are right, Kotlin makes it difficult to
adopt pieces of this code directly but I think it will be good to get
inspiration for the architecture and look at how particular problems have
been solved. It will be a great help for us I am sure.

Cheers,
Gyula

On Sat, Feb 5, 2022 at 12:28 PM Alexis Sarda-Espinosa <
alexis.sarda-espin...@microfocus.com> wrote:

> Hi everyone,
>
> just wanted to mention that my employer agreed to open source the PoC I
> developed: https://github.com/MicroFocus/opsb-flink-k8s-operator
>
> I understand the concern for maintainability, so Gradle & Kotlin might not
> be appealing to you, but at least it gives you another reference. The Helm
> resources in particular might be useful.
>
> There are bits and pieces there referring to Flink sessions, but those are
> just placeholders, the functioning parts use application mode with native
> integration.
>
> Regards,
> Alexis.
>
> 
> From: Thomas Weise 
> Sent: Saturday, February 5, 2022 2:41 AM
> To: dev 
> Subject: Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator
>
> Hi,
>
> Thanks for the continued feedback and discussion. Looks like we are
> ready to start a VOTE, I will initiate it shortly.
>
> In parallel it would be good to find the repository name.
>
> My suggestion would be: flink-kubernetes-operator
>
> I thought "flink-operator" could be a bit misleading since the term
> operator already has a meaning in Flink.
>
> I also considered "flink-k8s-operator" but that would be almost
> identical to existing operator implementations and could lead to
> confusion in the future.
>
> Thoughts?
>
> Thanks,
> Thomas
>
>
>
> On Fri, Feb 4, 2022 at 5:15 AM Gyula Fóra  wrote:
> >
> > Hi Danny,
> >
> > So far we have been focusing our dev efforts on the initial native
> > implementation with the team.
> > If the discussion and vote goes well for this FLIP we are looking forward
> > to contributing the initial version sometime next week (fingers crossed).
> >
> > At that point I think we can already start the dev work to support the
> > standalone mode as well, especially if you can dedicate some effort to
> > pushing that side.
> > Working together on this sounds like a great idea and we should start as
> > soon as possible! :)
> >
> > Cheers,
> > Gyula
> >
> > On Fri, Feb 4, 2022 at 2:07 PM Danny Cranmer 
> > wrote:
> >
> > > I have been discussing this one with my team. We are interested in the
> > > Standalone mode, and are willing to contribute towards the
> implementation.
> > > Potentially we can work together to support both modes in parallel?
> > >
> > > Thanks,
> > >
> > > On Wed, Feb 2, 2022 at 4:02 PM Gyula Fóra 
> wrote:
> > >
> > > > Hi Danny!
> > > >
> > > > Thanks for the feedback :)
> > > >
> > > > Versioning:
> > > > Versioning will be independent from Flink and the operator will
> depend
> > > on a
> > > > fixed flink version (in every given operator version).
> > > > This should be the exact same setup as with Stateful Functions (
> > > > https://github.com/apache/flink-statefun). So independent release
> cycle
> > > > but
> > > > still within the Flink umbrella.
> > > >
> > > > Deployment error handling:
> > > > I think that's a very good point, as general exception handling for
> the
> > > > different failure scenarios is a tricky problem. I think the
> exception
> > > > classifiers and retry strategies could avoid a lot of manual
> intervention
> > > > from the user. We will definitely need to add something like this.
> Once
> > > we
> > > > have the repo created with the initial operator code we should open
> some
> > > > tickets for this and put it on the short term roadmap!
> > > >
> > > > Cheers,
> > > > Gyula
> > > >
> > > > On Wed, Feb 2, 2022 at 4:50 PM Danny Cranmer <
> dannycran...@apache.org>
> > > > wrote:
> > > >
> > > > > Hey team,
> > > > >
> > > > > Great work on the FLIP, I am looking forward to this one. I agree
> that
> > > we
> > > > > can move forward to the voting stage.
> > > > >
> > > > > I have general feedback around how we will handle job submission
> > > failure
> > &

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-05 Thread Alexis Sarda-Espinosa
Hi everyone,

just wanted to mention that my employer agreed to open source the PoC I 
developed: https://github.com/MicroFocus/opsb-flink-k8s-operator

I understand the concern for maintainability, so Gradle & Kotlin might not be 
appealing to you, but at least it gives you another reference. The Helm 
resources in particular might be useful.

There are bits and pieces there referring to Flink sessions, but those are just 
placeholders, the functioning parts use application mode with native 
integration.

Regards,
Alexis.


From: Thomas Weise 
Sent: Saturday, February 5, 2022 2:41 AM
To: dev 
Subject: Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

Hi,

Thanks for the continued feedback and discussion. Looks like we are
ready to start a VOTE, I will initiate it shortly.

In parallel it would be good to find the repository name.

My suggestion would be: flink-kubernetes-operator

I thought "flink-operator" could be a bit misleading since the term
operator already has a meaning in Flink.

I also considered "flink-k8s-operator" but that would be almost
identical to existing operator implementations and could lead to
confusion in the future.

Thoughts?

Thanks,
Thomas



On Fri, Feb 4, 2022 at 5:15 AM Gyula Fóra  wrote:
>
> Hi Danny,
>
> So far we have been focusing our dev efforts on the initial native
> implementation with the team.
> If the discussion and vote goes well for this FLIP we are looking forward
> to contributing the initial version sometime next week (fingers crossed).
>
> At that point I think we can already start the dev work to support the
> standalone mode as well, especially if you can dedicate some effort to
> pushing that side.
> Working together on this sounds like a great idea and we should start as
> soon as possible! :)
>
> Cheers,
> Gyula
>
> On Fri, Feb 4, 2022 at 2:07 PM Danny Cranmer 
> wrote:
>
> > I have been discussing this one with my team. We are interested in the
> > Standalone mode, and are willing to contribute towards the implementation.
> > Potentially we can work together to support both modes in parallel?
> >
> > Thanks,
> >
> > On Wed, Feb 2, 2022 at 4:02 PM Gyula Fóra  wrote:
> >
> > > Hi Danny!
> > >
> > > Thanks for the feedback :)
> > >
> > > Versioning:
> > > Versioning will be independent from Flink and the operator will depend
> > on a
> > > fixed flink version (in every given operator version).
> > > This should be the exact same setup as with Stateful Functions (
> > > https://github.com/apache/flink-statefun). So independent release cycle
> > > but
> > > still within the Flink umbrella.
> > >
> > > Deployment error handling:
> > > I think that's a very good point, as general exception handling for the
> > > different failure scenarios is a tricky problem. I think the exception
> > > classifiers and retry strategies could avoid a lot of manual intervention
> > > from the user. We will definitely need to add something like this. Once
> > we
> > > have the repo created with the initial operator code we should open some
> > > tickets for this and put it on the short term roadmap!
> > >
> > > Cheers,
> > > Gyula
> > >
> > > On Wed, Feb 2, 2022 at 4:50 PM Danny Cranmer 
> > > wrote:
> > >
> > > > Hey team,
> > > >
> > > > Great work on the FLIP, I am looking forward to this one. I agree that
> > we
> > > > can move forward to the voting stage.
> > > >
> > > > I have general feedback around how we will handle job submission
> > failure
> > > > and retry. As discussed in the Rejected Alternatives section, we can
> > use
> > > > Java to handle job submission failures from the Flink client. It would
> > be
> > > > useful to have the ability to configure exception classifiers and retry
> > > > strategy as part of operator configuration.
> > > >
> > > > Given this will be in a separate Github repository I am curious how
> > ther
> > > > versioning strategy will work in relation to the Flink version? Do we
> > > have
> > > > any other components with a similar setup I can look at? Will the
> > > operator
> > > > version track Flink or will it use its own versioning strategy with a
> > > Flink
> > > > version support matrix, or similar?
> > > >
> > > > Thanks,
> > > >
> > > >
> > > >
> > > > On Tue, Feb 1, 2022 at 2:33 PM Márton Balas

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-04 Thread Thomas Weise
Hi,

Thanks for the continued feedback and discussion. Looks like we are
ready to start a VOTE, I will initiate it shortly.

In parallel it would be good to find the repository name.

My suggestion would be: flink-kubernetes-operator

I thought "flink-operator" could be a bit misleading since the term
operator already has a meaning in Flink.

I also considered "flink-k8s-operator" but that would be almost
identical to existing operator implementations and could lead to
confusion in the future.

Thoughts?

Thanks,
Thomas



On Fri, Feb 4, 2022 at 5:15 AM Gyula Fóra  wrote:
>
> Hi Danny,
>
> So far we have been focusing our dev efforts on the initial native
> implementation with the team.
> If the discussion and vote goes well for this FLIP we are looking forward
> to contributing the initial version sometime next week (fingers crossed).
>
> At that point I think we can already start the dev work to support the
> standalone mode as well, especially if you can dedicate some effort to
> pushing that side.
> Working together on this sounds like a great idea and we should start as
> soon as possible! :)
>
> Cheers,
> Gyula
>
> On Fri, Feb 4, 2022 at 2:07 PM Danny Cranmer 
> wrote:
>
> > I have been discussing this one with my team. We are interested in the
> > Standalone mode, and are willing to contribute towards the implementation.
> > Potentially we can work together to support both modes in parallel?
> >
> > Thanks,
> >
> > On Wed, Feb 2, 2022 at 4:02 PM Gyula Fóra  wrote:
> >
> > > Hi Danny!
> > >
> > > Thanks for the feedback :)
> > >
> > > Versioning:
> > > Versioning will be independent from Flink and the operator will depend
> > on a
> > > fixed flink version (in every given operator version).
> > > This should be the exact same setup as with Stateful Functions (
> > > https://github.com/apache/flink-statefun). So independent release cycle
> > > but
> > > still within the Flink umbrella.
> > >
> > > Deployment error handling:
> > > I think that's a very good point, as general exception handling for the
> > > different failure scenarios is a tricky problem. I think the exception
> > > classifiers and retry strategies could avoid a lot of manual intervention
> > > from the user. We will definitely need to add something like this. Once
> > we
> > > have the repo created with the initial operator code we should open some
> > > tickets for this and put it on the short term roadmap!
> > >
> > > Cheers,
> > > Gyula
> > >
> > > On Wed, Feb 2, 2022 at 4:50 PM Danny Cranmer 
> > > wrote:
> > >
> > > > Hey team,
> > > >
> > > > Great work on the FLIP, I am looking forward to this one. I agree that
> > we
> > > > can move forward to the voting stage.
> > > >
> > > > I have general feedback around how we will handle job submission
> > failure
> > > > and retry. As discussed in the Rejected Alternatives section, we can
> > use
> > > > Java to handle job submission failures from the Flink client. It would
> > be
> > > > useful to have the ability to configure exception classifiers and retry
> > > > strategy as part of operator configuration.
> > > >
> > > > Given this will be in a separate Github repository I am curious how
> > ther
> > > > versioning strategy will work in relation to the Flink version? Do we
> > > have
> > > > any other components with a similar setup I can look at? Will the
> > > operator
> > > > version track Flink or will it use its own versioning strategy with a
> > > Flink
> > > > version support matrix, or similar?
> > > >
> > > > Thanks,
> > > >
> > > >
> > > >
> > > > On Tue, Feb 1, 2022 at 2:33 PM Márton Balassi <
> > balassi.mar...@gmail.com>
> > > > wrote:
> > > >
> > > > > Hi team,
> > > > >
> > > > > Thank you for the great feedback, Thomas has updated the FLIP page
> > > > > accordingly. If you are comfortable with the currently existing
> > design
> > > > and
> > > > > depth in the FLIP [1] I suggest moving forward to the voting stage -
> > > once
> > > > > that reaches a positive conclusion it lets us create the separate
> > code
> > > > > repository under the flink project for the operator.
> > > > >
> > > > > I encourage everyone to keep improving the details in the meantime,
> > > > however
> > > > > I believe given the existing design and the general sentiment on this
> > > > > thread that the most efficient path from here is starting the
> > > > > implementation so that we can collectively iterate over it.
> > > > >
> > > > > [1]
> > > > >
> > > > >
> > > >
> > >
> > https://cwiki.apache.org/confluence/display/FLINK/FLIP-212%3A+Introduce+Flink+Kubernetes+Operator
> > > > >
> > > > > On Mon, Jan 31, 2022 at 10:15 PM Thomas Weise 
> > wrote:
> > > > >
> > > > > > HI Xintong,
> > > > > >
> > > > > > Thanks for the feedback and please see responses below -->
> > > > > >
> > > > > > On Fri, Jan 28, 2022 at 12:21 AM Xintong Song <
> > tonysong...@gmail.com
> > > >
> > > > > > wrote:
> > > > > >
> > > > > > > Thanks Thomas for drafting this FLIP, and everyone for the
> > > > 

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-04 Thread Gyula Fóra
Hi Danny,

So far we have been focusing our dev efforts on the initial native
implementation with the team.
If the discussion and vote goes well for this FLIP we are looking forward
to contributing the initial version sometime next week (fingers crossed).

At that point I think we can already start the dev work to support the
standalone mode as well, especially if you can dedicate some effort to
pushing that side.
Working together on this sounds like a great idea and we should start as
soon as possible! :)

Cheers,
Gyula

On Fri, Feb 4, 2022 at 2:07 PM Danny Cranmer 
wrote:

> I have been discussing this one with my team. We are interested in the
> Standalone mode, and are willing to contribute towards the implementation.
> Potentially we can work together to support both modes in parallel?
>
> Thanks,
>
> On Wed, Feb 2, 2022 at 4:02 PM Gyula Fóra  wrote:
>
> > Hi Danny!
> >
> > Thanks for the feedback :)
> >
> > Versioning:
> > Versioning will be independent from Flink and the operator will depend
> on a
> > fixed flink version (in every given operator version).
> > This should be the exact same setup as with Stateful Functions (
> > https://github.com/apache/flink-statefun). So independent release cycle
> > but
> > still within the Flink umbrella.
> >
> > Deployment error handling:
> > I think that's a very good point, as general exception handling for the
> > different failure scenarios is a tricky problem. I think the exception
> > classifiers and retry strategies could avoid a lot of manual intervention
> > from the user. We will definitely need to add something like this. Once
> we
> > have the repo created with the initial operator code we should open some
> > tickets for this and put it on the short term roadmap!
> >
> > Cheers,
> > Gyula
> >
> > On Wed, Feb 2, 2022 at 4:50 PM Danny Cranmer 
> > wrote:
> >
> > > Hey team,
> > >
> > > Great work on the FLIP, I am looking forward to this one. I agree that
> we
> > > can move forward to the voting stage.
> > >
> > > I have general feedback around how we will handle job submission
> failure
> > > and retry. As discussed in the Rejected Alternatives section, we can
> use
> > > Java to handle job submission failures from the Flink client. It would
> be
> > > useful to have the ability to configure exception classifiers and retry
> > > strategy as part of operator configuration.
> > >
> > > Given this will be in a separate Github repository I am curious how
> ther
> > > versioning strategy will work in relation to the Flink version? Do we
> > have
> > > any other components with a similar setup I can look at? Will the
> > operator
> > > version track Flink or will it use its own versioning strategy with a
> > Flink
> > > version support matrix, or similar?
> > >
> > > Thanks,
> > >
> > >
> > >
> > > On Tue, Feb 1, 2022 at 2:33 PM Márton Balassi <
> balassi.mar...@gmail.com>
> > > wrote:
> > >
> > > > Hi team,
> > > >
> > > > Thank you for the great feedback, Thomas has updated the FLIP page
> > > > accordingly. If you are comfortable with the currently existing
> design
> > > and
> > > > depth in the FLIP [1] I suggest moving forward to the voting stage -
> > once
> > > > that reaches a positive conclusion it lets us create the separate
> code
> > > > repository under the flink project for the operator.
> > > >
> > > > I encourage everyone to keep improving the details in the meantime,
> > > however
> > > > I believe given the existing design and the general sentiment on this
> > > > thread that the most efficient path from here is starting the
> > > > implementation so that we can collectively iterate over it.
> > > >
> > > > [1]
> > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-212%3A+Introduce+Flink+Kubernetes+Operator
> > > >
> > > > On Mon, Jan 31, 2022 at 10:15 PM Thomas Weise 
> wrote:
> > > >
> > > > > HI Xintong,
> > > > >
> > > > > Thanks for the feedback and please see responses below -->
> > > > >
> > > > > On Fri, Jan 28, 2022 at 12:21 AM Xintong Song <
> tonysong...@gmail.com
> > >
> > > > > wrote:
> > > > >
> > > > > > Thanks Thomas for drafting this FLIP, and everyone for the
> > > discussion.
> > > > > >
> > > > > > I also have a few questions and comments.
> > > > > >
> > > > > > ## Job Submission
> > > > > > Deploying a Flink session cluster via kubectl & CR and then
> > > submitting
> > > > > jobs
> > > > > > to the cluster via Flink cli / REST is probably the approach that
> > > > > requires
> > > > > > the least effort. However, I'd like to point out 2 weaknesses.
> > > > > > 1. A lot of users use Flink in perjob/application modes. For
> these
> > > > users,
> > > > > > having to run the job in two steps (deploy the cluster, and
> submit
> > > the
> > > > > job)
> > > > > > is not that convenient.
> > > > > > 2. One of our motivations is being able to manage Flink
> > applications'
> > > > > > lifecycles with kubectl. Submitting jobs from cli sounds not
> > aligned
> > > > with
> > > > > > this 

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-04 Thread Danny Cranmer
I have been discussing this one with my team. We are interested in the
Standalone mode, and are willing to contribute towards the implementation.
Potentially we can work together to support both modes in parallel?

Thanks,

On Wed, Feb 2, 2022 at 4:02 PM Gyula Fóra  wrote:

> Hi Danny!
>
> Thanks for the feedback :)
>
> Versioning:
> Versioning will be independent from Flink and the operator will depend on a
> fixed flink version (in every given operator version).
> This should be the exact same setup as with Stateful Functions (
> https://github.com/apache/flink-statefun). So independent release cycle
> but
> still within the Flink umbrella.
>
> Deployment error handling:
> I think that's a very good point, as general exception handling for the
> different failure scenarios is a tricky problem. I think the exception
> classifiers and retry strategies could avoid a lot of manual intervention
> from the user. We will definitely need to add something like this. Once we
> have the repo created with the initial operator code we should open some
> tickets for this and put it on the short term roadmap!
>
> Cheers,
> Gyula
>
> On Wed, Feb 2, 2022 at 4:50 PM Danny Cranmer 
> wrote:
>
> > Hey team,
> >
> > Great work on the FLIP, I am looking forward to this one. I agree that we
> > can move forward to the voting stage.
> >
> > I have general feedback around how we will handle job submission failure
> > and retry. As discussed in the Rejected Alternatives section, we can use
> > Java to handle job submission failures from the Flink client. It would be
> > useful to have the ability to configure exception classifiers and retry
> > strategy as part of operator configuration.
> >
> > Given this will be in a separate Github repository I am curious how ther
> > versioning strategy will work in relation to the Flink version? Do we
> have
> > any other components with a similar setup I can look at? Will the
> operator
> > version track Flink or will it use its own versioning strategy with a
> Flink
> > version support matrix, or similar?
> >
> > Thanks,
> >
> >
> >
> > On Tue, Feb 1, 2022 at 2:33 PM Márton Balassi 
> > wrote:
> >
> > > Hi team,
> > >
> > > Thank you for the great feedback, Thomas has updated the FLIP page
> > > accordingly. If you are comfortable with the currently existing design
> > and
> > > depth in the FLIP [1] I suggest moving forward to the voting stage -
> once
> > > that reaches a positive conclusion it lets us create the separate code
> > > repository under the flink project for the operator.
> > >
> > > I encourage everyone to keep improving the details in the meantime,
> > however
> > > I believe given the existing design and the general sentiment on this
> > > thread that the most efficient path from here is starting the
> > > implementation so that we can collectively iterate over it.
> > >
> > > [1]
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-212%3A+Introduce+Flink+Kubernetes+Operator
> > >
> > > On Mon, Jan 31, 2022 at 10:15 PM Thomas Weise  wrote:
> > >
> > > > HI Xintong,
> > > >
> > > > Thanks for the feedback and please see responses below -->
> > > >
> > > > On Fri, Jan 28, 2022 at 12:21 AM Xintong Song  >
> > > > wrote:
> > > >
> > > > > Thanks Thomas for drafting this FLIP, and everyone for the
> > discussion.
> > > > >
> > > > > I also have a few questions and comments.
> > > > >
> > > > > ## Job Submission
> > > > > Deploying a Flink session cluster via kubectl & CR and then
> > submitting
> > > > jobs
> > > > > to the cluster via Flink cli / REST is probably the approach that
> > > > requires
> > > > > the least effort. However, I'd like to point out 2 weaknesses.
> > > > > 1. A lot of users use Flink in perjob/application modes. For these
> > > users,
> > > > > having to run the job in two steps (deploy the cluster, and submit
> > the
> > > > job)
> > > > > is not that convenient.
> > > > > 2. One of our motivations is being able to manage Flink
> applications'
> > > > > lifecycles with kubectl. Submitting jobs from cli sounds not
> aligned
> > > with
> > > > > this motivation.
> > > > > I think it's probably worth it to support submitting jobs via
> > kubectl &
> > > > CR
> > > > > in the first version, both together with deploying the cluster like
> > in
> > > > > perjob/application mode and after deploying the cluster like in
> > session
> > > > > mode.
> > > > >
> > > >
> > > > The intention is to support application management through operator
> and
> > > CR,
> > > > which means there won't be any 2 step submission process, which as
> you
> > > > allude to would defeat the purpose of this project. The CR example
> > shows
> > > > the application part. Please note that the bare cluster support is an
> > > > *additional* feature for scenarios that require external job
> > management.
> > > Is
> > > > there anything on the FLIP page that creates a different impression?
> > > >
> > > >
> > > > >
> > > > > ## Versioning
> > > > > Which Flink 

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-02 Thread Gyula Fóra
Hi Danny!

Thanks for the feedback :)

Versioning:
Versioning will be independent from Flink and the operator will depend on a
fixed flink version (in every given operator version).
This should be the exact same setup as with Stateful Functions (
https://github.com/apache/flink-statefun). So independent release cycle but
still within the Flink umbrella.

Deployment error handling:
I think that's a very good point, as general exception handling for the
different failure scenarios is a tricky problem. I think the exception
classifiers and retry strategies could avoid a lot of manual intervention
from the user. We will definitely need to add something like this. Once we
have the repo created with the initial operator code we should open some
tickets for this and put it on the short term roadmap!

Cheers,
Gyula

On Wed, Feb 2, 2022 at 4:50 PM Danny Cranmer 
wrote:

> Hey team,
>
> Great work on the FLIP, I am looking forward to this one. I agree that we
> can move forward to the voting stage.
>
> I have general feedback around how we will handle job submission failure
> and retry. As discussed in the Rejected Alternatives section, we can use
> Java to handle job submission failures from the Flink client. It would be
> useful to have the ability to configure exception classifiers and retry
> strategy as part of operator configuration.
>
> Given this will be in a separate Github repository I am curious how ther
> versioning strategy will work in relation to the Flink version? Do we have
> any other components with a similar setup I can look at? Will the operator
> version track Flink or will it use its own versioning strategy with a Flink
> version support matrix, or similar?
>
> Thanks,
>
>
>
> On Tue, Feb 1, 2022 at 2:33 PM Márton Balassi 
> wrote:
>
> > Hi team,
> >
> > Thank you for the great feedback, Thomas has updated the FLIP page
> > accordingly. If you are comfortable with the currently existing design
> and
> > depth in the FLIP [1] I suggest moving forward to the voting stage - once
> > that reaches a positive conclusion it lets us create the separate code
> > repository under the flink project for the operator.
> >
> > I encourage everyone to keep improving the details in the meantime,
> however
> > I believe given the existing design and the general sentiment on this
> > thread that the most efficient path from here is starting the
> > implementation so that we can collectively iterate over it.
> >
> > [1]
> >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-212%3A+Introduce+Flink+Kubernetes+Operator
> >
> > On Mon, Jan 31, 2022 at 10:15 PM Thomas Weise  wrote:
> >
> > > HI Xintong,
> > >
> > > Thanks for the feedback and please see responses below -->
> > >
> > > On Fri, Jan 28, 2022 at 12:21 AM Xintong Song 
> > > wrote:
> > >
> > > > Thanks Thomas for drafting this FLIP, and everyone for the
> discussion.
> > > >
> > > > I also have a few questions and comments.
> > > >
> > > > ## Job Submission
> > > > Deploying a Flink session cluster via kubectl & CR and then
> submitting
> > > jobs
> > > > to the cluster via Flink cli / REST is probably the approach that
> > > requires
> > > > the least effort. However, I'd like to point out 2 weaknesses.
> > > > 1. A lot of users use Flink in perjob/application modes. For these
> > users,
> > > > having to run the job in two steps (deploy the cluster, and submit
> the
> > > job)
> > > > is not that convenient.
> > > > 2. One of our motivations is being able to manage Flink applications'
> > > > lifecycles with kubectl. Submitting jobs from cli sounds not aligned
> > with
> > > > this motivation.
> > > > I think it's probably worth it to support submitting jobs via
> kubectl &
> > > CR
> > > > in the first version, both together with deploying the cluster like
> in
> > > > perjob/application mode and after deploying the cluster like in
> session
> > > > mode.
> > > >
> > >
> > > The intention is to support application management through operator and
> > CR,
> > > which means there won't be any 2 step submission process, which as you
> > > allude to would defeat the purpose of this project. The CR example
> shows
> > > the application part. Please note that the bare cluster support is an
> > > *additional* feature for scenarios that require external job
> management.
> > Is
> > > there anything on the FLIP page that creates a different impression?
> > >
> > >
> > > >
> > > > ## Versioning
> > > > Which Flink versions does the operator plan to support?
> > > > 1. Native K8s deployment was firstly introduced in Flink 1.10
> > > > 2. Native K8s HA was introduced in Flink 1.12
> > > > 3. The Pod template support was introduced in Flink 1.13
> > > > 4. There was some changes to the Flink docker image entrypoint script
> > in,
> > > > IIRC, Flink 1.13
> > > >
> > >
> > > Great, thanks for providing this. It is important for the compatibility
> > > going forward also. We are targeting Flink 1.14.x upwards. Before the
> > > operator is ready there will 

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-02 Thread Danny Cranmer
Hey team,

Great work on the FLIP, I am looking forward to this one. I agree that we
can move forward to the voting stage.

I have general feedback around how we will handle job submission failure
and retry. As discussed in the Rejected Alternatives section, we can use
Java to handle job submission failures from the Flink client. It would be
useful to have the ability to configure exception classifiers and retry
strategy as part of operator configuration.

Given this will be in a separate Github repository I am curious how ther
versioning strategy will work in relation to the Flink version? Do we have
any other components with a similar setup I can look at? Will the operator
version track Flink or will it use its own versioning strategy with a Flink
version support matrix, or similar?

Thanks,



On Tue, Feb 1, 2022 at 2:33 PM Márton Balassi 
wrote:

> Hi team,
>
> Thank you for the great feedback, Thomas has updated the FLIP page
> accordingly. If you are comfortable with the currently existing design and
> depth in the FLIP [1] I suggest moving forward to the voting stage - once
> that reaches a positive conclusion it lets us create the separate code
> repository under the flink project for the operator.
>
> I encourage everyone to keep improving the details in the meantime, however
> I believe given the existing design and the general sentiment on this
> thread that the most efficient path from here is starting the
> implementation so that we can collectively iterate over it.
>
> [1]
>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-212%3A+Introduce+Flink+Kubernetes+Operator
>
> On Mon, Jan 31, 2022 at 10:15 PM Thomas Weise  wrote:
>
> > HI Xintong,
> >
> > Thanks for the feedback and please see responses below -->
> >
> > On Fri, Jan 28, 2022 at 12:21 AM Xintong Song 
> > wrote:
> >
> > > Thanks Thomas for drafting this FLIP, and everyone for the discussion.
> > >
> > > I also have a few questions and comments.
> > >
> > > ## Job Submission
> > > Deploying a Flink session cluster via kubectl & CR and then submitting
> > jobs
> > > to the cluster via Flink cli / REST is probably the approach that
> > requires
> > > the least effort. However, I'd like to point out 2 weaknesses.
> > > 1. A lot of users use Flink in perjob/application modes. For these
> users,
> > > having to run the job in two steps (deploy the cluster, and submit the
> > job)
> > > is not that convenient.
> > > 2. One of our motivations is being able to manage Flink applications'
> > > lifecycles with kubectl. Submitting jobs from cli sounds not aligned
> with
> > > this motivation.
> > > I think it's probably worth it to support submitting jobs via kubectl &
> > CR
> > > in the first version, both together with deploying the cluster like in
> > > perjob/application mode and after deploying the cluster like in session
> > > mode.
> > >
> >
> > The intention is to support application management through operator and
> CR,
> > which means there won't be any 2 step submission process, which as you
> > allude to would defeat the purpose of this project. The CR example shows
> > the application part. Please note that the bare cluster support is an
> > *additional* feature for scenarios that require external job management.
> Is
> > there anything on the FLIP page that creates a different impression?
> >
> >
> > >
> > > ## Versioning
> > > Which Flink versions does the operator plan to support?
> > > 1. Native K8s deployment was firstly introduced in Flink 1.10
> > > 2. Native K8s HA was introduced in Flink 1.12
> > > 3. The Pod template support was introduced in Flink 1.13
> > > 4. There was some changes to the Flink docker image entrypoint script
> in,
> > > IIRC, Flink 1.13
> > >
> >
> > Great, thanks for providing this. It is important for the compatibility
> > going forward also. We are targeting Flink 1.14.x upwards. Before the
> > operator is ready there will be another Flink release. Let's see if
> anyone
> > is interested in earlier versions?
> >
> >
> > >
> > > ## Compatibility
> > > What kind of API compatibility we can commit to? It's probably fine to
> > have
> > > alpha / beta version APIs that allow incompatible future changes for
> the
> > > first version. But eventually we would need to guarantee backwards
> > > compatibility, so that an early version CR can work with a new version
> > > operator.
> > >
> >
> > Another great point and please let me include that on the FLIP page. ;-)
> >
> > I think we should allow incompatible changes for the first one or two
> > versions, similar to how other major features have evolved recently, such
> > as FLIP-27.
> >
> > Would be great to get broader feedback on this one.
> >
> > Cheers,
> > Thomas
> >
> >
> >
> > >
> > > Thank you~
> > >
> > > Xintong Song
> > >
> > >
> > >
> > > On Fri, Jan 28, 2022 at 1:18 PM Thomas Weise  wrote:
> > >
> > > > Thanks for the feedback!
> > > >
> > > > >
> > > > > # 1 Flink Native vs Standalone integration
> > > > > Maybe we should make this 

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-01 Thread Márton Balassi
Hi team,

Thank you for the great feedback, Thomas has updated the FLIP page
accordingly. If you are comfortable with the currently existing design and
depth in the FLIP [1] I suggest moving forward to the voting stage - once
that reaches a positive conclusion it lets us create the separate code
repository under the flink project for the operator.

I encourage everyone to keep improving the details in the meantime, however
I believe given the existing design and the general sentiment on this
thread that the most efficient path from here is starting the
implementation so that we can collectively iterate over it.

[1]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-212%3A+Introduce+Flink+Kubernetes+Operator

On Mon, Jan 31, 2022 at 10:15 PM Thomas Weise  wrote:

> HI Xintong,
>
> Thanks for the feedback and please see responses below -->
>
> On Fri, Jan 28, 2022 at 12:21 AM Xintong Song 
> wrote:
>
> > Thanks Thomas for drafting this FLIP, and everyone for the discussion.
> >
> > I also have a few questions and comments.
> >
> > ## Job Submission
> > Deploying a Flink session cluster via kubectl & CR and then submitting
> jobs
> > to the cluster via Flink cli / REST is probably the approach that
> requires
> > the least effort. However, I'd like to point out 2 weaknesses.
> > 1. A lot of users use Flink in perjob/application modes. For these users,
> > having to run the job in two steps (deploy the cluster, and submit the
> job)
> > is not that convenient.
> > 2. One of our motivations is being able to manage Flink applications'
> > lifecycles with kubectl. Submitting jobs from cli sounds not aligned with
> > this motivation.
> > I think it's probably worth it to support submitting jobs via kubectl &
> CR
> > in the first version, both together with deploying the cluster like in
> > perjob/application mode and after deploying the cluster like in session
> > mode.
> >
>
> The intention is to support application management through operator and CR,
> which means there won't be any 2 step submission process, which as you
> allude to would defeat the purpose of this project. The CR example shows
> the application part. Please note that the bare cluster support is an
> *additional* feature for scenarios that require external job management. Is
> there anything on the FLIP page that creates a different impression?
>
>
> >
> > ## Versioning
> > Which Flink versions does the operator plan to support?
> > 1. Native K8s deployment was firstly introduced in Flink 1.10
> > 2. Native K8s HA was introduced in Flink 1.12
> > 3. The Pod template support was introduced in Flink 1.13
> > 4. There was some changes to the Flink docker image entrypoint script in,
> > IIRC, Flink 1.13
> >
>
> Great, thanks for providing this. It is important for the compatibility
> going forward also. We are targeting Flink 1.14.x upwards. Before the
> operator is ready there will be another Flink release. Let's see if anyone
> is interested in earlier versions?
>
>
> >
> > ## Compatibility
> > What kind of API compatibility we can commit to? It's probably fine to
> have
> > alpha / beta version APIs that allow incompatible future changes for the
> > first version. But eventually we would need to guarantee backwards
> > compatibility, so that an early version CR can work with a new version
> > operator.
> >
>
> Another great point and please let me include that on the FLIP page. ;-)
>
> I think we should allow incompatible changes for the first one or two
> versions, similar to how other major features have evolved recently, such
> as FLIP-27.
>
> Would be great to get broader feedback on this one.
>
> Cheers,
> Thomas
>
>
>
> >
> > Thank you~
> >
> > Xintong Song
> >
> >
> >
> > On Fri, Jan 28, 2022 at 1:18 PM Thomas Weise  wrote:
> >
> > > Thanks for the feedback!
> > >
> > > >
> > > > # 1 Flink Native vs Standalone integration
> > > > Maybe we should make this more clear in the FLIP but we agreed to do
> > the
> > > > first version of the operator based on the native integration.
> > > > While this clearly does not cover all use-cases and requirements, it
> > > seems
> > > > this would lead to a much smaller initial effort and a nicer first
> > > version.
> > > >
> > >
> > > I'm also leaning towards the native integration, as long as it reduces
> > the
> > > MVP effort. Ultimately the operator will need to also support the
> > > standalone mode. I would like to gain more confidence that native
> > > integration reduces the effort. While it cuts the effort to handle the
> TM
> > > pod creation, some mapping code from the CR to the native integration
> > > client and config needs to be created. As mentioned in the FLIP, native
> > > integration requires the Flink job manager to have access to the k8s
> API
> > to
> > > create pods, which in some scenarios may be seen as unfavorable.
> > >
> > >  > > > # Pod Template
> > > > > > Is the pod template in CR same with what Flink has already
> > > > supported[4]?
> > > > > > Then I am afraid 

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-01-31 Thread Thomas Weise
HI Xintong,

Thanks for the feedback and please see responses below -->

On Fri, Jan 28, 2022 at 12:21 AM Xintong Song  wrote:

> Thanks Thomas for drafting this FLIP, and everyone for the discussion.
>
> I also have a few questions and comments.
>
> ## Job Submission
> Deploying a Flink session cluster via kubectl & CR and then submitting jobs
> to the cluster via Flink cli / REST is probably the approach that requires
> the least effort. However, I'd like to point out 2 weaknesses.
> 1. A lot of users use Flink in perjob/application modes. For these users,
> having to run the job in two steps (deploy the cluster, and submit the job)
> is not that convenient.
> 2. One of our motivations is being able to manage Flink applications'
> lifecycles with kubectl. Submitting jobs from cli sounds not aligned with
> this motivation.
> I think it's probably worth it to support submitting jobs via kubectl & CR
> in the first version, both together with deploying the cluster like in
> perjob/application mode and after deploying the cluster like in session
> mode.
>

The intention is to support application management through operator and CR,
which means there won't be any 2 step submission process, which as you
allude to would defeat the purpose of this project. The CR example shows
the application part. Please note that the bare cluster support is an
*additional* feature for scenarios that require external job management. Is
there anything on the FLIP page that creates a different impression?


>
> ## Versioning
> Which Flink versions does the operator plan to support?
> 1. Native K8s deployment was firstly introduced in Flink 1.10
> 2. Native K8s HA was introduced in Flink 1.12
> 3. The Pod template support was introduced in Flink 1.13
> 4. There was some changes to the Flink docker image entrypoint script in,
> IIRC, Flink 1.13
>

Great, thanks for providing this. It is important for the compatibility
going forward also. We are targeting Flink 1.14.x upwards. Before the
operator is ready there will be another Flink release. Let's see if anyone
is interested in earlier versions?


>
> ## Compatibility
> What kind of API compatibility we can commit to? It's probably fine to have
> alpha / beta version APIs that allow incompatible future changes for the
> first version. But eventually we would need to guarantee backwards
> compatibility, so that an early version CR can work with a new version
> operator.
>

Another great point and please let me include that on the FLIP page. ;-)

I think we should allow incompatible changes for the first one or two
versions, similar to how other major features have evolved recently, such
as FLIP-27.

Would be great to get broader feedback on this one.

Cheers,
Thomas



>
> Thank you~
>
> Xintong Song
>
>
>
> On Fri, Jan 28, 2022 at 1:18 PM Thomas Weise  wrote:
>
> > Thanks for the feedback!
> >
> > >
> > > # 1 Flink Native vs Standalone integration
> > > Maybe we should make this more clear in the FLIP but we agreed to do
> the
> > > first version of the operator based on the native integration.
> > > While this clearly does not cover all use-cases and requirements, it
> > seems
> > > this would lead to a much smaller initial effort and a nicer first
> > version.
> > >
> >
> > I'm also leaning towards the native integration, as long as it reduces
> the
> > MVP effort. Ultimately the operator will need to also support the
> > standalone mode. I would like to gain more confidence that native
> > integration reduces the effort. While it cuts the effort to handle the TM
> > pod creation, some mapping code from the CR to the native integration
> > client and config needs to be created. As mentioned in the FLIP, native
> > integration requires the Flink job manager to have access to the k8s API
> to
> > create pods, which in some scenarios may be seen as unfavorable.
> >
> >  > > > # Pod Template
> > > > > Is the pod template in CR same with what Flink has already
> > > supported[4]?
> > > > > Then I am afraid not the arbitrary field(e.g. cpu/memory resources)
> > > could
> > > > > take effect.
> >
> > Yes, pod template would look almost identical. There are a few settings
> > that the operator will control (and that may need to be blacklisted), but
> > in general we would not want to place restrictions. I think a mechanism
> > where a pod template is merged from multiple layers would also be
> > interesting to make this more flexible.
> >
> > Cheers,
> > Thomas
> >
>


Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-01-28 Thread Xintong Song
Thanks Thomas for drafting this FLIP, and everyone for the discussion.

I also have a few questions and comments.

## Job Submission
Deploying a Flink session cluster via kubectl & CR and then submitting jobs
to the cluster via Flink cli / REST is probably the approach that requires
the least effort. However, I'd like to point out 2 weaknesses.
1. A lot of users use Flink in perjob/application modes. For these users,
having to run the job in two steps (deploy the cluster, and submit the job)
is not that convenient.
2. One of our motivations is being able to manage Flink applications'
lifecycles with kubectl. Submitting jobs from cli sounds not aligned with
this motivation.
I think it's probably worth it to support submitting jobs via kubectl & CR
in the first version, both together with deploying the cluster like in
perjob/application mode and after deploying the cluster like in session
mode.

## Versioning
Which Flink versions does the operator plan to support?
1. Native K8s deployment was firstly introduced in Flink 1.10
2. Native K8s HA was introduced in Flink 1.12
3. The Pod template support was introduced in Flink 1.13
4. There was some changes to the Flink docker image entrypoint script in,
IIRC, Flink 1.13

## Compatibility
What kind of API compatibility we can commit to? It's probably fine to have
alpha / beta version APIs that allow incompatible future changes for the
first version. But eventually we would need to guarantee backwards
compatibility, so that an early version CR can work with a new version
operator.

Thank you~

Xintong Song



On Fri, Jan 28, 2022 at 1:18 PM Thomas Weise  wrote:

> Thanks for the feedback!
>
> >
> > # 1 Flink Native vs Standalone integration
> > Maybe we should make this more clear in the FLIP but we agreed to do the
> > first version of the operator based on the native integration.
> > While this clearly does not cover all use-cases and requirements, it
> seems
> > this would lead to a much smaller initial effort and a nicer first
> version.
> >
>
> I'm also leaning towards the native integration, as long as it reduces the
> MVP effort. Ultimately the operator will need to also support the
> standalone mode. I would like to gain more confidence that native
> integration reduces the effort. While it cuts the effort to handle the TM
> pod creation, some mapping code from the CR to the native integration
> client and config needs to be created. As mentioned in the FLIP, native
> integration requires the Flink job manager to have access to the k8s API to
> create pods, which in some scenarios may be seen as unfavorable.
>
>  > > > # Pod Template
> > > > Is the pod template in CR same with what Flink has already
> > supported[4]?
> > > > Then I am afraid not the arbitrary field(e.g. cpu/memory resources)
> > could
> > > > take effect.
>
> Yes, pod template would look almost identical. There are a few settings
> that the operator will control (and that may need to be blacklisted), but
> in general we would not want to place restrictions. I think a mechanism
> where a pod template is merged from multiple layers would also be
> interesting to make this more flexible.
>
> Cheers,
> Thomas
>


Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-01-27 Thread Thomas Weise
Thanks for the feedback!

>
> # 1 Flink Native vs Standalone integration
> Maybe we should make this more clear in the FLIP but we agreed to do the
> first version of the operator based on the native integration.
> While this clearly does not cover all use-cases and requirements, it seems
> this would lead to a much smaller initial effort and a nicer first version.
>

I'm also leaning towards the native integration, as long as it reduces the
MVP effort. Ultimately the operator will need to also support the
standalone mode. I would like to gain more confidence that native
integration reduces the effort. While it cuts the effort to handle the TM
pod creation, some mapping code from the CR to the native integration
client and config needs to be created. As mentioned in the FLIP, native
integration requires the Flink job manager to have access to the k8s API to
create pods, which in some scenarios may be seen as unfavorable.

 > > > # Pod Template
> > > Is the pod template in CR same with what Flink has already
> supported[4]?
> > > Then I am afraid not the arbitrary field(e.g. cpu/memory resources)
> could
> > > take effect.

Yes, pod template would look almost identical. There are a few settings
that the operator will control (and that may need to be blacklisted), but
in general we would not want to place restrictions. I think a mechanism
where a pod template is merged from multiple layers would also be
interesting to make this more flexible.

Cheers,
Thomas


Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-01-27 Thread Gyula Fóra
Hi All!

Thanks for the questions, there are still quite a few unknowns and
decisions to be made but here are my current thoughts:

# 1 Flink Native vs Standalone integration
Maybe we should make this more clear in the FLIP but we agreed to do the
first version of the operator based on the native integration.
While this clearly does not cover all use-cases and requirements, it seems
this would lead to a much smaller initial effort and a nicer first version.

# How do we run a Flink job from a CR?
I am very much leaning toward using the ApplicationDeployer interface to
submit jobs directly from java. Again this would be a very nice and simple
Java solution.
I think this will also help making the deployment interfaces more solid so
we can then make them public.

If there is no way around it we could also invoke the CLI classes from
within the application but I would prefer not to.

# Pod template
I cannot comment on this yet :D

Cheers,
Gyula

On Wed, Jan 26, 2022 at 12:38 PM Yang Wang  wrote:

> Hi Biao,
>
> # 1 Flink Native vs Standalone integration
> I think we have got a trend in this discussion[1] that the newly introduced
> Flink K8s operator will start with native K8s integration first.
> Do you have some concerns about this?
>
> # 2 K8S StatefulSet v.s. K8S Deployment
> IIUC, the FlinkDeployment is just a custom resource name. It does not mean
> that we need to create a corresponding K8s deployment for JobManager or
> TaskManager.
> If we are using native K8s integration, the JobManager is started with K8s
> deployment while TaskManagers are naked pods managed by
> FlinkResourceManager.
>
> Actually, I think "FlinkDeployment" is easier to understand than
> "FlinkStatefulSet" :)
>
>
> [1]. https://lists.apache.org/thread/l1dkp8v4bhlcyb4tdts99g7w4wdglfy4
>
>
> Best,
> Yang
>
> Biao Geng  于2022年1月26日周三 18:00写道:
>
> > Hi Thomas,
> > Thanks a lot for the great efforts in this well-organized FLIP! After
> > reading the FLIP carefully, I think Yang has given some great feedback
> and
> > I just want to share some of my concerns:
> > # 1 Flink Native vs Standalone integration
> > I believe it is reasonable to support both modes in the long run but in
> the
> > FLIP and previous thread[1], it seems that we have not made a decision on
> > which one to implement initially. The FLIP mentioned "Maybe start with
> > support for Flink Native" for reusing codes in [2]. Is it the selected
> one
> > finally?
> > # 2 K8S StatefulSet v.s. K8S Deployment
> > In the CR Example, I notice that the kind we use is FlinkDeployment. I
> > would like to check if we have made the decision to use K8S Deployment
> > workload resource. As the name implies, StatefulSet is for stateful apps
> > while Deployment is usually for stateless apps. I think it is worthwhile
> to
> > consider the choice more carefully due to some user case in gcp
> > operator[3], which may influence our other design choices(like the Flink
> > application deletion strategy).
> >
> > Again, thanks for the work and I believe this FLIP is pretty useful for
> > many customers and I hope I can make some contributions to this FLIP
> impl!
> >
> > Best regard,
> > Biao Geng
> >
> > [1] https://lists.apache.org/thread/l1dkp8v4bhlcyb4tdts99g7w4wdglfy4
> > [2] https://github.com/wangyang0918/flink-native-k8s-operator
> > [3]
> https://github.com/GoogleCloudPlatform/flink-on-k8s-operator/pull/354
> >
> > Yang Wang  于2022年1月26日周三 15:25写道:
> >
> > > Thanks Thomas for creating FLIP-212 to introduce the Flink Kubernetes
> > > Operator.
> > >
> > > The proposal looks already very good to me and has integrated all the
> > input
> > > in the previous discussion(e.g. native K8s VS standalone, Go VS java).
> > >
> > > I read the FLIP carefully and have some questions that need to be
> > > clarified.
> > >
> > > # How do we run a Flink job from a CR?
> > > 1. Start a session cluster and then followed by submitting the Flink
> job
> > > via rest API
> > > 2. Start a Flink application cluster which bundles one or more Flink
> jobs
> > > It is not clear enough to me which way we will choose. It seems that
> the
> > > existing google/lyft K8s operator is using #1. But I lean to #2 in the
> > new
> > > introduced K8s operator.
> > > If #2 is the case, how could we get the job status when it finished or
> > > failed? Maybe FLINK-24113[1] and FLINK-25715[2] could help. Or we may
> > need
> > > to enable the Flink history server[3].
> > >
> > >
> > > # ApplicationDeployer Interface or "flink run-application" /
> > > "kubernetes-session.sh"
> > > How do we start the Flink application or session cluster?
> > > It will be great if we have the public and stable interfaces for
> > deployment
> > > in Flink. But currently we only have an internal interface
> > > *ApplicationDeployer* to deploy the application cluster and
> > > no interfaces for deploying session cluster.
> > > Of cause, we could also use the CLI command for submission. However, it
> > > will have poor performance when launching 

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-01-26 Thread Yang Wang
Hi Biao,

# 1 Flink Native vs Standalone integration
I think we have got a trend in this discussion[1] that the newly introduced
Flink K8s operator will start with native K8s integration first.
Do you have some concerns about this?

# 2 K8S StatefulSet v.s. K8S Deployment
IIUC, the FlinkDeployment is just a custom resource name. It does not mean
that we need to create a corresponding K8s deployment for JobManager or
TaskManager.
If we are using native K8s integration, the JobManager is started with K8s
deployment while TaskManagers are naked pods managed by
FlinkResourceManager.

Actually, I think "FlinkDeployment" is easier to understand than
"FlinkStatefulSet" :)


[1]. https://lists.apache.org/thread/l1dkp8v4bhlcyb4tdts99g7w4wdglfy4


Best,
Yang

Biao Geng  于2022年1月26日周三 18:00写道:

> Hi Thomas,
> Thanks a lot for the great efforts in this well-organized FLIP! After
> reading the FLIP carefully, I think Yang has given some great feedback and
> I just want to share some of my concerns:
> # 1 Flink Native vs Standalone integration
> I believe it is reasonable to support both modes in the long run but in the
> FLIP and previous thread[1], it seems that we have not made a decision on
> which one to implement initially. The FLIP mentioned "Maybe start with
> support for Flink Native" for reusing codes in [2]. Is it the selected one
> finally?
> # 2 K8S StatefulSet v.s. K8S Deployment
> In the CR Example, I notice that the kind we use is FlinkDeployment. I
> would like to check if we have made the decision to use K8S Deployment
> workload resource. As the name implies, StatefulSet is for stateful apps
> while Deployment is usually for stateless apps. I think it is worthwhile to
> consider the choice more carefully due to some user case in gcp
> operator[3], which may influence our other design choices(like the Flink
> application deletion strategy).
>
> Again, thanks for the work and I believe this FLIP is pretty useful for
> many customers and I hope I can make some contributions to this FLIP impl!
>
> Best regard,
> Biao Geng
>
> [1] https://lists.apache.org/thread/l1dkp8v4bhlcyb4tdts99g7w4wdglfy4
> [2] https://github.com/wangyang0918/flink-native-k8s-operator
> [3] https://github.com/GoogleCloudPlatform/flink-on-k8s-operator/pull/354
>
> Yang Wang  于2022年1月26日周三 15:25写道:
>
> > Thanks Thomas for creating FLIP-212 to introduce the Flink Kubernetes
> > Operator.
> >
> > The proposal looks already very good to me and has integrated all the
> input
> > in the previous discussion(e.g. native K8s VS standalone, Go VS java).
> >
> > I read the FLIP carefully and have some questions that need to be
> > clarified.
> >
> > # How do we run a Flink job from a CR?
> > 1. Start a session cluster and then followed by submitting the Flink job
> > via rest API
> > 2. Start a Flink application cluster which bundles one or more Flink jobs
> > It is not clear enough to me which way we will choose. It seems that the
> > existing google/lyft K8s operator is using #1. But I lean to #2 in the
> new
> > introduced K8s operator.
> > If #2 is the case, how could we get the job status when it finished or
> > failed? Maybe FLINK-24113[1] and FLINK-25715[2] could help. Or we may
> need
> > to enable the Flink history server[3].
> >
> >
> > # ApplicationDeployer Interface or "flink run-application" /
> > "kubernetes-session.sh"
> > How do we start the Flink application or session cluster?
> > It will be great if we have the public and stable interfaces for
> deployment
> > in Flink. But currently we only have an internal interface
> > *ApplicationDeployer* to deploy the application cluster and
> > no interfaces for deploying session cluster.
> > Of cause, we could also use the CLI command for submission. However, it
> > will have poor performance when launching multiple applications.
> >
> >
> > # Pod Template
> > Is the pod template in CR same with what Flink has already supported[4]?
> > Then I am afraid not the arbitrary field(e.g. cpu/memory resources) could
> > take effect.
> >
> >
> > [1]. https://issues.apache.org/jira/browse/FLINK-24113
> > [2]. https://issues.apache.org/jira/browse/FLINK-25715
> > [3].
> >
> >
> https://nightlies.apache.org/flink/flink-docs-release-1.14/docs/deployment/advanced/historyserver/
> > [4].
> >
> >
> https://nightlies.apache.org/flink/flink-docs-release-1.14/docs/deployment/resource-providers/native_kubernetes/#pod-template
> >
> >
> >
> > Best,
> > Yang
> >
> >
> > Thomas Weise  于2022年1月25日周二 13:08写道:
> >
> > > Hi,
> > >
> > > As promised in [1] we would like to start the discussion on the
> > > addition of a Kubernetes operator to the Flink project as FLIP-212:
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-212%3A+Introduce+Flink+Kubernetes+Operator
> > >
> > > Please note that the FLIP is currently focussed on the overall
> > > direction; the intention is to fill in more details once we converge
> > > on the high level plan.
> > >
> > > Thanks and looking forward to a 

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-01-26 Thread Biao Geng
Hi Thomas,
Thanks a lot for the great efforts in this well-organized FLIP! After
reading the FLIP carefully, I think Yang has given some great feedback and
I just want to share some of my concerns:
# 1 Flink Native vs Standalone integration
I believe it is reasonable to support both modes in the long run but in the
FLIP and previous thread[1], it seems that we have not made a decision on
which one to implement initially. The FLIP mentioned "Maybe start with
support for Flink Native" for reusing codes in [2]. Is it the selected one
finally?
# 2 K8S StatefulSet v.s. K8S Deployment
In the CR Example, I notice that the kind we use is FlinkDeployment. I
would like to check if we have made the decision to use K8S Deployment
workload resource. As the name implies, StatefulSet is for stateful apps
while Deployment is usually for stateless apps. I think it is worthwhile to
consider the choice more carefully due to some user case in gcp
operator[3], which may influence our other design choices(like the Flink
application deletion strategy).

Again, thanks for the work and I believe this FLIP is pretty useful for
many customers and I hope I can make some contributions to this FLIP impl!

Best regard,
Biao Geng

[1] https://lists.apache.org/thread/l1dkp8v4bhlcyb4tdts99g7w4wdglfy4
[2] https://github.com/wangyang0918/flink-native-k8s-operator
[3] https://github.com/GoogleCloudPlatform/flink-on-k8s-operator/pull/354

Yang Wang  于2022年1月26日周三 15:25写道:

> Thanks Thomas for creating FLIP-212 to introduce the Flink Kubernetes
> Operator.
>
> The proposal looks already very good to me and has integrated all the input
> in the previous discussion(e.g. native K8s VS standalone, Go VS java).
>
> I read the FLIP carefully and have some questions that need to be
> clarified.
>
> # How do we run a Flink job from a CR?
> 1. Start a session cluster and then followed by submitting the Flink job
> via rest API
> 2. Start a Flink application cluster which bundles one or more Flink jobs
> It is not clear enough to me which way we will choose. It seems that the
> existing google/lyft K8s operator is using #1. But I lean to #2 in the new
> introduced K8s operator.
> If #2 is the case, how could we get the job status when it finished or
> failed? Maybe FLINK-24113[1] and FLINK-25715[2] could help. Or we may need
> to enable the Flink history server[3].
>
>
> # ApplicationDeployer Interface or "flink run-application" /
> "kubernetes-session.sh"
> How do we start the Flink application or session cluster?
> It will be great if we have the public and stable interfaces for deployment
> in Flink. But currently we only have an internal interface
> *ApplicationDeployer* to deploy the application cluster and
> no interfaces for deploying session cluster.
> Of cause, we could also use the CLI command for submission. However, it
> will have poor performance when launching multiple applications.
>
>
> # Pod Template
> Is the pod template in CR same with what Flink has already supported[4]?
> Then I am afraid not the arbitrary field(e.g. cpu/memory resources) could
> take effect.
>
>
> [1]. https://issues.apache.org/jira/browse/FLINK-24113
> [2]. https://issues.apache.org/jira/browse/FLINK-25715
> [3].
>
> https://nightlies.apache.org/flink/flink-docs-release-1.14/docs/deployment/advanced/historyserver/
> [4].
>
> https://nightlies.apache.org/flink/flink-docs-release-1.14/docs/deployment/resource-providers/native_kubernetes/#pod-template
>
>
>
> Best,
> Yang
>
>
> Thomas Weise  于2022年1月25日周二 13:08写道:
>
> > Hi,
> >
> > As promised in [1] we would like to start the discussion on the
> > addition of a Kubernetes operator to the Flink project as FLIP-212:
> >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-212%3A+Introduce+Flink+Kubernetes+Operator
> >
> > Please note that the FLIP is currently focussed on the overall
> > direction; the intention is to fill in more details once we converge
> > on the high level plan.
> >
> > Thanks and looking forward to a lively discussion!
> >
> > Thomas
> >
> > [1] https://lists.apache.org/thread/l1dkp8v4bhlcyb4tdts99g7w4wdglfy4
> >
>


Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-01-25 Thread Yang Wang
Thanks Thomas for creating FLIP-212 to introduce the Flink Kubernetes
Operator.

The proposal looks already very good to me and has integrated all the input
in the previous discussion(e.g. native K8s VS standalone, Go VS java).

I read the FLIP carefully and have some questions that need to be clarified.

# How do we run a Flink job from a CR?
1. Start a session cluster and then followed by submitting the Flink job
via rest API
2. Start a Flink application cluster which bundles one or more Flink jobs
It is not clear enough to me which way we will choose. It seems that the
existing google/lyft K8s operator is using #1. But I lean to #2 in the new
introduced K8s operator.
If #2 is the case, how could we get the job status when it finished or
failed? Maybe FLINK-24113[1] and FLINK-25715[2] could help. Or we may need
to enable the Flink history server[3].


# ApplicationDeployer Interface or "flink run-application" /
"kubernetes-session.sh"
How do we start the Flink application or session cluster?
It will be great if we have the public and stable interfaces for deployment
in Flink. But currently we only have an internal interface
*ApplicationDeployer* to deploy the application cluster and
no interfaces for deploying session cluster.
Of cause, we could also use the CLI command for submission. However, it
will have poor performance when launching multiple applications.


# Pod Template
Is the pod template in CR same with what Flink has already supported[4]?
Then I am afraid not the arbitrary field(e.g. cpu/memory resources) could
take effect.


[1]. https://issues.apache.org/jira/browse/FLINK-24113
[2]. https://issues.apache.org/jira/browse/FLINK-25715
[3].
https://nightlies.apache.org/flink/flink-docs-release-1.14/docs/deployment/advanced/historyserver/
[4].
https://nightlies.apache.org/flink/flink-docs-release-1.14/docs/deployment/resource-providers/native_kubernetes/#pod-template



Best,
Yang


Thomas Weise  于2022年1月25日周二 13:08写道:

> Hi,
>
> As promised in [1] we would like to start the discussion on the
> addition of a Kubernetes operator to the Flink project as FLIP-212:
>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-212%3A+Introduce+Flink+Kubernetes+Operator
>
> Please note that the FLIP is currently focussed on the overall
> direction; the intention is to fill in more details once we converge
> on the high level plan.
>
> Thanks and looking forward to a lively discussion!
>
> Thomas
>
> [1] https://lists.apache.org/thread/l1dkp8v4bhlcyb4tdts99g7w4wdglfy4
>