rator picks up that resource and tries to create a savepoint
> > > > indefinitely until the savepoint has been successfully created. We
> > report
> > > > the savepoint status and location in the "status" field.
> > > >
> > > > We could even
+1 (binding) since ZK HA is still widely used.
Best,
Yang
On Thu, Mar 14, 2024 at 6:27 PM Matthias Pohl
wrote:
> Nothing to add from my side. Thanks, Alex.
>
> +1 (binding)
>
> On Thu, Mar 7, 2024 at 4:09 PM Alex Nitavsky
> wrote:
>
> > Hi everyone,
> >
> > I'd like to start a vote on FLIP-40
This might be related with FLINK-28481, which is a bug in fabric8io k8s
client.
[1]. https://issues.apache.org/jira/browse/FLINK-28481
Best,
Yang
On Tue, Feb 6, 2024 at 12:30 PM Lavkesh Lahngir wrote:
> Hi, Matthias, I was wondering if there are any timeout or heartbeat
> configurations for Ku
I am completely in favor of splitting the LeaderServices and
PersistenceServices
while sharing the same concern that MaterialProvider is not very easy to
understand.
It just feels like we do the separation but not thoroughly.
If you have a clear plan for the subsequent improvements, I am fine that
I could share some metrics about Alibaba Cloud EMR clusters.
The ratio of Hadoop2 VS Hadoop3 is 1:3.
Best,
Yang
On Thu, Dec 28, 2023 at 8:16 PM Martijn Visser
wrote:
> Hi all,
>
> I want to get some insights on how many users are still using Hadoop 2
> vs how many users are using Hadoop 3. Fli
+1 (binding)
Best,
Yang
On Thu, Jan 11, 2024 at 9:53 AM liu ron wrote:
> +1 non-binding
>
> Best
> Ron
>
> Matthias Pohl 于2024年1月10日周三 23:05写道:
>
> > +1 (binding)
> >
> > On Wed, Jan 10, 2024 at 3:35 PM ConradJam wrote:
> >
> > > +1 non-binding
> > >
> > > Dawid Wysakowicz 于2024年1月10日周三 21:
Yang Wang created FLINK-33155:
-
Summary: Flink ResourceManager continuously fails to start TM
container on YARN when Kerberos enabled
Key: FLINK-33155
URL: https://issues.apache.org/jira/browse/FLINK-33155
Since the users could always use the old Flink Kubernetes Operator version
along with old Flink versions, I am totally in favor of this proposal to
reduce maintenance burden.
Best,
Yang
Biao Geng 于2023年9月6日周三 18:15写道:
> +1 for the proposal.
>
> Best,
> Biao Geng
>
> Gyula Fóra 于2023年9月6日周三 16:
+1 (binding)
Best,
Yang
Becket Qin 于2023年9月14日周四 11:01写道:
> +1 (binding)
>
> Thanks for the FLIP, Archit.
>
> Cheers,
>
> Jiangjie (Becket) Qin
>
>
> On Thu, Sep 14, 2023 at 10:31 AM Dong Lin wrote:
>
> > Thanks Archit for the FLIP.
> >
> > +1 (binding)
> >
> > Regards,
> > Dong
> >
> > On Thu
+1 for this FLIP
Maybe a FLIP is an overkill for this enhancement.
Best,
Yang
Venkatakrishnan Sowrirajan 于2023年8月23日周三 01:44写道:
> Thanks for the FLIP, Archit.
>
> This is definitely quite a useful addition to the *yarn.provided.lib.dirs*
> . +1.
>
> IMO, except for the fact that *yarn.provide
Sorry for the late reply. I am in favor of introducing such a built-in
resource localization mechanism
based on Flink FileSystem. Then FLINK-28915[1] could be the second step
which will download
the jars and dependencies to the JobManager/TaskManager local directory
before working.
The first step
+1 (binding)
Best,
Yang
Becket Qin 于2023年6月6日周二 10:35写道:
> +1 (binding)
>
> Thanks for driving the FLIP, Archit.
>
> Cheers,
>
> Jiangjie (Becket) Qin
>
> On Tue, Jun 6, 2023 at 4:33 AM Venkatakrishnan Sowrirajan <
> vsowr...@asu.edu>
> wrote:
>
> > Thanks for starting the vote on this one, Arc
ore descriptive.
> 3. ACLs apply to logs as well as permissions to kill the application.
> Also, in the PR we are setting ACLs for Task Manager
> (createTaskExecutorContext) as well as Job Manager (startAppMaster).
>
> Thanks,
> Archit Goyal
>
> From: Yang Wang
> Date:
Thanks for creating this FLIP.
This sounds like a useful feature to make the Flink applications running on
YARN cluster more securely.
However, I think we still miss some important parts in the FLIP.
1. Which hadoop versions this FLIP relies on?
2. We need to describe a bit more about how the YAR
Thanks Gyula for driving the release again.
It is really exciting to see the auto-scaling coming out of the box.
Best,
Yang
Gyula Fóra 于2023年2月6日周一 19:43写道:
> Hi Devs!
>
> Based on the previously agreed upon release schedule (
>
> https://cwiki.apache.org/confluence/display/FLINK/Release+Sched
+1 (Binding)
Best,
Yang
ConradJam 于2023年1月31日周二 12:09写道:
> +1 non-binding
>
> Matthias Pohl 于2023年1月25日周三 17:34写道:
>
> > Hi everyone,
> > After the discussion thread [1] on FLIP-285 [2] didn't bring up any new
> > items, I want to start voting on FLIP-285. This FLIP will not only align
> > the
Having the *start()* in *LeaderContender* interface and bringing back the
*LeaderElection* with some new methods make sense to me.
I have no more concerns now.
>- *LeaderContender*: The LeaderContender is integrated as usual except
>that it accesses the LeaderElection instead of the Lead
vice might be
> good enough. But I still kept the LeaderElection interface as an (optional)
> extension of FLIP-285 as it might improve testability a bit. I added some
> diagrams and descriptions to the FLIP hoping that this helps answer your
> questions, Yang.
>
> Best,
> Matth
Do you build your own flink-kubernetes-operator image with the flink-s3-fs
plugin bundled[1]?
[1].
https://nightlies.apache.org/flink/flink-kubernetes-operator-docs-release-1.3/docs/custom-resource/overview/#flinksessionjob-spec-overview
Best,
Yang
Weihua Hu 于2023年1月17日周二 10:47写道:
> Hi, Rahul
Matthias Pohl 于2023年1月12日周四 22:07写道:
> Thanks Yang Wang for sharing your view on this. Please find my responses
> below.
>
> # HA data format in the HA backend(e.g. ZK, K8s ConfigMap)
> > We have already changed the HA data format after introducing the multiple
> > comp
Thanks Matthias for preparing this thorough FLIP, which has taken us
reviewing the multiple component leader election.
I am totally in favor of doing the code clean-up. The current
implementation does not have very good readability due to legacy
compatibility.
And I just have a few comments.
# HA
Congratulations, Lincoln!
Best,
Yang
Lincoln Lee 于2023年1月12日周四 12:13写道:
> Thank you all!
>
> I'm honored to join the committers and look forward to continue working
> with the community.
>
> Best,
> Lincoln Lee
>
>
> Shengkai Fang 于2023年1月12日周四 09:55写道:
>
> > Congratulations, Lincoln!
> >
> >
+1 (binding)
Best,
Yang
Őrhidi Mátyás 于2022年11月24日周四 12:04写道:
> +1 (binding)
>
> On Wed, Nov 23, 2022 at 11:46 AM Gyula Fóra wrote:
>
> > +1 (binding)
> >
> > Gyula
> >
> > On Wed, Nov 23, 2022 at 5:25 PM Maximilian Michels
> wrote:
> >
> > > Hi everyone,
> > >
> > > I'd like to start a vote
Congrats Matyas!
Best,
Yang
Maximilian Michels 于2022年11月23日周三 23:27写道:
> Congrats Matyas! Finally :)
>
> On Wed, Nov 23, 2022 at 3:32 PM Martijn Visser
> wrote:
>
> > Congratulations and welcome!
> >
> > On Wed, Nov 23, 2022 at 2:25 AM yuxia
> wrote:
> >
> > > Congrats Matyas!
> > >
> > > Bes
Improving the visibility of Flink Kubernetes Operator is great. And I agree
OLM could help with this.
I just hope this will not make the whole release process too complicated.
Of course, if we want to integrate the OLM into the official release, it
should be tested by the users easily.
Best,
Yang
Congratulations, Godfrey and Xingbo!
Best,
Yang
Jing Ge 于2022年11月24日周四 02:06写道:
> Congrats, Godfrey! Congrats, Xingbo!
>
> Best regards,
> Jing
>
> On Wed, Nov 23, 2022 at 6:11 PM Maximilian Michels wrote:
>
> > Welcome aboard Godfrey and Xingbo!
> >
> > -Max
> >
> > On Wed, Nov 23, 2022 at 5:
Thanks for the fruitful discussion and I am really excited to see that the
auto-scaling really happens for
Flink Kubernetes operator. It will be a very important step to make the
long-running Flink job more smoothly.
I just have some immature ideas and want to share them here.
# Resource Reserv
Thanks Matthias for continuously improving the clean-up process.
Given that we highly depends on K8s APIServer for HA implementation, I am
not in favor of storing too many entries in the ConfigMap,
as well as adding more update requests to the APIServer. So I lean towards
Proposal #2. It just work
Thanks Jacky Lau for starting this discussion.
I understand that you are trying to find a convenient way to specify
dependency jars along with user jar. However,
let's try to narrow down by differentiating deployment modes.
# Standalone mode
No matter you are using the standalone mode on virtual
+1(binding)
* Built from source
* Verified signature and checksum
* Build docker image with flink binary
* Submit/stop a streaming and batch job with Flink Kubernetes Operator and
everything works well
* Check the metrics and logs via ingress webUI
Best,
Yang
Mason Chen 于2022年10月25日周二 14:43写道:
Given that we do not bundle any hadoop classes in the Flink binary, do you
mean simply bump the hadoop version in the parent pom?
If it is, why do not we use the latest stable hadoop version 3.3.4? It
seems that our cron build has verified that hadoop3 could work.
Best,
Yang
David Morávek 于2022年
Yang Wang created FLINK-29705:
-
Summary: Document the least access with RBAC setting for native
K8s integration
Key: FLINK-29705
URL: https://issues.apache.org/jira/browse/FLINK-29705
Project: Flink
+1 for increasing the visibility of flink-kubernetes-operator.
Best,
Yang
Thomas Weise 于2022年10月13日周四 07:49写道:
> +1
>
>
> On Wed, Oct 12, 2022 at 5:03 PM Martijn Visser
> wrote:
>
> > +1 from my end to include the operator in the related Kubernetes sections
> > of the Flink docs
> >
> > On Wed
Congratulations Danny!
Best,
Yang
Hang Ruan 于2022年10月13日周四 10:58写道:
> Congratulations Danny!
>
> Best,
> Hang
>
> Yun Gao 于2022年10月13日周四 10:56写道:
>
> > Congratulations Danny!
> > Best,
> > Yun Gao
> > --
> > From:yuxia
> > Send T
This will make it possible to replace the current rough implementation[1]
with a common yaml parser.
And then we could avoid some unexpected behaviors[2].
+1
[1].
https://github.com/apache/flink/blob/master/flink-core/src/main/java/org/apache/flink/configuration/GlobalConfiguration.java#L179
[2].
Thanks Gyula for managing the release.
+1 for the time schedule.
Best,
Yang
Őrhidi Mátyás 于2022年9月19日周一 22:28写道:
> Thanks Gyula!
>
> Sounds good! Happy to help as always.
>
> Cheers,
> Matyas
>
> On Mon, Sep 19, 2022 at 1:37 PM Gyula Fóra wrote:
>
> > Hi Devs!
> >
> > The originally planne
I think you have already found the solution.
Pod template[1] is exactly what you want.
[1].
https://nightlies.apache.org/flink/flink-docs-release-1.15/docs/deployment/resource-providers/native_kubernetes/#pod-template
Best,
Yang
Hao t Chang 于2022年9月13日周二 07:41写道:
> Hi Biao
> I think this modi
Congrats, Martijn!
Best,
Yang
Lijie Wang 于2022年9月13日周二 10:10写道:
> Congratulations, Martijn!
>
> Best,
> Lijie
>
> yuxia 于2022年9月13日周二 09:52写道:
>
> > Congrats, Martijn!
> >
> > Best regards,
> > Yuxia
> >
> > - 原始邮件 -
> > 发件人: "Steven Wu"
> > 收件人: "dev"
> > 发送时间: 星期二, 2022年 9 月 13日 上午
Congrats Caizhi
Best,
Yang
Jark Wu 于2022年9月9日周五 11:46写道:
> Congrats ChaiZhi!
>
> Cheers,
> Jark
>
> > 2022年9月9日 11:26,Lijie Wang 写道:
> >
> > Congratulations Caizhi
> >
> > Best,
> > Lijie
> >
> > Yuxin Tan 于2022年9月9日周五 10:19写道:
> >
> >> Caizhi, Congratulations!
> >>
> >> Best,
> >> Yuxin
> >>
Thanks Bo for starting this discussion.
I think it is really useful to have the CI for ARM platform. But I am not
sure what is the current situation. TBH, I have not build an ARM image for
Flink.
Given that we have not finished the migration from azure pipelines to
github action, I believe we sti
Congrats Junhan!
Best,
Yang
Matthias Pohl 于2022年8月22日周一 15:35写道:
> Congratulations & welcome! :-)
>
> Matthias
>
> On Sun, Aug 21, 2022 at 5:42 AM Yuan Mei wrote:
>
> > Congratulations Junhan!
> >
> > Best,
> > Yuan
> >
> > On Sat, Aug 20, 2022 at 2:11 PM Danny Cranmer
> > wrote:
> >
> > > C
Congrats Lijie!
Best,
Yang
Matthias Pohl 于2022年8月22日周一 15:34写道:
> Congrats & welcome to the team! :-)
>
> Matthias
>
> On Sun, Aug 21, 2022 at 5:41 AM Yuan Mei wrote:
>
> > Congratulations, Lijie!
> >
> > Best,
> > Yuan
> >
> > On Sat, Aug 20, 2022 at 2:12 PM Danny Cranmer
> > wrote:
> >
> >
+1 (binding)
Best,
Yang
bo zhaobo 于2022年7月25日周一 09:38写道:
> Hi all,
>
> Thank you very much for all feedback after the discussion in [2][3].
> Now I'd like to proceed with the vote for FLIP-250 [1], as no more
> objections
> or issues were raised in ML thread [2][3].
>
> The vote will be opened
Congrats! Thanks Gyula for driving this release, and thanks to all
contributors!
Best,
Yang
Gyula Fóra 于2022年7月25日周一 10:44写道:
> The Apache Flink community is very happy to announce the release of Apache
> Flink Kubernetes Operator 1.1.0.
>
> The Flink Kubernetes Operator allows users to manage
+1 (binding)
Successfully verified the following:
- Verify that the checksums and GPG files
- Verify that the source distributions do not contain any binaries
- Build binary and image from release source
- Verify the NOTICE and licenses in source release and the docker image
- Verify the helm
I think we could go over the customized scheduler plugin mechanism again
with YuniKorn to make sure that it is common enough.
But the implementation could be deferred.
And maybe we also could ping Yikun Jiang who has done similar things in
Spark.
For the e2e tests, I admit that they could be impr
Thanks for the explanation. Only having 1 API call in most cases makes
sense to me.
Could you please elaborate more about why do we need the *plan* in CR
status?
Best,
Yang
Gyula Fóra 于2022年7月12日周二 17:36写道:
> Hi Devs!
>
> I discussed with Daren offline, and I agree with him that technically w
I share mytyas's concern if we list the jobs first and then followed by
some get-job-detail requests.
It is a bit expensive and I do not see the benefit to store jobPlan in the
CR status.
Best,
Yang
Őrhidi Mátyás 于2022年7月11日周一 21:43写道:
> Hi Daren,
>
> At the moment the Operator fetches the job
Yang Wang created FLINK-28481:
-
Summary: Bump the fabric8 kubernetes-client to 5.12.3
Key: FLINK-28481
URL: https://issues.apache.org/jira/browse/FLINK-28481
Project: Flink
Issue Type: Bug
onfluence/display/FLINK/Release+Schedule+and+Planning
> > > >>
> > > >> On Tue, Jul 5, 2022 at 2:10 PM Őrhidi Mátyás <
> matyas.orh...@gmail.com>
> > > >> wrote:
> > > >>
> > > >>> Both sql and py support is re
Thanks zhaobo for starting the discussion and preparing the FLIP.
The customized Kubernetes Schedulers support will be very helpful for the
users who still hesitates to migrate the Flink workloads from YARN to
Kubernetes.
Now leveraging the ability of customized K8s scheduler, many advanced
schedu
> https://nightlies.apache.org/flink/flink-docs-master/docs/dev/python/dependency_management/
>
> On Mon, Jul 4, 2022 at 11:52 AM Yang Wang wrote:
>
>> AFAIK, the python job could be considered as a special case of jar job.
>> The user jar is flink-python-*.jar and is located in
AFAIK, the python job could be considered as a special case of jar job. The
user jar is flink-python-*.jar and is located in the opt directory.
The python script is just the argument of this user jar. So I believe the
users already could submit python jobs via Flink Kubernetes operator.
However, th
+1 (binding)
Best,
Yang
Zhu Zhu 于2022年6月29日周三 14:31写道:
> +1 (binding)
>
> Thanks,
> Zhu
>
> Xintong Song 于2022年6月23日周四 17:01写道:
> >
> > +1 (binding)
> >
> > Best,
> >
> > Xintong
> >
> >
> >
> > On Thu, Jun 23, 2022 at 1:49 PM Yangze Guo wrote:
> >
> > > +1 (binding)
> > >
> > > Best,
> > > Y
Thanks Gyula for working on the first patch release for the Flink
Kubernetes Operator project.
Best,
Yang
Gyula Fóra 于2022年6月28日周二 00:22写道:
> The Apache Flink community is very happy to announce the release of Apache
> Flink Kubernetes Operator 1.0.1.
>
> The Flink Kubernetes Operator allows
+1 (binding)
Successfully verified the following:
- Verify that the checksums and GPG files
- Verify that the source distributions do not contain any binaries
- Build binary and image from release source
- Verify the NOTICE and licenses in source release and the docker image
- Verify the helm char
Thanks Gyula for preparing the first patch release for Flink Kubernetes
operator.
+1 for this.
Best,
Yang
Őrhidi Mátyás 于2022年6月22日周三 23:38写道:
> +1 for the patch release. Thanks Gyula!
>
> On Wed, Jun 22, 2022 at 5:35 PM Márton Balassi
> wrote:
>
>> Hi team,
>>
>> +1 for having a 1.0.1 for th
+1 for 2 month release cycles.
Since we have promised the backward compatibility for CRD, I think it is
also reasonable for us to maintain the latest two minor versions with patch
releases.
Given that we only have 5~6 weeks for feature development, maybe we need to
confirm down the features as so
Congratulations, Qingsheng and ShengKai.
Best,
Yang
Benchao Li 于2022年6月21日周二 19:33写道:
> Congratulations!
>
> weijie guo 于2022年6月21日周二 13:44写道:
>
> > Congratulations, Qingsheng and ShengKai!
> >
> > Best regards,
> >
> > Weijie
> >
> >
> > Yuan Mei 于2022年6月21日周二 13:07写道:
> >
> > > Congrats Qi
+1(binding)
Best,
Yang
Yun Gao 于2022年6月17日周五 18:03写道:
> +1 (binding)
>
> Thanks for the discussion and updates!
>
> Best,
> Yun Gao
>
>
> --
> From:Peter Huang
> Send Time:2022 Jun. 16 (Thu.) 00:05
> To:dev
> Subject:Re: [VOTE] F
t,
>
> Xintong
>
>
>
> On Thu, Jun 16, 2022 at 11:43 AM Yang Wang wrote:
>
> > This is a very useful feature both for finished streaming and batch jobs.
> >
> > Except for the WebUI & REST API improvements, I am curious whether we
> could
> > also
>From what I have learned from the last 1.0.0 release, I think it is not too
complicated and a big burden for flink-kubernetes-operator patch releases.
The major bottleneck might be the VOTE duration(e.g. jet lag, the weekend).
We almost need one week for the release after all the blockers are reso
This is a very useful feature both for finished streaming and batch jobs.
Except for the WebUI & REST API improvements, I am curious whether we could
also integrate some critical information(e.g. latest checkpoint) into the
job result store[1].
I am just feeling this is also somehow related with "
Congrats, Jingsong!
Best,
Yang
Zakelly Lan 于2022年6月16日周四 11:16写道:
> Congrats & well deserved!
>
> Best,
> Zakelly
>
> On Thu, Jun 16, 2022 at 10:36 AM Guowei Ma wrote:
>
> > Congrats, Jingsong!
> >
> > Best,
> > Guowei
> >
> >
> > On Thu, Jun 16, 2022 at 9:49 AM Hangxiang Yu
> wrote:
> >
> >
The Apache Flink community is very happy to announce the release of Apache
Flink Kubernetes Operator 1.0.0.
The Flink Kubernetes Operator allows users to manage their Apache Flink
applications and their lifecycle through native k8s tooling like kubectl.
This is the first production ready release a
I'm happy to announce that we have unanimously approved this release.
There are 5 approving votes, 3 of which are binding:
* Marton Balassi (binding)
* Gyula Fora (binding)
* Biao Geng (non-binding)
* Jim Busche (non-binding)
* Yang Wang (binding)
There are no disapproving votes.
I am closing this VOTE since it has run for enough time and there's no more
feedback.
Best,
Yang
Yang Wang 于2022年6月2日周四 12:35写道:
> +1 (binding)
>
> Successfully verified the following:
> - Verify that the checksums and GPG files
> - Verify that the source distribution
+1 (binding)
Successfully verified the following:
- Verify that the checksums and GPG files
- Verify that the source distributions do not contain any binaries
- Build binary and image from release source
- Verify the NOTICE and licenses in source release and the docker image
- Verify the helm char
Hi everyone,
Please review and vote on the release candidate #4 for the version 1.0.0 of
Apache Flink Kubernetes Operator,
as follows:
[ ] +1, Approve the release
[ ] -1, Do not approve the release (please provide specific comments)
**Release Overview**
As an overview, the release consists of th
o also set scheduler and declarative resource management settings
> behind the scenes.
>
> Thanks
>
> [1]
> https://kubernetes.io/docs/tasks/extend-kubernetes/custom-resources/custom-resource-definitions/#scale-subresource
>
> On Mon, May 30, 2022 at 2:25 AM Yang Wang wrote:
&g
Yang Wang created FLINK-27860:
-
Summary: List the CSS/docs dependencies in the NOTICE
Key: FLINK-27860
URL: https://issues.apache.org/jira/browse/FLINK-27860
Project: Flink
Issue Type: Bug
Thanks all for your testing and patience.
And sorry for I have to cancel this VOTE since @Márton Balassi
found a license issue. We do not list the
CSS/docs dependencies in the NOTICE file of source release[1].
I will create another release candidate today including this fix. Given
that there's n
Nicholas Jiang
>
> On 2022/05/31 06:26:02 Yang Wang wrote:
> > Hi everyone,
> >
> > Please review and vote on the release candidate #3 for the version 1.0.0
> of
> > Apache Flink Kubernetes Operator,
> > as follows:
> > [ ] +1, Approve the
Hi everyone,
Please review and vote on the release candidate #3 for the version 1.0.0 of
Apache Flink Kubernetes Operator,
as follows:
[ ] +1, Approve the release
[ ] -1, Do not approve the release (please provide specific comments)
**Release Overview**
As an overview, the release consists of th
loyment box?
>
> Cheers,
> Gyula
>
> On Wed, May 25, 2022 at 7:50 PM Talat Uyarer
> wrote:
>
>> Hi Yang,
>>
>> I thought we could enable Adaptive Scheduler, so adding or removing a task
>> manager is the same as restarting a job when we use an adaptive
Yang Wang created FLINK-27834:
-
Summary: Flink kubernetes operator dockerfile could not work with
podman
Key: FLINK-27834
URL: https://issues.apache.org/jira/browse/FLINK-27834
Project: Flink
Thanks Jim for providing the information and Gyula for sharing the concerns.
I will create the release candidate #3 after we have more progress on
the FLINK-27804, whether we are sure it is resolved or we have found the
root cause.
Also I will try to make the Dockerfile could work for podman.
B
Thanks Jim for the testing.
Could you please share the docker version you are using to build the image?
It works well for "20.10.8".
*COPY *.git ./.git*
The above copy command should ignore the .git directory if it does not
exist.
Best,
Yang
Jim Busche 于2022年5月27日周五 02:57写道:
> Hi Yang,
>
>
>
Hi everyone,
Please review and vote on the release candidate #2 for the version 1.0.0 of
Apache Flink Kubernetes Operator,
as follows:
[ ] +1, Approve the release
[ ] -1, Do not approve the release (please provide specific comments)
**Release Overview**
As an overview, the release consists of th
does not contain unexpected things
>
> I have not found new issues besides what Gyula has mentioned. I will follow
> our jira list to validate other new features or improvements.
>
> Best,
> Biao Geng
>
>
> Yang Wang 于2022年5月24日周二 16:08写道:
>
> > Thanks Gyula for s
Thanks for the interesting discussion.
Compared with reactive mode, leveraging the flink-kubernetes-operator to do
the job restarting/upgrading is another solution for auto-scaling.
Given that fully restarting a Flink application on K8s is not too slow,
this is a reasonable way.
Really hope we cou
Yang Wang created FLINK-27759:
-
Summary: Rethink how to get the git commit id for docker image in
Flink Kubernetes operator
Key: FLINK-27759
URL: https://issues.apache.org/jira/browse/FLINK-27759
Project
)
> - I could not build the Docker image from the source release, getting the
> following error:
>
>
> > [build 11/14] COPY .git ./.git:
>
> --
>
> failed to compute cache key: "/.git" not found: not found
>
>
> I will continue with further func
Yang Wang created FLINK-27747:
-
Summary: Flink kubernetes operator helm chart release the
Chart.yaml file doesn't have an apache license header
Key: FLINK-27747
URL: https://issues.apache.org/jira/browse/FLINK-
Yang Wang created FLINK-27746:
-
Summary: Flink kubernetes operator docker image could not build
with source release
Key: FLINK-27746
URL: https://issues.apache.org/jira/browse/FLINK-27746
Project: Flink
Hi everyone,
Please review and vote on the release candidate #1 for the version 1.0.0 of
Apache Flink Kubernetes Operator,
as follows:
[ ] +1, Approve the release
[ ] -1, Do not approve the release (please provide specific comments)
**Release Overview**
As an overview, the release consists of th
All the blockers and major issues have been merged into release-1.0 branch.
Just follow what we have promised, I am preparing the first release
candidate now and will share it for the community-wide review today.
Best,
Yang
Márton Balassi 于2022年5月18日周三 00:29写道:
> Thanks Gyula and Yang. Awesome
+1 (binding)
And thanks Xintong for driving this work.
Best,
Yang
Jingsong Li 于2022年5月17日周二 17:00写道:
> Thank Xintong for driving this work.
>
> +1
>
> Best,
> Jingsong
>
> On Tue, May 17, 2022 at 4:49 PM Martijn Visser
> wrote:
>
> > +1 (binding)
> >
> > On Tue, 17 May 2022 at 10:38, Yu Li
target day for the first release candidate is next Monday.
>
> The release managers for this release will be Yang Wang and myself.
>
> Cheers,
> Gyula
>
> On Wed, Apr 27, 2022 at 11:28 AM Yang Wang wrote:
>
>> Thanks @Chesnay Schepler for pointing out this.
>>
&
> Yun Gao 于2022年5月7日周六 10:44写道:
> > > > > >
> > > > > > > Congratulations Yang!
> > > > > > >
> > > > > > > Best,
> > > > > > > Yun Gao
> > > > >
I assume this is the responsibility of job result store[1]. However, it
seems that it does not work as expected.
[1].
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=195726435
Best,
Yang
Gyula Fóra 于2022年5月11日周三 12:55写道:
> Sorry I messed up the email, I meant false .
>
> So wh
information will be shared among all jobs in a
> cluster/session. The JM will sync blocklist information with RM.
>
> @Martijn
>
> > I agree with Yang Wang on this.
> As Zhu Zhu and I mentioned above, we think the MARK_BLOCKLISTED(Just limits
> the load of the node and does no
Congratulations!
Thanks Yun Gao, Till and Joe for driving this release and everyone who made
this release happen.
Best,
Yang
Jingsong Li 于2022年5月5日周四 16:04写道:
> Cheers! Congratulations!
>
> Thank you very much! And thank all who contributed to this release.
>
> Best,
> Jingsong
>
> On Thu, M
+1 (binding)
Best,
Yang
Danny Cranmer 于2022年5月4日周三 20:54写道:
> +1 (binding)
>
> Thanks,
> Danny
>
> On Wed, May 4, 2022 at 1:34 PM Gyula Fóra wrote:
>
> > +1
> >
> > Gyula
> >
> > On Wed, May 4, 2022 at 2:32 PM Jassat, Usamah >
> > wrote:
> >
> > > Hi everyone,
> > >
> > > Thanks for the feedb
Yang Wang created FLINK-27491:
-
Summary: Support env replacement in flink-kubernetes-operator CR
Key: FLINK-27491
URL: https://issues.apache.org/jira/browse/FLINK-27491
Project: Flink
Issue Type
The flink-kubernetes-operator project is only published
via apache/flink-kubernetes-operator on docker hub and github packages.
We do not find the obvious advantages by using docker hub official images.
Best,
Yang
Xintong Song 于2022年4月28日周四 19:27写道:
> I agree with you that doing QA for the imag
Thanks Lijie and Zhu for creating the proposal.
I want to share some thoughts about Flink cluster operations.
In the production environment, the SRE(aka Site Reliability Engineer)
already has many tools to detect the unstable nodes, which could take the
system logs/metrics into consideration.
The
Thanks @Chesnay Schepler for pointing out this.
The only public interface the flink-kubernetes-operator provides is the
CRD[1]. We are trying to stabilize the CRD from v1beta1.
If more fields are introduced to support new features(e.g. standalone mode,
SQL jobs), they should have the default valu
Yang Wang created FLINK-27422:
-
Summary: Do not create temporary pod template files for JobManager
and TaskManager if not configured explicitly
Key: FLINK-27422
URL: https://issues.apache.org/jira/browse/FLINK-27422
Thanks for creating the FLIP-223 and starting the discussion.
I have some quick questions.
# The TaskManager replicas
The TaskManager replicas need to be configured both for standalone session
and application. Because it could not be calculated if the parallelism is
set via java codes.
# How
1 - 100 of 448 matches
Mail list logo