Re: [DISCUSS] FLIP-59: Enable execution configuration from Configuration object

2019-11-07 Thread Dawid Wysakowicz
Hi, Thank you for the comments Kostas, Timo, Aljoscha. I also like the pipeline/execution naming. I tried to apply most of your suggestions Aljoscha. There are a few cases when I did not. You mentioned a few options that are already present, and I planned to reuse the existing options (latencyTra

[VOTE] FLIP-59: Enable execution configuration from Configuration object

2019-11-07 Thread Dawid Wysakowicz
Hello, please vote for FLIP-59 . The discussion thread can be found here

[jira] [Created] (FLINK-14651) Set default value of config option jobmanager.scheduler to "ng"

2019-11-07 Thread Gary Yao (Jira)
Gary Yao created FLINK-14651: Summary: Set default value of config option jobmanager.scheduler to "ng" Key: FLINK-14651 URL: https://issues.apache.org/jira/browse/FLINK-14651 Project: Flink Issu

Re: [VOTE] FLIP-69: Flink SQL DDL Enhancement

2019-11-07 Thread Rui Li
I see, thanks for the clarification. In current implementation, it seems just a duplicate of comment. So I'd prefer not to display it for DESCRIBE DATABASE, because 1) users have no control over the content and 2) it's totally redundant. We can add it in the future when we come up with something mo

HadoopInputFormat Custom Partitioning

2019-11-07 Thread Dominik Wosiński
Hey, I wanted to ask if the *HadoopInputFormat* does currently support some custom partitioning scheme ? Say I have 200 files in HDFS each having the partitioning key in name, can we ATM use HadoopInputFormat to distribute reading to multiple TaskManagers using the key ?? Best Regards, Dom.

Re: [VOTE] FLIP-59: Enable execution configuration from Configuration object

2019-11-07 Thread tison
Hi Dawid, I'm afraid that you list the wrong FLIP page. Although the content is FLIP-59 but it directs to FLIP-67. Best, tison. Dawid Wysakowicz 于2019年11月7日周四 下午5:04写道: > Hello, > > please vote for FLIP-59 >

Re: [VOTE] FLIP-59: Enable execution configuration from Configuration object

2019-11-07 Thread Dawid Wysakowicz
Thank you tison. You are right. I did not update the hyperlinks. Sorry for that. Once again then: please vote for FLIP-59 https://cwiki.apache.org/confluence/display/FLINK/FLIP-59%3A+Enable+execution+configuration+from+Configuration+object. The discussion thread can be found here http://apache-f

Re: [VOTE] FLIP-69: Flink SQL DDL Enhancement

2019-11-07 Thread Terry Wang
Hi Rui~ What you suggested makes sense, remove description and detailed description from `DESCRIBE DATABASE`. Open to more comments and votes :) Best, Terry Wang > 2019年11月7日 17:15,Rui Li 写道: > > I see, thanks for the clarification. In current implementation, it seems > just a duplicate of c

[jira] [Created] (FLINK-14652) Refactor checkpointing related parts into one place on task side

2019-11-07 Thread Yun Tang (Jira)
Yun Tang created FLINK-14652: Summary: Refactor checkpointing related parts into one place on task side Key: FLINK-14652 URL: https://issues.apache.org/jira/browse/FLINK-14652 Project: Flink Iss

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

2019-11-07 Thread Yu Li
Thanks for the comments. bq. I think the perf e2e test suites will also need to be designed as supporting running on both standalone env and distributed env. will be helpful for developing & evaluating the perf. Agreed and marked down, the benchmark will be able to be executed in standalone mode.

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

2019-11-07 Thread Yu Li
Thanks for the suggestion Jingsong! I've added a stage for adding more metrics in FLIP document, please check and let me know if any further concerns. Thanks. Best Regards, Yu On Mon, 4 Nov 2019 at 17:37, Jingsong Li wrote: > +1 for the idea. Thanks Yu for driving this. > Just curious about t

[jira] [Created] (FLINK-14653) Job-related errors in snapshotState do not result in job failure

2019-11-07 Thread Maximilian Michels (Jira)
Maximilian Michels created FLINK-14653: -- Summary: Job-related errors in snapshotState do not result in job failure Key: FLINK-14653 URL: https://issues.apache.org/jira/browse/FLINK-14653 Project:

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

2019-11-07 Thread Yu Li
Thanks for the comments Biao! bq. It seems this proposal is separated into several stages. Is there a more detailed plan? Good point! For stage one we'd like to try introducing the benchmark first, so we could guard the release (hopefully starting from 1.10). For other stages, we don't have detail

[jira] [Created] (FLINK-14654) Fix the arguments number mismatching with placeholders in log statements

2019-11-07 Thread Yun Tang (Jira)
Yun Tang created FLINK-14654: Summary: Fix the arguments number mismatching with placeholders in log statements Key: FLINK-14654 URL: https://issues.apache.org/jira/browse/FLINK-14654 Project: Flink

Re: How long is the flink sql task state default ttl?

2019-11-07 Thread Dian Fu
It's disabled by default. BTW: You only need to send it to user ML and it's not necessary to send it to the dev ML. > 在 2019年11月7日,下午3:36,LakeShen 写道: > > Hi community, as I know I can use idle state retention time to clear the > flink sql task state,I have a question is that how long the fl

[jira] [Created] (FLINK-14655) Change Type of Field jobStatusListeners from CopyOnWriteArrayList to ArrayList

2019-11-07 Thread vinoyang (Jira)
vinoyang created FLINK-14655: Summary: Change Type of Field jobStatusListeners from CopyOnWriteArrayList to ArrayList Key: FLINK-14655 URL: https://issues.apache.org/jira/browse/FLINK-14655 Project: Flink

[jira] [Created] (FLINK-14656) blink planner should convert catalog statistics to TableStats for permanent table instead of temporary table

2019-11-07 Thread godfrey he (Jira)
godfrey he created FLINK-14656: -- Summary: blink planner should convert catalog statistics to TableStats for permanent table instead of temporary table Key: FLINK-14656 URL: https://issues.apache.org/jira/browse/FLINK

[jira] [Created] (FLINK-14657) Generalize and move YarnConfigUtils from flink-yarn to flink-core

2019-11-07 Thread Kostas Kloudas (Jira)
Kostas Kloudas created FLINK-14657: -- Summary: Generalize and move YarnConfigUtils from flink-yarn to flink-core Key: FLINK-14657 URL: https://issues.apache.org/jira/browse/FLINK-14657 Project: Flink

[jira] [Created] (FLINK-14658) Drop ".returns()" for TypeInformation in the DataStream API

2019-11-07 Thread Stephan Ewen (Jira)
Stephan Ewen created FLINK-14658: Summary: Drop ".returns()" for TypeInformation in the DataStream API Key: FLINK-14658 URL: https://issues.apache.org/jira/browse/FLINK-14658 Project: Flink

[DISCUSS] Stateful Functions - Contribution Details

2019-11-07 Thread Igal Shilman
Hello everyone! Following the successful vote to accept Stateful Functions into Flink [1], I would like to start a discussion regarding the technical aspects of the contribution. Once the discussion will finalize I will summarize the results into a FLIP and bring it up to a vote. 1) External repo

Re: [DISCUSS] Stateful Functions - Contribution Details

2019-11-07 Thread Chesnay Schepler
[1] Does not directly link to the voting thread. 1) I skimmed all 3 threads about the stateful functions proposal and could not find a rational for the repository name, I'd appreciate a direct link to the relevant post. 2.1) +1 as we use o.a.f also for flink-shaded 3) +1 as it follows the ex

[jira] [Created] (FLINK-14659) add 'LOAD MODULE' and 'UNLOAD MODULE' sql commands to sql parser

2019-11-07 Thread Bowen Li (Jira)
Bowen Li created FLINK-14659: Summary: add 'LOAD MODULE' and 'UNLOAD MODULE' sql commands to sql parser Key: FLINK-14659 URL: https://issues.apache.org/jira/browse/FLINK-14659 Project: Flink Iss

[jira] [Created] (FLINK-14660) add 'SHOW MODULES' sql command

2019-11-07 Thread Bowen Li (Jira)
Bowen Li created FLINK-14660: Summary: add 'SHOW MODULES' sql command Key: FLINK-14660 URL: https://issues.apache.org/jira/browse/FLINK-14660 Project: Flink Issue Type: Sub-task Compone

[jira] [Created] (FLINK-14661) rename args of setters in sql cli Environment

2019-11-07 Thread Bowen Li (Jira)
Bowen Li created FLINK-14661: Summary: rename args of setters in sql cli Environment Key: FLINK-14661 URL: https://issues.apache.org/jira/browse/FLINK-14661 Project: Flink Issue Type: Improvemen

[jira] [Created] (FLINK-14662) Distinguish unknown table stats and zero

2019-11-07 Thread Kurt Young (Jira)
Kurt Young created FLINK-14662: -- Summary: Distinguish unknown table stats and zero Key: FLINK-14662 URL: https://issues.apache.org/jira/browse/FLINK-14662 Project: Flink Issue Type: Improvement

[jira] [Created] (FLINK-14663) Distinguish unknown column stats and zero

2019-11-07 Thread Kurt Young (Jira)
Kurt Young created FLINK-14663: -- Summary: Distinguish unknown column stats and zero Key: FLINK-14663 URL: https://issues.apache.org/jira/browse/FLINK-14663 Project: Flink Issue Type: Improvement

Re: [VOTE] FLIP-59: Enable execution configuration from Configuration object

2019-11-07 Thread Terry Wang
Thanks for driving on this. +1 from my side (non-binding) Best, Terry Wang > 2019年11月7日 17:34,Dawid Wysakowicz 写道: > > Thank you tison. You are right. I did not update the hyperlinks. Sorry > for that. Once again then: > > please vote for FLIP-59 > https://cwiki.apache.org/confluence/display

[jira] [Created] (FLINK-14664) Support to reference user defined functions of external catalog in computed columns

2019-11-07 Thread Danny Chen (Jira)
Danny Chen created FLINK-14664: -- Summary: Support to reference user defined functions of external catalog in computed columns Key: FLINK-14664 URL: https://issues.apache.org/jira/browse/FLINK-14664 Proje

Re: [VOTE] FLIP-69: Flink SQL DDL Enhancement

2019-11-07 Thread Kurt Young
Hi all, I think we should focus to discuss the document in [DISCUSS] thread and keep this vote thread purely for voting. Otherwise, it's hard for others to collect feedbacks for this topic. Best, Kurt On Thu, Nov 7, 2019 at 5:51 PM Terry Wang wrote: > Hi Rui~ > What you suggested makes sense

[jira] [Created] (FLINK-14665) Support computed column for create table statement in blink-planner

2019-11-07 Thread Danny Chen (Jira)
Danny Chen created FLINK-14665: -- Summary: Support computed column for create table statement in blink-planner Key: FLINK-14665 URL: https://issues.apache.org/jira/browse/FLINK-14665 Project: Flink

Re: [DISCUSS] FLIP 69 - Flink SQL DDL Enhancement

2019-11-07 Thread Kurt Young
Hi, Sorry to join this so late and thanks for proposing this FLIP. After going through the proposal details, I would +1 for the changes. However, the FLIP name is kind of confusing me. It says will do DDL enhancement, and picked up a few new features to do. It looks to me the goal and content of

[jira] [Created] (FLINK-14666) support multiple catalog in flink table sql

2019-11-07 Thread yuemeng (Jira)
yuemeng created FLINK-14666: --- Summary: support multiple catalog in flink table sql Key: FLINK-14666 URL: https://issues.apache.org/jira/browse/FLINK-14666 Project: Flink Issue Type: Bug C

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

2019-11-07 Thread Peter Huang
Dear All, I would like to start the vote for 1.10 release features in FLIP-79 [1] which is discussed and research consensus in the discussion thread [2]. For the advanced feature, such as loading function from remote resources, support scala/python function, we will have the further discussion aft

[jira] [Created] (FLINK-14667) flink1.9,1.8.2 run flinkSql fat jar ,can't load the right tableFactory (TableSourceFactory) for the kafka

2019-11-07 Thread chun111111 (Jira)
chun11 created FLINK-14667: -- Summary: flink1.9,1.8.2 run flinkSql fat jar ,can't load the right tableFactory (TableSourceFactory) for the kafka Key: FLINK-14667 URL: https://issues.apache.org/jira/browse/FLINK-1

Re: [DISCUSS] FLIP 69 - Flink SQL DDL Enhancement

2019-11-07 Thread Terry Wang
Hi, Kurt~ Thanks for your vote and pointing out some deficiency of this flip. I’ll try to avoid making similar mistakes. Best, Terry Wang > 2019年11月8日 11:28,Kurt Young 写道: > > Hi, > > Sorry to join this so late and thanks for proposing this FLIP. After > going through the proposal details,

Re: [DISCUSS] FLIP 69 - Flink SQL DDL Enhancement

2019-11-07 Thread Kurt Young
Hi Terry, I wouldn't say it's a mistake, I also don't have any suggestions about the issue. I just saw this and want to point it out to bring more attention. Maybe someone has some good opinion on that part and we can discuss around it and have some good advises for the community in the future. B

Re: [VOTE] FLIP-69: Flink SQL DDL Enhancement

2019-11-07 Thread Kurt Young
Forgot to vote.. +1 from my side. Best, Kurt On Fri, Nov 8, 2019 at 11:00 AM Kurt Young wrote: > Hi all, > > I think we should focus to discuss the document in [DISCUSS] thread and > keep this vote thread purely for voting. > > Otherwise, it's hard for others to collect feedbacks for this topi

[jira] [Created] (FLINK-14668) LocalExecutor#getOrCreateExecutionContext not working as expected

2019-11-07 Thread zhangwei (Jira)
zhangwei created FLINK-14668: Summary: LocalExecutor#getOrCreateExecutionContext not working as expected Key: FLINK-14668 URL: https://issues.apache.org/jira/browse/FLINK-14668 Project: Flink Is

Re: Flunk savepoin(checkpoint) load api or debug

2019-11-07 Thread qq
Hi all, Thanks very much. I wants to debug checkpoint with code. Below is my code. Anyway I am sorry I doesn’t understand UT class. def demo(): Unit = { val env = StreamExecutionEnvironment.getExecutionEnvironment env.setParallelism(1) env.enableCheckpointing(1) val checkpointConf

[jira] [Created] (FLINK-14669) All hadoop-2.4.1 related nightly end-to-end tests failed on travis

2019-11-07 Thread Yu Li (Jira)
Yu Li created FLINK-14669: - Summary: All hadoop-2.4.1 related nightly end-to-end tests failed on travis Key: FLINK-14669 URL: https://issues.apache.org/jira/browse/FLINK-14669 Project: Flink Issue T