Re: [DISCUSS] remove the default in-memory catalog from Flink table module

2019-11-19 Thread Jingsong Li
Hi Bowen: Thanks for you proposal. You mean even if there is no catalog, users can completely walk through the set of temporary objects and work well? - If it is, I am +1, in memory catalog actually is a temporary catalog, it can not persist, we can replace it with real temporary objects. - If

[jira] [Created] (FLINK-14871) Better formatter of toString method for StateTransition

2019-11-19 Thread chaiyongqiang (Jira)
chaiyongqiang created FLINK-14871: - Summary: Better formatter of toString method for StateTransition Key: FLINK-14871 URL: https://issues.apache.org/jira/browse/FLINK-14871 Project: Flink

[jira] [Created] (FLINK-14870) Introducing non-nullable assumption for task slot sharing group

2019-11-19 Thread Zhu Zhu (Jira)
Zhu Zhu created FLINK-14870: --- Summary: Introducing non-nullable assumption for task slot sharing group Key: FLINK-14870 URL: https://issues.apache.org/jira/browse/FLINK-14870 Project: Flink Issue

How to estimate the memory size of flink state

2019-11-19 Thread 刘建刚
We are using flink 1.6.2. For filesystem backend, we want to monitor the state size in memory. Once the state size becomes bigger, we can get noticed and take measures such as rescaling the job, or the job may fail because of the memory. We have tried to get the memory usage for the

[jira] [Created] (FLINK-14869) Force an extended resource to be null if it's value is 0

2019-11-19 Thread Zhu Zhu (Jira)
Zhu Zhu created FLINK-14869: --- Summary: Force an extended resource to be null if it's value is 0 Key: FLINK-14869 URL: https://issues.apache.org/jira/browse/FLINK-14869 Project: Flink Issue Type:

[jira] [Created] (FLINK-14868) Provides the ability for multiple sinks to write data serially

2019-11-19 Thread ShenDa (Jira)
ShenDa created FLINK-14868: -- Summary: Provides the ability for multiple sinks to write data serially Key: FLINK-14868 URL: https://issues.apache.org/jira/browse/FLINK-14868 Project: Flink Issue

Re: [DISCUSS] Support configure remote flink jar

2019-11-19 Thread tison
Thanks for your participation! @Yang: Great to hear. I'd like to know whether or not a remote flink jar path conflicts with FLINK-13938. IIRC FLINK-13938 auto excludes local flink jar from shipping which possibly not works for the remote one. @Thomas: It inspires a lot URL becomes the unified

[jira] [Created] (FLINK-14867) Move TextInputFormat & TextOutputFormat to flink-core

2019-11-19 Thread Zili Chen (Jira)
Zili Chen created FLINK-14867: - Summary: Move TextInputFormat & TextOutputFormat to flink-core Key: FLINK-14867 URL: https://issues.apache.org/jira/browse/FLINK-14867 Project: Flink Issue Type:

[jira] [Created] (FLINK-14866) A few documentation links are broken

2019-11-19 Thread Dian Fu (Jira)
Dian Fu created FLINK-14866: --- Summary: A few documentation links are broken Key: FLINK-14866 URL: https://issues.apache.org/jira/browse/FLINK-14866 Project: Flink Issue Type: Bug

Re: [DISCUSS] Primary keys in Table API

2019-11-19 Thread Jark Wu
Hi Dawid, Thanks for driving this. Primary key is a very important and useful feature to Flink Table API / SQL. I'm +1 to the general design. And have some thoughts as following: 1. > The design says Flink only support NOT ENFORCED mode. But the DDL and KeyConstraint#primaryKey(..) can pass in

Re: [VOTE] FLIP-83: Flink End-to-end Performance Testing Framework

2019-11-19 Thread aihua li
hi,Becket, Thanks for the comments! > 1. How do the testing records look like? The size and key distributions. The records looks like a long string,which default size is 1k. The key is randomly generated according to the specified range plus a fixed string to assure that the data is

[jira] [Created] (FLINK-14865) PyFlinkBlinkStreamUserDefinedFunctionTests#test_udf_in_join_condition_2 failed in travis

2019-11-19 Thread Dian Fu (Jira)
Dian Fu created FLINK-14865: --- Summary: PyFlinkBlinkStreamUserDefinedFunctionTests#test_udf_in_join_condition_2 failed in travis Key: FLINK-14865 URL: https://issues.apache.org/jira/browse/FLINK-14865

Re: [DISCUSS] Stateful Functions - Contribution Details

2019-11-19 Thread tison
Thanks for your summary Stephan. All entries make sense to me. Let's play statefun :-) Best, tison. Stephan Ewen 于2019年11月20日周三 上午12:53写道: > I am also fine with skipping a FLIP, if no one objects. > > The discussion seemed rather converged (or stalled). There was a concern > about the name,

[DISCUSS] remove the default in-memory catalog from Flink table module

2019-11-19 Thread Bowen Li
Hi all, I want to kick off a discussion of whether we should remove the default in-memory catalog from Flink table module. Background: Currently Flink always has a default in-memory catalog named "default_catalog" upon start. This behavior is added in 1.9.0 along with new Catalog APIs. Till 1.9,

[VOTE] Release flink-shaded 9.0, release candidate #1

2019-11-19 Thread Chesnay Schepler
Hi everyone, Please review and vote on the release candidate #1 for the version 9.0, as follows: [ ] +1, Approve the release [ ] -1, Do not approve the release (please provide specific comments) The complete staging area is available for your review, which includes: * JIRA release notes [1],

[jira] [Created] (FLINK-14864) Inconsistent REST Parameter Name: `programArg` vs `programArgs`

2019-11-19 Thread Kelly Smith (Jira)
Kelly Smith created FLINK-14864: --- Summary: Inconsistent REST Parameter Name: `programArg` vs `programArgs` Key: FLINK-14864 URL: https://issues.apache.org/jira/browse/FLINK-14864 Project: Flink

[jira] [Created] (FLINK-14863) remove default in-memory catalog from CatalogManager

2019-11-19 Thread Bowen Li (Jira)
Bowen Li created FLINK-14863: Summary: remove default in-memory catalog from CatalogManager Key: FLINK-14863 URL: https://issues.apache.org/jira/browse/FLINK-14863 Project: Flink Issue Type:

Re: [DISCUSSION] Kafka Metrics Reporter

2019-11-19 Thread Bowen Li
Hi, What still unclear to me so far is - As I don't see any yet., what would be the fundamental differences between this Kafka reporter and Flink’s existing Kafka producer? I’ve been thinking of Flink metrics for a while, and the “metric reporter” feels a bit redundant to me. As you may already

[jira] [Created] (FLINK-14862) Fuse initalizeState and open passes in StreamTask#invoke

2019-11-19 Thread Igal Shilman (Jira)
Igal Shilman created FLINK-14862: Summary: Fuse initalizeState and open passes in StreamTask#invoke Key: FLINK-14862 URL: https://issues.apache.org/jira/browse/FLINK-14862 Project: Flink

Re: [DISCUSS] Stateful Functions - Contribution Details

2019-11-19 Thread Stephan Ewen
I am also fine with skipping a FLIP, if no one objects. The discussion seemed rather converged (or stalled). There was a concern about the name, but in the absence of another candidate for the name, I would go ahead with the current one. For the other aspects, we seem to have converged in the

Re: [DISCUSS] Support configure remote flink jar

2019-11-19 Thread Stephan Ewen
Would that be a feature specific to Yarn? (and maybe standalone sessions) For containerized setups, and init container seems like a nice way to solve this. Also more flexible, when it comes to supporting authentication mechanisms for the target storage system, etc. On Tue, Nov 19, 2019 at 5:29

[ANNOUNCE] Feature freeze for Apache Flink 1.10.0 release

2019-11-19 Thread Yu Li
Hi devs, Per the feature discussions and progress updates for 1.10.0 [1] [2] [3], we hereby announce the official feature freeze for Flink 1.10.0 to be on December 8. A release feature branch for 1.10 will be cut following that date. We’re roughly two and a half weeks away from this date, but

[DISCUSS] Primary keys in Table API

2019-11-19 Thread Dawid Wysakowicz
Hi, I wanted to bring up the topic of primary key constraints that we could leverage for runtime optimizations. Please have a look at the proposal and I would especially draw your attention to the topic of nullability of columns that are part of a primary key. Some time in the future we will also

[jira] [Created] (FLINK-14861) parallelism.default in flink-conf.yaml do not work which is a bug imported by[FLINK-14745]

2019-11-19 Thread Leonard Xu (Jira)
Leonard Xu created FLINK-14861: -- Summary: parallelism.default in flink-conf.yaml do not work which is a bug imported by[FLINK-14745] Key: FLINK-14861 URL: https://issues.apache.org/jira/browse/FLINK-14861

Re: [DISCUSS] Stateful Functions - Contribution Details

2019-11-19 Thread Igal Shilman
Hi Robert, Your proposal skipping FLIP and the vote sounds reasonable to me. The project is currently built (with tests, shading, spotbugs etc') in around 2-3 minutes, but since it will reside in its own repository, it will not affect Flink build time. Thanks, Igal On Tue, Nov 19, 2019 at 3:36

Re: [DISCUSS] Stateful Functions - Contribution Details

2019-11-19 Thread Robert Metzger
+1 on what has been decided so far in this thread (including using the same ML, and sticking to the statefun name). I'm not 100% sure if we need a FLIP for this, as we have VOTEd already with a 2/3 majority on accepting this contribution, and there are no changes to the Flink codebase, or

[jira] [Created] (FLINK-14860) Provide a new InputSplitAssigner that consider page cache utilization ratio

2019-11-19 Thread Jingsong Lee (Jira)
Jingsong Lee created FLINK-14860: Summary: Provide a new InputSplitAssigner that consider page cache utilization ratio Key: FLINK-14860 URL: https://issues.apache.org/jira/browse/FLINK-14860 Project:

[jira] [Created] (FLINK-14859) Avoid leaking unassigned Slot in DefaultScheduler when Deployment is outdated

2019-11-19 Thread Gary Yao (Jira)
Gary Yao created FLINK-14859: Summary: Avoid leaking unassigned Slot in DefaultScheduler when Deployment is outdated Key: FLINK-14859 URL: https://issues.apache.org/jira/browse/FLINK-14859 Project: Flink

[jira] [Created] (FLINK-14858) Coding error,this should be "else if" not the "if".

2019-11-19 Thread zhaozhangwan (Jira)
zhaozhangwan created FLINK-14858: Summary: Coding error,this should be "else if" not the "if". Key: FLINK-14858 URL: https://issues.apache.org/jira/browse/FLINK-14858 Project: Flink Issue

[jira] [Created] (FLINK-14857) Deprecate checkpoint lock from the Operators API

2019-11-19 Thread Roman Khachatryan (Jira)
Roman Khachatryan created FLINK-14857: - Summary: Deprecate checkpoint lock from the Operators API Key: FLINK-14857 URL: https://issues.apache.org/jira/browse/FLINK-14857 Project: Flink

Re: [VOTE] FLIP-83: Flink End-to-end Performance Testing Framework

2019-11-19 Thread lining jing
+1 (non-binding) Dian Fu 于2019年11月19日周二 下午6:45写道: > +1 (non-binding) > > Regards, > Dian > > > 在 2019年11月19日,下午6:31,Piotr Nowojski 写道: > > > > +1 (non-binding) > > > > Piotrek > > > >> On 19 Nov 2019, at 04:20, Yang Wang wrote: > >> > >> +1 (non-binding) > >> > >> It is great to have a new

Re: [VOTE] FLIP-83: Flink End-to-end Performance Testing Framework

2019-11-19 Thread Dian Fu
+1 (non-binding) Regards, Dian > 在 2019年11月19日,下午6:31,Piotr Nowojski 写道: > > +1 (non-binding) > > Piotrek > >> On 19 Nov 2019, at 04:20, Yang Wang wrote: >> >> +1 (non-binding) >> >> It is great to have a new end-to-end test framework, even it is only for >> performance tests now. >> >>

Re: [VOTE] FLIP-83: Flink End-to-end Performance Testing Framework

2019-11-19 Thread Piotr Nowojski
+1 (non-binding) Piotrek > On 19 Nov 2019, at 04:20, Yang Wang wrote: > > +1 (non-binding) > > It is great to have a new end-to-end test framework, even it is only for > performance tests now. > > Best, > Yang > > Jingsong Li 于2019年11月19日周二 上午9:54写道: > >> +1 (non-binding) >> >> Best, >>

[jira] [Created] (FLINK-14856) Allow setting "state.checkpoints.num-retained" for individual jobs

2019-11-19 Thread Robert Metzger (Jira)
Robert Metzger created FLINK-14856: -- Summary: Allow setting "state.checkpoints.num-retained" for individual jobs Key: FLINK-14856 URL: https://issues.apache.org/jira/browse/FLINK-14856 Project:

Re: [DISCUSS] Expose or setup a secur...@flink.apache.org mailing list for security report and discussion

2019-11-19 Thread Dian Fu
Hi all, Thanks for sharing your thoughts. Appreciated! Let me try to summarize the information and thoughts received so far. Please feel free to let me know if there is anything wrong or missing. 1. Setup project specific security mailing list Pros: - The security reports received by

Re: [DISCUSS] Release flink-shaded 9.0

2019-11-19 Thread Dian Fu
I see, thanks for the reminder @Chesnay! Will help on the release check once the RC is out. Regards, Dian On Tue, Nov 19, 2019 at 4:41 PM Chesnay Schepler wrote: > Thanks for the offer, but without being a committer I don't think > there's a lot to do :/ > > @Uce If no one else steps up I'll

[jira] [Created] (FLINK-14855) travis-ci error.

2019-11-19 Thread ouyangwulin (Jira)
ouyangwulin created FLINK-14855: --- Summary: travis-ci error. Key: FLINK-14855 URL: https://issues.apache.org/jira/browse/FLINK-14855 Project: Flink Issue Type: Bug Components:

[jira] [Created] (FLINK-14854) Add executeAsync() method to environments

2019-11-19 Thread Aljoscha Krettek (Jira)
Aljoscha Krettek created FLINK-14854: Summary: Add executeAsync() method to environments Key: FLINK-14854 URL: https://issues.apache.org/jira/browse/FLINK-14854 Project: Flink Issue

Re: [ANNOUNCE] Launch of flink-packages.org: A website to foster the Flink Ecosystem

2019-11-19 Thread vino yang
Hi Robert, Just added it under the "Tools" category[1]. [1]: https://flink-packages.org/packages/kylin-flink-cube-engine Best, Vino Robert Metzger 于2019年11月19日周二 下午4:33写道: > Thanks. > You can add Kylin whenever you think it is ready. > > On Tue, Nov 19, 2019 at 9:07 AM vino yang wrote: > >

Re: [DISCUSS] Release flink-shaded 9.0

2019-11-19 Thread Chesnay Schepler
Thanks for the offer, but without being a committer I don't think there's a lot to do :/ @Uce If no one else steps up I'll kick it off later today myself; this would mean a release on Friday. On 19/11/2019 09:17, Dian Fu wrote: Hi Chesnay, Thanks a lot for kicking off this release. +1 to

[jira] [Created] (FLINK-14853) Use higher granularity units in generated docs for Duration & MemorySize if possible

2019-11-19 Thread Dawid Wysakowicz (Jira)
Dawid Wysakowicz created FLINK-14853: Summary: Use higher granularity units in generated docs for Duration & MemorySize if possible Key: FLINK-14853 URL: https://issues.apache.org/jira/browse/FLINK-14853

Re: [DISCUSS] Release flink-shaded 9.0

2019-11-19 Thread vino yang
+1 Best, Vino Dian Fu 于2019年11月19日周二 下午4:17写道: > Hi Chesnay, > > Thanks a lot for kicking off this release. +1 to release flink-shaded 9.0. > > I'm willing to help on the release. Please feel free to let me know > if there is anything I could help. > > Regards, > Dian > > On Mon, Nov 18, 2019

Re: [ANNOUNCE] Launch of flink-packages.org: A website to foster the Flink Ecosystem

2019-11-19 Thread Robert Metzger
Thanks. You can add Kylin whenever you think it is ready. On Tue, Nov 19, 2019 at 9:07 AM vino yang wrote: > Thanks Robert. Great job! The web site looks great. > > In the future, we can also add my Kylin Flink cube engine[1] to the > ecosystem projects list. > > [1]:

[jira] [Created] (FLINK-14852) translate concepts/programming-model to chinese

2019-11-19 Thread Zhangcheng Hu (Jira)
Zhangcheng Hu created FLINK-14852: - Summary: translate concepts/programming-model to chinese Key: FLINK-14852 URL: https://issues.apache.org/jira/browse/FLINK-14852 Project: Flink Issue

[jira] [Created] (FLINK-14851) Make the (Stream)ContextEnvironment use the Executors

2019-11-19 Thread Kostas Kloudas (Jira)
Kostas Kloudas created FLINK-14851: -- Summary: Make the (Stream)ContextEnvironment use the Executors Key: FLINK-14851 URL: https://issues.apache.org/jira/browse/FLINK-14851 Project: Flink

[jira] [Created] (FLINK-14850) Refactor Executor interface and introduce a minimal JobClient interface

2019-11-19 Thread Kostas Kloudas (Jira)
Kostas Kloudas created FLINK-14850: -- Summary: Refactor Executor interface and introduce a minimal JobClient interface Key: FLINK-14850 URL: https://issues.apache.org/jira/browse/FLINK-14850 Project:

Re: [DISCUSS] Release flink-shaded 9.0

2019-11-19 Thread Dian Fu
Hi Chesnay, Thanks a lot for kicking off this release. +1 to release flink-shaded 9.0. I'm willing to help on the release. Please feel free to let me know if there is anything I could help. Regards, Dian On Mon, Nov 18, 2019 at 8:43 PM Ufuk Celebi wrote: > @Chesnay: I know you said that you

Re: [VOTE] FLIP-79: Flink Function DDL Support (1.10 Release Feature Only)

2019-11-19 Thread Dawid Wysakowicz
Hi, Thank you for your answers and updating the WIKI. @Bowen Thank you for updating the wiki page and stepping up for shepherding the FLIP. Just wanted to raise a slight concern if it is possible to accept a FLIP partially according to our FLIP process. From what I know so far we were rather

Re: [ANNOUNCE] Launch of flink-packages.org: A website to foster the Flink Ecosystem

2019-11-19 Thread vino yang
Thanks Robert. Great job! The web site looks great. In the future, we can also add my Kylin Flink cube engine[1] to the ecosystem projects list. [1]: https://github.com/apache/kylin/tree/engine-flink Best, Vino Oytun Tez 于2019年11月19日周二 上午12:09写道: > Congratulations! This is exciting. > > >