Re: [VOTE] Release 1.11.2, release candidate #1

2020-09-14 Thread Yang Wang
+1 (non-binding) * Build from source and check the signature, checksum * Check the K8s notice file is correct since the pom has changed * Run standalone session/application clusters on K8s * Run native Flink session/application clusters on K8s Best, Yang Dian Fu 于2020年9月15日周二 上午10:11写道: > +1

[DISCUSS] FLIP-144: Native Kubernetes HA for Flink

2020-09-14 Thread Yang Wang
Hi devs and users, I would like to start the discussion about FLIP-144[1], which will introduce a new native high availability service for Kubernetes. Currently, Flink has provided Zookeeper HA service and been widely used in production environments. It could be integrated in standalone

Re: [DISCUSS] FLIP-36 - Support Interactive Programming in Flink Table API

2020-09-14 Thread Xuannan Su
Hi Aljoscha, Thanks for your comment. I agree that we should not introduce tight coupling with PipelineExecutor to the execution environment. With that in mind, to distinguish the per-job and session mode, we can introduce a new method, naming isPerJobModeExecutor, in the

Re: [VOTE] Release 1.11.2, release candidate #1

2020-09-14 Thread Zhu Zhu
Thank you all for helping to test and verify the release! The vote has lasted for more than 72 hours and it already has enough approvals. I will finalize the vote result soon in a separate email. Thanks, Zhu Zhijiang 于2020年9月15日周二 下午12:12写道: > +1 (binding) > > - checked the checksums and GPG

Re: [ANNOUNCE] New Apache Flink Committer - Arvid Heise

2020-09-14 Thread Yangze Guo
Congrats! Arvid. Best, Yangze Guo On Tue, Sep 15, 2020 at 1:31 PM Congxian Qiu wrote: > > Congratulations Arvid > > Best, > Congxian > > > Benchao Li 于2020年9月15日周二 下午12:52写道: > > > Congratulations! > > > > Zhu Zhu 于2020年9月15日周二 上午11:05写道: > > > > > Congratulations, Arvid! > > > > > > Thanks,

Re: [VOTE] Release 1.11.2, release candidate #1

2020-09-14 Thread Zhijiang
+1 (binding) - checked the checksums and GPG files - verified that the source archives do not contains any binaries - checked that all POM files point to the same version - reviewed the web site PR https://github.com/apache/flink-web/pull/377 - checked the release note Best, Zhijiang

[RESULT][VOTE] Release 1.11.2, release candidate #1

2020-09-14 Thread Zhu Zhu
Hi everyone, I'm happy to announce that we have unanimously approved this release. There are 8 approving votes, 3 of which are binding: * Robert Metzger (binding) * Dian Fu (binding) * Zhijiang (binding) * David Anderson * Congxian Qiu * Fabian Paul * Xingbo Huang * Yang Wang There are no

Re: [ANNOUNCE] New Apache Flink Committer - Arvid Heise

2020-09-14 Thread Congxian Qiu
Congratulations Arvid Best, Congxian Benchao Li 于2020年9月15日周二 下午12:52写道: > Congratulations! > > Zhu Zhu 于2020年9月15日周二 上午11:05写道: > > > Congratulations, Arvid! > > > > Thanks, > > Zhu > > > > jincheng sun 于2020年9月15日周二 上午10:55写道: > > > > > Congratulations and welcome, Arvid ! > > > > > >

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

2020-09-14 Thread Guowei Ma
## Concurrent checkpoints AFAIK the committer would not see the file-1-2 when ck1 happens in the ExactlyOnce mode. ## Committable bookkeeping and combining I agree with you that the "CombineGlobalCommitter" would work. But we put more optimization logic in the committer, which will make the

Re: [ANNOUNCE] New Apache Flink Committer - Arvid Heise

2020-09-14 Thread Benchao Li
Congratulations! Zhu Zhu 于2020年9月15日周二 上午11:05写道: > Congratulations, Arvid! > > Thanks, > Zhu > > jincheng sun 于2020年9月15日周二 上午10:55写道: > > > Congratulations and welcome, Arvid ! > > > > Best, > > Jincheng > > > > > > Dian Fu 于2020年9月15日周二 上午10:53写道: > > > > > Congratulations! > > > > > >

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

2020-09-14 Thread Guowei Ma
Hi, aljoscha >I don't understand why we need the "Drain and Snapshot" section. It >seems to be some details about stop-with-savepoint and drain, and the >relation to BATCH execution but I don't know if it is needed to >understand the rest of the document. I'm happy to be wrong here, though, >if

Re: [ANNOUNCE] New Apache Flink Committer - Niels Basjes

2020-09-14 Thread Congxian Qiu
Congratulations Best, Congxian Yang Wang 于2020年9月15日周二 上午10:50写道: > Congratulations! > > Best, > Yang > > Xintong Song 于2020年9月15日周二 上午10:41写道: > > > Congrats! > > > > Thank you~ > > > > Xintong Song > > > > > > > > On Tue, Sep 15, 2020 at 10:40 AM Zhijiang > > wrote: > > > > > Congrats,

[jira] [Created] (FLINK-19229) Support ValueState and Python UDAF on blink stream planner

2020-09-14 Thread Wei Zhong (Jira)
Wei Zhong created FLINK-19229: - Summary: Support ValueState and Python UDAF on blink stream planner Key: FLINK-19229 URL: https://issues.apache.org/jira/browse/FLINK-19229 Project: Flink Issue

[jira] [Created] (FLINK-19234) Support FILTER KeyWord for Python UDAF

2020-09-14 Thread Wei Zhong (Jira)
Wei Zhong created FLINK-19234: - Summary: Support FILTER KeyWord for Python UDAF Key: FLINK-19234 URL: https://issues.apache.org/jira/browse/FLINK-19234 Project: Flink Issue Type: Sub-task

[jira] [Created] (FLINK-19235) Support mixed use with built-in aggs for Python UDAF

2020-09-14 Thread Wei Zhong (Jira)
Wei Zhong created FLINK-19235: - Summary: Support mixed use with built-in aggs for Python UDAF Key: FLINK-19235 URL: https://issues.apache.org/jira/browse/FLINK-19235 Project: Flink Issue Type:

[jira] [Created] (FLINK-19233) Support DISTINCT KeyWord for Python UDAF

2020-09-14 Thread Wei Zhong (Jira)
Wei Zhong created FLINK-19233: - Summary: Support DISTINCT KeyWord for Python UDAF Key: FLINK-19233 URL: https://issues.apache.org/jira/browse/FLINK-19233 Project: Flink Issue Type: Sub-task

Re: [VOTE] Release 1.11.2, release candidate #1

2020-09-14 Thread Dian Fu
+1 (binding) - checked the signature and checksum - reviewed the web-site PR and it looks good to me - checked the diff for dependencies changes since 1.11.1: https://github.com/apache/flink/compare/release-1.11.1..release-1.11.2-rc1

[jira] [Created] (FLINK-19231) Support ListState and ListView for Python UDAF

2020-09-14 Thread Wei Zhong (Jira)
Wei Zhong created FLINK-19231: - Summary: Support ListState and ListView for Python UDAF Key: FLINK-19231 URL: https://issues.apache.org/jira/browse/FLINK-19231 Project: Flink Issue Type:

[jira] [Created] (FLINK-19232) Support MapState and MapView for Python UDAF

2020-09-14 Thread Wei Zhong (Jira)
Wei Zhong created FLINK-19232: - Summary: Support MapState and MapView for Python UDAF Key: FLINK-19232 URL: https://issues.apache.org/jira/browse/FLINK-19232 Project: Flink Issue Type: Sub-task

[jira] [Created] (FLINK-19230) Support Python UDAF on blink batch planner

2020-09-14 Thread Wei Zhong (Jira)
Wei Zhong created FLINK-19230: - Summary: Support Python UDAF on blink batch planner Key: FLINK-19230 URL: https://issues.apache.org/jira/browse/FLINK-19230 Project: Flink Issue Type: Sub-task

[jira] [Created] (FLINK-19236) Optimize the performance of Python UDAF

2020-09-14 Thread Wei Zhong (Jira)
Wei Zhong created FLINK-19236: - Summary: Optimize the performance of Python UDAF Key: FLINK-19236 URL: https://issues.apache.org/jira/browse/FLINK-19236 Project: Flink Issue Type: Sub-task

Re: [DISCUSS] FLIP-107: Reading table columns from different parts of source records

2020-09-14 Thread Leonard Xu
Hi, Timo Thanks for your explanation, it makes sense to me. Best, Leonard >> Hi, Timo >> Thanks for the update >> I have a minor suggestion about the debezium metadata key, >> Could we use the original debezium key rather than import new key? >> debezium-json.schema

Re: [VOTE] FLIP-107: Handling of metadata in SQL connectors

2020-09-14 Thread Leonard Xu
+1(non-binding) Leonard > 在 2020年9月12日,21:46,Danny Chan 写道: > > +1, non-binding ~ > > Konstantin Knauf 于2020年9月11日 周五上午2:04写道: > >> +1 (binding) >> >> >> >> On Thu, Sep 10, 2020 at 4:29 PM Dawid Wysakowicz >> >> wrote: >> >> >> >>> +1 (binding) >> >>> >> >>> On 10/09/2020 14:03,

[ANNOUNCE] New Apache Flink Committer - Arvid Heise

2020-09-14 Thread Zhijiang
Hi all, On behalf of the PMC, I’m very happy to announce Arvid Heise as a new Flink committer. Arvid has been an active community member for more than a year, with 138 contributions including 116 commits, reviewed many PRs with good quality comments. He is mainly working on the runtime scope,

Re: [ANNOUNCE] New Apache Flink Committer - Niels Basjes

2020-09-14 Thread Xintong Song
Congrats! Thank you~ Xintong Song On Tue, Sep 15, 2020 at 10:40 AM Zhijiang wrote: > Congrats, Niels! > > Best, > Zhijiang > > > -- > From:Darion Yaphet > Send Time:2020年9月15日(星期二) 10:02 > To:dev > Subject:Re: [ANNOUNCE] New

Re:[ANNOUNCE] New Apache Flink Committer - Arvid Heise

2020-09-14 Thread Matt Wang
Congrats! -- Best, Matt Wang On 09/15/2020 10:38,Zhijiang wrote: Hi all, On behalf of the PMC, I’m very happy to announce Arvid Heise as a new Flink committer. Arvid has been an active community member for more than a year, with 138 contributions including 116 commits, reviewed many PRs

Re: [ANNOUNCE] New Apache Flink Committer - Arvid Heise

2020-09-14 Thread Xintong Song
Congratulations Arvid, welcome abord~! Thank you~ Xintong Song On Tue, Sep 15, 2020 at 10:38 AM Zhijiang wrote: > Hi all, > > On behalf of the PMC, I’m very happy to announce Arvid Heise as a new > Flink committer. > > Arvid has been an active community member for more than a year, with

Re: [ANNOUNCE] New Apache Flink Committer - Niels Basjes

2020-09-14 Thread Zhijiang
Congrats, Niels! Best, Zhijiang -- From:Darion Yaphet Send Time:2020年9月15日(星期二) 10:02 To:dev Subject:Re: [ANNOUNCE] New Apache Flink Committer - Niels Basjes Congratulations! 刘建刚 于2020年9月15日周二 上午9:53写道: > Congratulations! > >

Re: [ANNOUNCE] New Apache Flink Committer - Niels Basjes

2020-09-14 Thread Yang Wang
Congratulations! Best, Yang Xintong Song 于2020年9月15日周二 上午10:41写道: > Congrats! > > Thank you~ > > Xintong Song > > > > On Tue, Sep 15, 2020 at 10:40 AM Zhijiang > wrote: > > > Congrats, Niels! > > > > Best, > > Zhijiang > > > > > >

Re: [ANNOUNCE] New Apache Flink Committer - Arvid Heise

2020-09-14 Thread Dian Fu
Congratulations! Regards, Dian > 在 2020年9月15日,上午10:41,Matt Wang 写道: > > Congrats! > > > -- > > Best, > Matt Wang > > > On 09/15/2020 10:38,Zhijiang wrote: > Hi all, > > On behalf of the PMC, I’m very happy to announce Arvid Heise as a new Flink > committer. > > Arvid has been an active

Re: [ANNOUNCE] New Apache Flink Committer - Arvid Heise

2020-09-14 Thread jincheng sun
Congratulations and welcome, Arvid ! Best, Jincheng Dian Fu 于2020年9月15日周二 上午10:53写道: > Congratulations! > > Regards, > Dian > > > 在 2020年9月15日,上午10:41,Matt Wang 写道: > > > > Congrats! > > > > > > -- > > > > Best, > > Matt Wang > > > > > > On 09/15/2020 10:38,Zhijiang wrote: > > Hi all, > > >

Re: [ANNOUNCE] New Apache Flink Committer - Arvid Heise

2020-09-14 Thread Zhu Zhu
Congratulations, Arvid! Thanks, Zhu jincheng sun 于2020年9月15日周二 上午10:55写道: > Congratulations and welcome, Arvid ! > > Best, > Jincheng > > > Dian Fu 于2020年9月15日周二 上午10:53写道: > > > Congratulations! > > > > Regards, > > Dian > > > > > 在 2020年9月15日,上午10:41,Matt Wang 写道: > > > > > > Congrats! > >

[jira] [Created] (FLINK-19237) LeaderChangeClusterComponentsTest.testReelectionOfJobMaster failed with "NoResourceAvailableException: Could not allocate the required slot within slot request timeout"

2020-09-14 Thread Dian Fu (Jira)
Dian Fu created FLINK-19237: --- Summary: LeaderChangeClusterComponentsTest.testReelectionOfJobMaster failed with "NoResourceAvailableException: Could not allocate the required slot within slot request timeout" Key: FLINK-19237

Re: [ANNOUNCE] New Apache Flink Committer - Niels Basjes

2020-09-14 Thread tison
Congrats! Best, tison. Aljoscha Krettek 于2020年9月14日周一 下午4:38写道: > Congratulations!  > > Aljoscha > > On 14.09.20 10:37, Robert Metzger wrote: > > Hi all, > > > > On behalf of the PMC, I’m very happy to announce Niels Basjes as a new > > Flink committer. > > > > Niels has been an active

Re: [VOTE] Release 1.11.2, release candidate #1

2020-09-14 Thread David Anderson
+1 Checks: - Verified that the fix for FLINK-19109 solves the problem I reported, running against the maven artifacts On Thu, Sep 10, 2020 at 9:04 AM Zhu Zhu wrote: > Hi everyone, > > Please review and vote on the release candidate #1 for the version 1.11.2, > as follows: > [ ] +1, Approve

Re: [ANNOUNCE] New Apache Flink Committer - Niels Basjes

2020-09-14 Thread Konstantin Knauf
Congratulations! On Mon, Sep 14, 2020 at 10:51 AM tison wrote: > Congrats! > > Best, > tison. > > > Aljoscha Krettek 于2020年9月14日周一 下午4:38写道: > > > Congratulations!  > > > > Aljoscha > > > > On 14.09.20 10:37, Robert Metzger wrote: > > > Hi all, > > > > > > On behalf of the PMC, I’m very happy

[jira] [Created] (FLINK-19213) Update the Chinese documentation

2020-09-14 Thread Dawid Wysakowicz (Jira)
Dawid Wysakowicz created FLINK-19213: Summary: Update the Chinese documentation Key: FLINK-19213 URL: https://issues.apache.org/jira/browse/FLINK-19213 Project: Flink Issue Type:

Re: [ANNOUNCE] New Apache Flink Committer - Niels Basjes

2020-09-14 Thread Aljoscha Krettek
Congratulations!  Aljoscha On 14.09.20 10:37, Robert Metzger wrote: Hi all, On behalf of the PMC, I’m very happy to announce Niels Basjes as a new Flink committer. Niels has been an active community member since the early days of Flink, with 19 commits dating back until 2015. Besides his

Re: [VOTE] FLIP-136: Improve interoperability between DataStream and Table API

2020-09-14 Thread Dawid Wysakowicz
+1 On 11/09/2020 16:19, Timo Walther wrote: > Hi all, > > after the discussion in [1], I would like to open a voting thread for > FLIP-136 [2] which covers different topic to improve the > back-and-forth communication between DataStream API and Table API. > > The vote will be open until September

[jira] [Created] (FLINK-19214) Update the flink-web

2020-09-14 Thread Dawid Wysakowicz (Jira)
Dawid Wysakowicz created FLINK-19214: Summary: Update the flink-web Key: FLINK-19214 URL: https://issues.apache.org/jira/browse/FLINK-19214 Project: Flink Issue Type: Sub-task

Re: [ANNOUNCE] New Apache Flink Committer - Niels Basjes

2020-09-14 Thread Matt Wang
Congratulations, Niels! -- Best, Matt Wang On 09/14/2020 17:02,Konstantin Knauf wrote: Congratulations! On Mon, Sep 14, 2020 at 10:51 AM tison wrote: Congrats! Best, tison. Aljoscha Krettek 于2020年9月14日周一 下午4:38写道: Congratulations!  Aljoscha On 14.09.20 10:37, Robert Metzger wrote:

Re: [VOTE] Release 1.11.2, release candidate #1

2020-09-14 Thread Robert Metzger
Thanks a lot for putting a release candidate together! +1 Checks: - Manually checked the git diff https://github.com/apache/flink/compare/release-1.11.1..release-1.11.2-rc1 - in flink-kubernetes, the shading configuration got changed, but the NOTICE file is correct (checked the shade plugin

Re: [DISCUSS] Deprecate and remove UnionList OperatorState

2020-09-14 Thread Alexey Trenikhun
-1 We use union state to generate sequences, each operator generates offset0 + number-of-tasks - task-index + task-specific-counter * number-of-tasks (e.g. for 2 instances of operator -one instance produce even number, another odd). Last generated sequence number is stored union list state,

[ANNOUNCE] New Apache Flink Committer - Niels Basjes

2020-09-14 Thread Robert Metzger
Hi all, On behalf of the PMC, I’m very happy to announce Niels Basjes as a new Flink committer. Niels has been an active community member since the early days of Flink, with 19 commits dating back until 2015. Besides his work on the code, he has been driving initiatives on dev@ list, supporting

Re: [ANNOUNCE] New Apache Flink Committer - Niels Basjes

2020-09-14 Thread Dian Fu
Congratulations! Regards, Dian > 在 2020年9月14日,下午7:45,Zhu Zhu 写道: > > Congratulations! > > Thanks, > Zhu > > Matt Wang 于2020年9月14日周一 下午5:22写道: > >> Congratulations, Niels! >> >> >> -- >> >> Best, >> Matt Wang >> >> >> On 09/14/2020 17:02,Konstantin Knauf wrote: >> Congratulations! >>

Re: [ANNOUNCE] New Apache Flink Committer - Niels Basjes

2020-09-14 Thread Xingbo Huang
Congratulations! Best, Xingbo Dian Fu 于2020年9月14日周一 下午8:06写道: > Congratulations! > > Regards, > Dian > > > 在 2020年9月14日,下午7:45,Zhu Zhu 写道: > > > > Congratulations! > > > > Thanks, > > Zhu > > > > Matt Wang 于2020年9月14日周一 下午5:22写道: > > > >> Congratulations, Niels! > >> > >> > >> -- > >> > >>

[jira] [Created] (FLINK-19219) Run JobManager initialization in a separate thread, to make it cancellable

2020-09-14 Thread Robert Metzger (Jira)
Robert Metzger created FLINK-19219: -- Summary: Run JobManager initialization in a separate thread, to make it cancellable Key: FLINK-19219 URL: https://issues.apache.org/jira/browse/FLINK-19219

Re: [VOTE] Release 1.11.2, release candidate #1

2020-09-14 Thread Zhu Zhu
Thank you all for the verification and voting! @Robert I tried building from sources again and the log/ dir is there. I will keep watching if anyone else encounters this problem. Thanks, Zhu Congxian Qiu 于2020年9月14日周一 下午7:47写道: > +1 (no-binding) > > checked > - sha verified ok > - gpg

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

2020-09-14 Thread Piotr Nowojski
Hi, I've just briefly skimmed over the proposed interfaces. I would suggest one addition to the Writer interface (as I understand this is the runtime interface in this proposal?): add some availability method, to avoid, if possible, blocking calls on the sink. We already have similar availability

Re: [VOTE] Release 1.11.2, release candidate #1

2020-09-14 Thread Fabian Paul
+1 (non-binding) Checks: - Verified signature - Built from source (Java8) - Ran custom jobs on Kubernetes Regards, Fabian

Re: [VOTE] Release 1.11.2, release candidate #1

2020-09-14 Thread Xingbo Huang
+1 (non-binding) Checks: - Pip install PyFlink from wheel packages with Python 3.5,3.6 and 3.7 in Mac and Linux. - Test Python UDF/Pandas UDF - Test from_pandas/to_pandas Best, Xingbo Fabian Paul 于2020年9月14日周一 下午8:46写道: > +1 (non-binding) > > Checks: > > - Verified signature > - Built from

Java and Scala code format

2020-09-14 Thread Darion Yaphet
Hi team: I have an idea about code format. This is more readable and good for development. But it also may bring a lot of changes. Could you tell me what you think ? thanks ~ -- long is the way and hard that out of Hell leads up to light

Re: [VOTE] FLIP-140: Introduce bounded style execution for keyed streams

2020-09-14 Thread Seth Wiesman
+1 (binding) Seth On Thu, Sep 10, 2020 at 9:13 AM Aljoscha Krettek wrote: > +1 (binding) > > Aljoscha >

Re: [VOTE] Release 1.11.2, release candidate #1

2020-09-14 Thread Congxian Qiu
+1 (no-binding) checked - sha verified ok - gpg verifed ok - build from source, ok - check license ok, use the diff generated here[1] - run some demo locally, ok [1] https://github.com/apache/flink/compare/release-1.11.1..release-1.11.2-rc1 Best, Congxian David Anderson 于2020年9月14日周一

Re: [ANNOUNCE] New Apache Flink Committer - Niels Basjes

2020-09-14 Thread Zhu Zhu
Congratulations! Thanks, Zhu Matt Wang 于2020年9月14日周一 下午5:22写道: > Congratulations, Niels! > > > -- > > Best, > Matt Wang > > > On 09/14/2020 17:02,Konstantin Knauf wrote: > Congratulations! > > On Mon, Sep 14, 2020 at 10:51 AM tison wrote: > > Congrats! > > Best, > tison. > > > Aljoscha

[VOTE] FLIP-134: Batch execution for the DataStream API

2020-09-14 Thread Aljoscha Krettek
Hi all, After the discussion in [1], I would like to open a voting thread for FLIP-134 (https://s.apache.org/FLIP-134) [2] which discusses a new BATCH execution mode for the DataStream API. The vote will be open until September 17, unless there is an objection or not enough votes.

[jira] [Created] (FLINK-19216) Reduce the duplicate argument check

2020-09-14 Thread darion yaphet (Jira)
darion yaphet created FLINK-19216: - Summary: Reduce the duplicate argument check Key: FLINK-19216 URL: https://issues.apache.org/jira/browse/FLINK-19216 Project: Flink Issue Type:

[jira] [Created] (FLINK-19218) Remove inconsistent host logic for LocalFileSystem

2020-09-14 Thread Stephan Ewen (Jira)
Stephan Ewen created FLINK-19218: Summary: Remove inconsistent host logic for LocalFileSystem Key: FLINK-19218 URL: https://issues.apache.org/jira/browse/FLINK-19218 Project: Flink Issue

[jira] [Created] (FLINK-19215) "Resuming Savepoint (rocks, scale down, rocks timers) end-to-end test" failed with "Dispatcher REST endpoint has not started within a timeout of 20 sec"

2020-09-14 Thread Dian Fu (Jira)
Dian Fu created FLINK-19215: --- Summary: "Resuming Savepoint (rocks, scale down, rocks timers) end-to-end test" failed with "Dispatcher REST endpoint has not started within a timeout of 20 sec" Key: FLINK-19215 URL:

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

2020-09-14 Thread Aljoscha Krettek
On 14.09.20 01:23, Steven Wu wrote: ## Writer interface For the Writer interface, should we add "*prepareSnapshot"* before the checkpoint barrier emitted downstream? IcebergWriter would need it. Or would the framework call "*flush*" before the barrier emitted downstream? that guarantee would

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

2020-09-14 Thread Aljoscha Krettek
I thought about this some more. One of the important parts of the Iceberg sink is to know whether we have already committed some DataFiles. Currently, this is implemented by writing a (JobId, MaxCheckpointId) tuple to the Iceberg table when committing. When restoring from a failure we check

[jira] [Created] (FLINK-19217) Functions repeated extend Serializable interface

2020-09-14 Thread darion yaphet (Jira)
darion yaphet created FLINK-19217: - Summary: Functions repeated extend Serializable interface Key: FLINK-19217 URL: https://issues.apache.org/jira/browse/FLINK-19217 Project: Flink Issue

Re: [ANNOUNCE] New Apache Flink Committer - Niels Basjes

2020-09-14 Thread Benchao Li
Congratulations! Xingbo Huang 于2020年9月14日周一 下午9:36写道: > Congratulations! > > Best, > Xingbo > > Dian Fu 于2020年9月14日周一 下午8:06写道: > > > Congratulations! > > > > Regards, > > Dian > > > > > 在 2020年9月14日,下午7:45,Zhu Zhu 写道: > > > > > > Congratulations! > > > > > > Thanks, > > > Zhu > > > > > >

Re: Java and Scala code format

2020-09-14 Thread Aljoscha Krettek
For some reference, this is a Jira issue that was created by the OP about using Scalafmt for the Flink code base: https://issues.apache.org/jira/browse/FLINK-19159. Best, Aljoscha

Re: Java and Scala code format

2020-09-14 Thread Darion Yaphet
Yes, this is my submission Aljoscha Krettek 于2020年9月14日周一 下午10:58写道: > For some reference, this is a Jira issue that was created by the OP > about using Scalafmt for the Flink code base: > https://issues.apache.org/jira/browse/FLINK-19159. > > Best, > Aljoscha > -- long is the way and hard

[jira] [Created] (FLINK-19220) Add a way to close internal resources

2020-09-14 Thread Igal Shilman (Jira)
Igal Shilman created FLINK-19220: Summary: Add a way to close internal resources Key: FLINK-19220 URL: https://issues.apache.org/jira/browse/FLINK-19220 Project: Flink Issue Type:

[VOTE] FLIP-142: Disentangle StateBackends from Checkpointing

2020-09-14 Thread Seth Wiesman
Hi all, After the discussion in [1], I would like to open a voting thread for FLIP-142 [2] which discusses disentangling state backends from checkpointing. The vote will be open until 16th September (72h), unless there is an objection or not enough votes. Seth [1]

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

2020-09-14 Thread Dawid Wysakowicz
Hi all, > I would think that we only need flush() and the semantics are that it > prepares for a commit, so on a physical level it would be called from > "prepareSnapshotPreBarrier". Now that I'm thinking about it more I > think flush() should be renamed to something like "prepareCommit()".

[jira] [Created] (FLINK-19222) Elevate external SDKs

2020-09-14 Thread Seth Wiesman (Jira)
Seth Wiesman created FLINK-19222: Summary: Elevate external SDKs Key: FLINK-19222 URL: https://issues.apache.org/jira/browse/FLINK-19222 Project: Flink Issue Type: Improvement

[jira] [Created] (FLINK-19221) Exploit LocatableFileStatus from Hadoop

2020-09-14 Thread Stephan Ewen (Jira)
Stephan Ewen created FLINK-19221: Summary: Exploit LocatableFileStatus from Hadoop Key: FLINK-19221 URL: https://issues.apache.org/jira/browse/FLINK-19221 Project: Flink Issue Type:

Re: [ANNOUNCE] New Apache Flink Committer - Niels Basjes

2020-09-14 Thread David Anderson
Congratulations! --David On Mon, Sep 14, 2020 at 8:24 PM Arvid Heise wrote: > Congrats Niels! > > On Mon, Sep 14, 2020 at 4:04 PM Benchao Li wrote: > > > Congratulations! > > > > Xingbo Huang 于2020年9月14日周一 下午9:36写道: > > > > > Congratulations! > > > > > > Best, > > > Xingbo > > > > > > Dian

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

2020-09-14 Thread Guowei Ma
Hi all, Very thanks for the discussion and the valuable opinions! Currently there are several ongoing issues and we would like to show what we are thinking in the next few mails. It seems that the biggest issue now is about the topology of the sinks. Before deciding what the sink API would look

Re: [ANNOUNCE] New Apache Flink Committer - Niels Basjes

2020-09-14 Thread Arvid Heise
Congrats Niels! On Mon, Sep 14, 2020 at 4:04 PM Benchao Li wrote: > Congratulations! > > Xingbo Huang 于2020年9月14日周一 下午9:36写道: > > > Congratulations! > > > > Best, > > Xingbo > > > > Dian Fu 于2020年9月14日周一 下午8:06写道: > > > > > Congratulations! > > > > > > Regards, > > > Dian > > > > > > > 在

[jira] [Created] (FLINK-19223) Simplify Availability Future Model in Base Connector

2020-09-14 Thread Stephan Ewen (Jira)
Stephan Ewen created FLINK-19223: Summary: Simplify Availability Future Model in Base Connector Key: FLINK-19223 URL: https://issues.apache.org/jira/browse/FLINK-19223 Project: Flink Issue

Re: [ANNOUNCE] New Apache Flink Committer - Niels Basjes

2020-09-14 Thread Caito Scherr
Congratulations, Niels!! On Mon, Sep 14, 2020 at 1:37 AM Robert Metzger wrote: > Hi all, > > > > On behalf of the PMC, I’m very happy to announce Niels Basjes as a new > > Flink committer. > > > > Niels has been an active community member since the early days of Flink, > > with 19 commits

Re: [ANNOUNCE] New Apache Flink Committer - Niels Basjes

2020-09-14 Thread Marta Paes Moreira
Congrats, Niels! On Mon, Sep 14, 2020 at 8:56 PM David Anderson wrote: > Congratulations! > > --David > > On Mon, Sep 14, 2020 at 8:24 PM Arvid Heise wrote: > > > Congrats Niels! > > > > On Mon, Sep 14, 2020 at 4:04 PM Benchao Li wrote: > > > > > Congratulations! > > > > > > Xingbo Huang

[jira] [Created] (FLINK-19225) Improve code and logging in SourceReaderBase

2020-09-14 Thread Stephan Ewen (Jira)
Stephan Ewen created FLINK-19225: Summary: Improve code and logging in SourceReaderBase Key: FLINK-19225 URL: https://issues.apache.org/jira/browse/FLINK-19225 Project: Flink Issue Type:

Re: [ANNOUNCE] New Apache Flink Committer - Niels Basjes

2020-09-14 Thread Peter Huang
Congratulations, Niels!! On Mon, Sep 14, 2020 at 2:28 PM Caito Scherr wrote: > Congratulations, Niels!! > > On Mon, Sep 14, 2020 at 1:37 AM Robert Metzger > wrote: > > > Hi all, > > > > > > > > On behalf of the PMC, I’m very happy to announce Niels Basjes as a new > > > > Flink committer. > >

[jira] [Created] (FLINK-19224) Provide an easy way to read window state

2020-09-14 Thread Seth Wiesman (Jira)
Seth Wiesman created FLINK-19224: Summary: Provide an easy way to read window state Key: FLINK-19224 URL: https://issues.apache.org/jira/browse/FLINK-19224 Project: Flink Issue Type:

[jira] [Created] (FLINK-19227) The catalog is still created after opening failed in catalog registering

2020-09-14 Thread Jingsong Lee (Jira)
Jingsong Lee created FLINK-19227: Summary: The catalog is still created after opening failed in catalog registering Key: FLINK-19227 URL: https://issues.apache.org/jira/browse/FLINK-19227 Project:

Re: [ANNOUNCE] New Apache Flink Committer - Niels Basjes

2020-09-14 Thread Darion Yaphet
Congratulations! 刘建刚 于2020年9月15日周二 上午9:53写道: > Congratulations! > > Best, > liujiangang > > Danny Chan 于2020年9月15日周二 上午9:44写道: > > > Congratulations!  > > > > Best, > > Danny Chan > > 在 2020年9月15日 +0800 AM9:31,dev@flink.apache.org,写道: > > > > > > Congratulations!  > > > -- long is the

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

2020-09-14 Thread Steven Wu
## concurrent checkpoints @Aljoscha Krettek regarding the concurrent checkpoints, let me illustrate with a simple DAG below. [image: image.png] Let's assume each writer emits one file per checkpoint cycle and *writer-2 is slow*. Now let's look at what the global committer receives timeline:

[jira] [Created] (FLINK-19226) [Kinesis] [EFO] Connector reaches default max attempts for describeStream and describeStreamConsumer when parallelism is high

2020-09-14 Thread Hong Liang Teoh (Jira)
Hong Liang Teoh created FLINK-19226: --- Summary: [Kinesis] [EFO] Connector reaches default max attempts for describeStream and describeStreamConsumer when parallelism is high Key: FLINK-19226 URL:

Re: [ANNOUNCE] New Apache Flink Committer - Niels Basjes

2020-09-14 Thread Leonard Xu
Congrats, Niels! Best, Leonard > 在 2020年9月15日,05:29,Peter Huang 写道: > > Congratulations, Niels!! > > > On Mon, Sep 14, 2020 at 2:28 PM Caito Scherr wrote: > >> Congratulations, Niels!! >> >> On Mon, Sep 14, 2020 at 1:37 AM Robert Metzger >> wrote: >> >>> Hi all, >>> >>> >>> >>> On

Re: [ANNOUNCE] New Apache Flink Committer - Niels Basjes

2020-09-14 Thread Danny Chan
Congratulations!  Best, Danny Chan 在 2020年9月15日 +0800 AM9:31,dev@flink.apache.org,写道: > > Congratulations! 

Re: [ANNOUNCE] New Apache Flink Committer - Niels Basjes

2020-09-14 Thread 刘建刚
Congratulations! Best, liujiangang Danny Chan 于2020年9月15日周二 上午9:44写道: > Congratulations!  > > Best, > Danny Chan > 在 2020年9月15日 +0800 AM9:31,dev@flink.apache.org,写道: > > > > Congratulations!  >

[jira] [Created] (FLINK-19228) Avoid accessing FileSystem in client for file system connector

2020-09-14 Thread Jingsong Lee (Jira)
Jingsong Lee created FLINK-19228: Summary: Avoid accessing FileSystem in client for file system connector Key: FLINK-19228 URL: https://issues.apache.org/jira/browse/FLINK-19228 Project: Flink