[VOTE] Apache Flink Stateful Functions 2.2.0, release candidate #1

2020-09-20 Thread Tzu-Li (Gordon) Tai
Hi everyone, Please review and vote on the release candidate #1 for the version 2.2.0 of Apache Flink Stateful Functions, as follows: [ ] +1, Approve the release [ ] -1, Do not approve the release (please provide specific comments) ***Testing Guideline*** You can find here [1] a page in the proj

Re: Can you unify the language ?

2020-09-20 Thread Timo Walther
Hi, you are right. Having two languages in the code base doesn't make our lives easier. But Flink is a big project with a long history, multiple design shifts, and many contributors. It is naturally that the bigger a code base gets, the messier it looks like. It must be a continuous effort to

[jira] [Created] (FLINK-19302) Flushing of BoundedBlockingResultPartition should finish current BufferBuilder

2020-09-20 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-19302: --- Summary: Flushing of BoundedBlockingResultPartition should finish current BufferBuilder Key: FLINK-19302 URL: https://issues.apache.org/jira/browse/FLINK-19302 Project:

Re: [ANNOUNCE] Apache Flink 1.11.2 released

2020-09-20 Thread Yu Li
Thanks Zhu Zhu for being our release manager and everyone else who made the release possible! Best Regards, Yu On Thu, 17 Sep 2020 at 13:29, Zhu Zhu wrote: > The Apache Flink community is very happy to announce the release of Apache > Flink 1.11.2, which is the second bugfix release for the Ap

Re: [DISCUSS][Release 1.12] Stale blockers and build instabilities

2020-09-20 Thread Robert Metzger
Hi all, An update on the release status: 1. We have 35 days = *5 weeks left until feature freeze* 2. There are currently 2 blockers for Flink , all making progress 3. We have 72 test instabilities

[jira] [Created] (FLINK-19301) Improve the package structure of Python DataStream API

2020-09-20 Thread Dian Fu (Jira)
Dian Fu created FLINK-19301: --- Summary: Improve the package structure of Python DataStream API Key: FLINK-19301 URL: https://issues.apache.org/jira/browse/FLINK-19301 Project: Flink Issue Type: Imp

[jira] [Created] (FLINK-19300) Timer loss after restoring from savepoint

2020-09-20 Thread Xiang Gao (Jira)
Xiang Gao created FLINK-19300: - Summary: Timer loss after restoring from savepoint Key: FLINK-19300 URL: https://issues.apache.org/jira/browse/FLINK-19300 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-19299) NettyShuffleEnvironmentBuilder#setBufferSize does not take effect

2020-09-20 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-19299: --- Summary: NettyShuffleEnvironmentBuilder#setBufferSize does not take effect Key: FLINK-19299 URL: https://issues.apache.org/jira/browse/FLINK-19299 Project: Flink

[RESULT][VOTE] FLIP-36 - Support Interactive Programming in Flink Table API

2020-09-20 Thread Xuannan Su
Hi all, The voting time for FLIP-36 has passed. I'm closing the vote now. There were 3 binding votes: - Aljoscha (binding) - Timo (binding) - Becket (binding) There were no disapproving votes. Thus, FLIP-36 has been accepted. Thanks everyone for joining the discussion and giving feedback! Bes

[jira] [Created] (FLINK-19298) Maven enforce goal dependency-convergence failed on flink-json

2020-09-20 Thread Jark Wu (Jira)
Jark Wu created FLINK-19298: --- Summary: Maven enforce goal dependency-convergence failed on flink-json Key: FLINK-19298 URL: https://issues.apache.org/jira/browse/FLINK-19298 Project: Flink Issue T

[jira] [Created] (FLINK-19297) Make ResultPartitionWriter record-oriented

2020-09-20 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-19297: --- Summary: Make ResultPartitionWriter record-oriented Key: FLINK-19297 URL: https://issues.apache.org/jira/browse/FLINK-19297 Project: Flink Issue Type: Improvem

[VOTE] FLIP-33: Standardize connector metrics

2020-09-20 Thread Becket Qin
Hi all, I would like to start the voting thread for FLIP-33 which proposes to standardize the metrics of Flink connectors. In short, we would like to provide a convention and guidance of Flink connector metrics. It will help simplify the monitoring and alerting on Flink jobs. The FLIP link is fol

Re: [DISCUSS] Forwarding Kafka's metrics groups

2020-09-20 Thread Becket Qin
I agree that we can remove the double registered metrics. Maybe we can make this part of the effort to change the connectors to conform to FLIP-33. What do you think? Thanks, Jiangjie (Becket) Qin On Wed, Sep 16, 2020 at 3:58 PM Dawid Wysakowicz wrote: > Hi, > > When answering a user questi

Re: Timed out patterns handling using MATCH_RECOGNIZE

2020-09-20 Thread Jark Wu
Hi Kosma, Thanks for the proposal. I like it and we also have supported similar syntax in our company. The problem is that Flink SQL leverages Calcite as the query parser, so if we want to support this syntax, we may have to push this syntax back to the Calcite community. Besides, the SQL standard

Re: Re: [DISCUSS] Support source/sink parallelism config in Flink sql

2020-09-20 Thread Jark Wu
Since FLIP-95, the parallelism is decoupled from the runtime class (DataStream/SourceFunction), so we need to have an API to tell the planner what the parallelism of the source/sink is. This is indeed the purpose of a previous discussion: [DISCUSS] Introduce SupportsParallelismReport and SupportsS

Re: Re: [ANNOUNCE] New Apache Flink Committer - Godfrey He

2020-09-20 Thread godfrey he
Thanks everyone for the warm reception! Best, Godfrey Rui Li 于2020年9月18日周五 下午6:21写道: > Congrats Godfrey! Well deserved! > > On Fri, Sep 18, 2020 at 5:12 PM Yun Gao > wrote: > >> Congratulations Godfrey! >> >> Best, >> Yun >> >> >> >> --Original Mail -- >> Sende

Re: [DISCUSS] FLIP-143: Unified Sink API

2020-09-20 Thread Steven Wu
> I think Iceberg sink needs to do the dedup in the `commit` call. The `recoveredGlobalCommittables` is just for restoring the ids. @Guowei Ma It is undesirable to do the dedup check in the `commit` call, because it happens for each checkpoint cycle. We only need to do the de-dup check one time

[jira] [Created] (FLINK-19296) RetryingCallback is not aware of task cancaltion

2020-09-20 Thread Igal Shilman (Jira)
Igal Shilman created FLINK-19296: Summary: RetryingCallback is not aware of task cancaltion Key: FLINK-19296 URL: https://issues.apache.org/jira/browse/FLINK-19296 Project: Flink Issue Type:

Re: Re: [DISCUSS] Support source/sink parallelism config in Flink sql

2020-09-20 Thread 刘大龙
+1 > -原始邮件- > 发件人: "Benchao Li" > 发送时间: 2020-09-20 16:28:20 (星期日) > 收件人: dev > 抄送: > 主题: Re: [DISCUSS] Support source/sink parallelism config in Flink sql > > Hi admin, > > Thanks for bringing up this discussion. > IMHO, it's a valuable feature. We also added this feature for our int

[jira] [Created] (FLINK-19295) YARNSessionFIFOITCase.checkForProhibitedLogContents found a log with prohibited string

2020-09-20 Thread Dian Fu (Jira)
Dian Fu created FLINK-19295: --- Summary: YARNSessionFIFOITCase.checkForProhibitedLogContents found a log with prohibited string Key: FLINK-19295 URL: https://issues.apache.org/jira/browse/FLINK-19295 Project:

Re: [DISCUSS] FLIP-143: Unified Sink API

2020-09-20 Thread Guowei Ma
I would like to summarize the file type sink in the thread and their possible topologies. I also try to give pros and cons of every topology option. Correct me if I am wrong. ### FileSink Topology Option: TmpFileWriter + Committer. ### IceBerg Sink Topology Option1: `DataFileWriter` + `Gl

Re: [DISCUSS] FLIP-143: Unified Sink API

2020-09-20 Thread Guowei Ma
Hi, Stevn I want to make a clarification first, the following reply only considers the Iceberge sink, but does not consider other sinks. Before make decision we should consider all the sink.I would try to summary all the sink requirments in the next mail >> run global committer in jobmanager (e

Re: [DISCUSS] Support source/sink parallelism config in Flink sql

2020-09-20 Thread Benchao Li
Hi admin, Thanks for bringing up this discussion. IMHO, it's a valuable feature. We also added this feature for our internal SQL engine. And our way is very similar to your proposal. Regarding the implementation, there is one shorthand that we should modify each connector to support this property