Re: [DISCUSS] Use SET statement to set table config in Flink SQL and implement a unified SQL call method

2020-03-09 Thread Jark Wu
Hi Weike and Tison, This is already covered in FLIP-84 [1], we will propose a new method "executeStatement(String statement)" which can execute arbitrary statement including SET, CREATE. This is in the progress [2]. Best, Jark [1]: https://cwiki.apache.org/confluence/pages/viewpage.action?pageId

Re: [DISCUSS] Use SET statement to set table config in Flink SQL and implement a unified SQL call method

2020-03-09 Thread DONG, Weike
Hi Tison and all, Thanks for the timely response, and I have carefully examined the aforementioned FLIP-84. As I see it, executeStatement() is kind of akin to our original design of sql() method, but with more detailed considerations included. However, it does not cover SET statement to tune Tabl

[jira] [Created] (FLINK-16500) Hive get primary key should not throw exception when Invalid method name

2020-03-09 Thread Jingsong Lee (Jira)
Jingsong Lee created FLINK-16500: Summary: Hive get primary key should not throw exception when Invalid method name Key: FLINK-16500 URL: https://issues.apache.org/jira/browse/FLINK-16500 Project: Fli

Re: [DISCUSS] FLIP-85: Delayed Job Graph Generation

2020-03-09 Thread Yang Wang
Hi Becket, Thanks for your suggestion. We will update the FLIP to add/enrich the following parts. * User cli option change, use "-R/--remote" to apply the cluster deploy mode * Configuration change, how to specify remote user jars and dependencies * The whole story about how "application mode" wor

Re: [VOTE] FLIP-100[NEW]: Add Attempt Information

2020-03-09 Thread Yadong Xie
Thanks all for the votes. So far, we have - 3 binding +1 votes (Jark, Kurt, Gary) - No -1 votes The voting time has past and there are enough +1 votes to consider the FLIP-100 approved. Thank you all. Jark Wu 于2020年3月4日周三 下午8:55写道: > +1 > > On Wed, 4 Mar 2020 at 20:29, Kurt Young wrote: > >

Re: [DISCUSS] Disable "Squash and merge" button for Flink repository on GitHub

2020-03-09 Thread Jark Wu
Hi everyone, Updates: GitHub fixed this to preserve the authors information. However, the "committed" field will be "GitHub " instead who merges PR. I reported this in GitHub Community [1] and will track the problem later. Not sure it is a GitHub bug or my setting problem. Best, Jark [1]: https:

Re: [DISCUSS] FLIP-106: Support Python UDF in SQL Function DDL

2020-03-09 Thread Wei Zhong
Hi Dawid, I think defering the instantiation of temporary functions to compile time is quite a good idea but needs further discussion. As it is orthogonal with this FLIP, we could continue the discussion in a new thread later. What do you think? Best, Wei > 在 2020年3月5日,21:11,Wei Zhong 写道: >

[DISCUSS]FLIP-113: Support SQL and planner hints

2020-03-09 Thread Danny Chan
Hi, fellows ~ I would like to propose the supports for SQL hints for our Flink SQL. We would support hints syntax as following: select /*+ NO_HASH_JOIN, RESOURCE(mem='128mb', parallelism='24') */ from emp /*+ INDEX(idx1, idx2) */ join dept /*+ PROPERTIES(k1='v1', k2='v2') */ on emp.deptno = dept

Re: Flink dev blog

2020-03-09 Thread Arvid Heise
Dear all, Robert added a link to the engine room blog on the Apache wiki [1]. It's currently empty except for one test post. To add a new post, you just create a new blog post from the top menu ("engine room" is just a fancy link to the default blog). Robert would like to write about the migratio

Re: Flink dev blog

2020-03-09 Thread tison
Thank Arvid & Robert for the effort. Amazing! I'm curious the procedure a blog get posted. Follow the discussion so far it seems any contributor can post his blog under the directory as he wishes, is it the case? Best, tison. Arvid Heise 于2020年3月9日周一 下午4:54写道: > Dear all, > > Robert added a l

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

2020-03-09 Thread Danny Chan
Note: we only plan to support table hints in Flink release 1.11, so please focus mainly on the table hints part and just ignore the planner hints, sorry for that mistake ~ Best, Danny Chan 在 2020年3月9日 +0800 PM4:36,Danny Chan ,写道: > Hi, fellows ~ > > I would like to propose the supports for SQL h

[jira] [Created] (FLINK-16501) Support IS JSON predication for Table API in blink planner

2020-03-09 Thread Jark Wu (Jira)
Jark Wu created FLINK-16501: --- Summary: Support IS JSON predication for Table API in blink planner Key: FLINK-16501 URL: https://issues.apache.org/jira/browse/FLINK-16501 Project: Flink Issue Type:

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

2020-03-09 Thread DONG, Weike
Hi Danny, This is a nice feature, +1. One thing I am interested in but not mentioned in the proposal is the error handling, as it is quite common for users to write inappropriate hints in SQL code, if illegal or "bad" hints are given, would the system simply ignore them or throw exceptions? Than

[jira] [Created] (FLINK-16502) Add documentation for all JSON function

2020-03-09 Thread Jark Wu (Jira)
Jark Wu created FLINK-16502: --- Summary: Add documentation for all JSON function Key: FLINK-16502 URL: https://issues.apache.org/jira/browse/FLINK-16502 Project: Flink Issue Type: Sub-task

[jira] [Created] (FLINK-16503) Don't assign ScheduledExecutorService into variables of ExecutorService or Executor types

2020-03-09 Thread Roman Leventov (Jira)
Roman Leventov created FLINK-16503: -- Summary: Don't assign ScheduledExecutorService into variables of ExecutorService or Executor types Key: FLINK-16503 URL: https://issues.apache.org/jira/browse/FLINK-16503

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

2020-03-09 Thread Jark Wu
Thanks Danny for starting the discussion. +1 for this feature. If we just focus on the table hints not the query hints in this release, could you split the FLIP into two FLIPs? Because it's hard to vote on partial part of a FLIP. You can keep the table hints proposal in FLIP-113 and move query hin

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

2020-03-09 Thread Jingsong Li
Hi Danny, +1 for table hints, thanks for driving. I took a look to FLIP, most of content are talking about query hints. It is hard to discussion and voting. So +1 to split it as Jark said. Another thing is configuration that suitable to config with table hints: "connector.path" and "connector.top

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

2020-03-09 Thread Timo Walther
Hi Danny, thanks for the proposal. I agree with Jark and Jingsong. Planner hints and table hints are orthogonal topics that should be discussed separately. I share Jingsong's opinion that we should not use planner hints for passing connector properties. Planner hints should be optional at any

Re: Flink dev blog

2020-03-09 Thread Robert Metzger
Hey Tison, only people we have manually given write permission to the Wiki are able to add a blog post. If somebody is posting something we don't want there, we can just revoke that person's permission to write on the blog. It is definitely something we should keep an eye on, but I don't think we

How to change the flink web-ui jobServer?

2020-03-09 Thread LakeShen
Hi community, now I am moving the flink job to k8s,and I plan to use the ingress to show the flink web ui , the problem is that fink job server aren't correct, so I want to change the flink web-ui jobserver ,I don't find the any method to change it ,are there some method to do that?

Re: [DISCUSS] FLIP-85: Delayed Job Graph Generation

2020-03-09 Thread Becket Qin
Thanks Yang, That would be very helpful! Jiangjie (Becket) Qin On Mon, Mar 9, 2020 at 3:31 PM Yang Wang wrote: > Hi Becket, > > Thanks for your suggestion. We will update the FLIP to add/enrich the > following parts. > * User cli option change, use "-R/--remote" to apply the cluster deploy > m

Re: Flink dev blog

2020-03-09 Thread tison
Thanks for your reply Robert. That sounds great. Best, tison. Robert Metzger 于2020年3月9日周一 下午5:46写道: > Hey Tison, > > only people we have manually given write permission to the Wiki are able to > add a blog post. If somebody is posting something we don't want there, we > can just revoke that pe

Re: [DISCUSS] FLIP-106: Support Python UDF in SQL Function DDL

2020-03-09 Thread Timo Walther
Hi Wei, I agree with Dawid that we should defer the instantiation of temporary functions to compile time. In the long-term, we would like to integrate FunctionCatalog as a component of CatalogManager and unify the handling of catalog objects as much as possible. We should aim for your propos

Re: [DISCUSS] Use SET statement to set table config in Flink SQL and implement a unified SQL call method

2020-03-09 Thread Timo Walther
Hi Weike, thanks for your feedback. Your use case is definitely on our agenda. The redesign of big parts of the API is still in progress. In the mid-term, most of the SQL Client commands should be present in the SQL API as well such that platform teams can built their custom logic (like REST A

Re: [DISCUSS] FLIP-85: Delayed Job Graph Generation

2020-03-09 Thread Kostas Kloudas
Hi all, And thanks for the discussion topics. For the cluster lifecycle, it is the Entrypoint that will tear down the cluster when the application finishes. Probably we should emphasise it a bit more in the FLIP. For the -R flag, this was in the PoC that I published just as a quick implementatio

Re: Flink dev blog

2020-03-09 Thread Yu Li
Hurray! Thanks Arvid and Robert! Will ask the team here to prepare some RocksDB backend related posts. Best Regards, Yu On Mon, 9 Mar 2020 at 17:51, tison wrote: > Thanks for your reply Robert. That sounds great. > > Best, > tison. > > > Robert Metzger 于2020年3月9日周一 下午5:46写道: > > > Hey Tison,

[DISCUSS] Update on Flink Stateful Functions & what are the next steps?

2020-03-09 Thread Tzu-Li (Gordon) Tai
Hi devs, I’d like to briefly provide an update / overview on what has been going on in the Flink Stateful Functions project since it was moved to Apache Flink [1]. One other main purpose of this is to figure out the next steps for the project, such as release timing, and some open questions surro

Re: [DISCUSS] Update on Flink Stateful Functions & what are the next steps?

2020-03-09 Thread Igal Shilman
Hi Gordon, Thanks a lot for the summary and for kicking off the discussion! Regarding the next steps: * +1 to kick off the first release managed by the Flink community, as the amount of added features and fixies is substantial. * +1 to have the docs centralized under the Apache infra and made acc

Re: [DISCUSS] Update on Flink Stateful Functions & what are the next steps?

2020-03-09 Thread Stephan Ewen
Thanks, Gordon for this update. I think it would be great to do an Apache Release now. Given how much the "polyglot functions" (and the code-free ingress/egress configuration) change the applicable use cases, I would actually call this version 2.0. What do you think? About the https://statefun.io

[jira] [Created] (FLINK-16504) Add a AWS DynamoDB (sink) connector

2020-03-09 Thread Robert Metzger (Jira)
Robert Metzger created FLINK-16504: -- Summary: Add a AWS DynamoDB (sink) connector Key: FLINK-16504 URL: https://issues.apache.org/jira/browse/FLINK-16504 Project: Flink Issue Type: Improveme

[jira] [Created] (FLINK-16505) Document how to use the FlinkKinesisConsumer to read a CDC stream from DynamoDB

2020-03-09 Thread Robert Metzger (Jira)
Robert Metzger created FLINK-16505: -- Summary: Document how to use the FlinkKinesisConsumer to read a CDC stream from DynamoDB Key: FLINK-16505 URL: https://issues.apache.org/jira/browse/FLINK-16505 P

[jira] [Created] (FLINK-16506) Sql

2020-03-09 Thread Terry Wang (Jira)
Terry Wang created FLINK-16506: -- Summary: Sql Key: FLINK-16506 URL: https://issues.apache.org/jira/browse/FLINK-16506 Project: Flink Issue Type: Bug Reporter: Terry Wang -- Thi

[jira] [Created] (FLINK-16507) "update_branch_version" should also update the version in Stateful Function's Python SDK setup.py file

2020-03-09 Thread Tzu-Li (Gordon) Tai (Jira)
Tzu-Li (Gordon) Tai created FLINK-16507: --- Summary: "update_branch_version" should also update the version in Stateful Function's Python SDK setup.py file Key: FLINK-16507 URL: https://issues.apache.org/jira/

Re: [DISCUSS] FLIP-85: Delayed Job Graph Generation

2020-03-09 Thread Aljoscha Krettek
> For the -R flag, this was in the PoC that I published just as a quick > implementation, so that I can move fast to the entrypoint part. > Personally, I would not even be against having a separate command in > the CLI for this, sth like run-on-cluster or something along those > lines. > What do y

Re: [PROPOSAL] Reverse the dependency from flink-streaming-java to flink-client

2020-03-09 Thread Aljoscha Krettek
On 09.03.20 03:15, tison wrote: So far, there is a PR[1] that implements the proposal in this thread. I look forward to your reviews or start a vote if required. Nice, I'll try and get to review that this week. Best, Aljoscha

[DISCUSS] FLIP-114: Support Python UDF in SQL Client

2020-03-09 Thread Wei Zhong
Hi everyone, I would like to start discussion about how to support Python UDF in SQL Client. Flink Python UDF(FLIP-58[1]) has already been introduced in the release of 1.10.0 and the support for SQL DDL is introduced in FLIP-106[2]. SQL Client defines UDF via the environment file and has its

Re: [DISCUSS] Update on Flink Stateful Functions & what are the next steps?

2020-03-09 Thread Seth Wiesman
+1 to release. Stateful Functions has a strong set of core features, and a released version will help drive adoption which will in turn help shape feature development. Seth On Mon, Mar 9, 2020 at 6:59 AM Stephan Ewen wrote: > Thanks, Gordon for this update. > > I think it would be great to do

Re: [DISCUSS] FLIP-114: Support Python UDF in SQL Client

2020-03-09 Thread godfrey he
Hi Wei, thanks for the proposal. I think it's better to give two more examples, one is how to use python UDF in SQL, another is how to start sql-client.sh with full python dependencies. Best, Godfrey Wei Zhong 于2020年3月9日周一 下午10:09写道: > Hi everyone, > > I would like to start discussion about ho

Re: [DISCUSS] FLIP-114: Support Python UDF in SQL Client

2020-03-09 Thread Xingbo Huang
Hi Wei, Thanks a lot for the proposal. Best, Xingbo godfrey he 于2020年3月9日周一 下午10:24写道: > Hi Wei, thanks for the proposal. > > I think it's better to give two more examples, one is how to use python UDF > in SQL, another is how to start sql-client.sh with full python > dependencies. > > Best, >

[jira] [Created] (FLINK-16508) Name the ports exposed by the main Container in Pod

2020-03-09 Thread Canbin Zheng (Jira)
Canbin Zheng created FLINK-16508: Summary: Name the ports exposed by the main Container in Pod Key: FLINK-16508 URL: https://issues.apache.org/jira/browse/FLINK-16508 Project: Flink Issue Typ

[jira] [Created] (FLINK-16509) FlinkKafkaConsumerBase tries to log a context that may not have been initialized and fails

2020-03-09 Thread Arnaud Linz (Jira)
Arnaud Linz created FLINK-16509: --- Summary: FlinkKafkaConsumerBase tries to log a context that may not have been initialized and fails Key: FLINK-16509 URL: https://issues.apache.org/jira/browse/FLINK-16509

Re: [DISCUSS] FLIP-114: Support Python UDF in SQL Client

2020-03-09 Thread Xingbo Huang
Hi Godfrey, thanks for your suggestion. I have added two examples how to use python UDF in SQL and how to start sql-client.sh with full python dependencies In FLIP. Best, Xingo godfrey he 于2020年3月9日周一 下午10:24写道: > Hi Wei, thanks for the proposal. > > I think it's better to give two more example

[jira] [Created] (FLINK-16510) Task manager safeguard shutdown may not be reliable

2020-03-09 Thread Maximilian Michels (Jira)
Maximilian Michels created FLINK-16510: -- Summary: Task manager safeguard shutdown may not be reliable Key: FLINK-16510 URL: https://issues.apache.org/jira/browse/FLINK-16510 Project: Flink

[jira] [Created] (FLINK-16511) Task cancellation timeout is not effective on OOM errors

2020-03-09 Thread Maximilian Michels (Jira)
Maximilian Michels created FLINK-16511: -- Summary: Task cancellation timeout is not effective on OOM errors Key: FLINK-16511 URL: https://issues.apache.org/jira/browse/FLINK-16511 Project: Flink

[jira] [Created] (FLINK-16512) Add persistence API

2020-03-09 Thread Roman Khachatryan (Jira)
Roman Khachatryan created FLINK-16512: - Summary: Add persistence API Key: FLINK-16512 URL: https://issues.apache.org/jira/browse/FLINK-16512 Project: Flink Issue Type: Sub-task

[jira] [Created] (FLINK-16513) Implement persistence API

2020-03-09 Thread Roman Khachatryan (Jira)
Roman Khachatryan created FLINK-16513: - Summary: Implement persistence API Key: FLINK-16513 URL: https://issues.apache.org/jira/browse/FLINK-16513 Project: Flink Issue Type: Sub-task

[jira] [Created] (FLINK-16514) SQLClientKafkaITCase fails with output mismatch

2020-03-09 Thread Robert Metzger (Jira)
Robert Metzger created FLINK-16514: -- Summary: SQLClientKafkaITCase fails with output mismatch Key: FLINK-16514 URL: https://issues.apache.org/jira/browse/FLINK-16514 Project: Flink Issue Typ

[jira] [Created] (FLINK-16515) Refactor statefun documentation for multi-language SDKs

2020-03-09 Thread Seth Wiesman (Jira)
Seth Wiesman created FLINK-16515: Summary: Refactor statefun documentation for multi-language SDKs Key: FLINK-16515 URL: https://issues.apache.org/jira/browse/FLINK-16515 Project: Flink Issue

[jira] [Created] (FLINK-16516) Remove Python UDF Codegen Code

2020-03-09 Thread Huang Xingbo (Jira)
Huang Xingbo created FLINK-16516: Summary: Remove Python UDF Codegen Code Key: FLINK-16516 URL: https://issues.apache.org/jira/browse/FLINK-16516 Project: Flink Issue Type: Improvement

[jira] [Created] (FLINK-16517) Add a long running WordCount example

2020-03-09 Thread Ethan Li (Jira)
Ethan Li created FLINK-16517: Summary: Add a long running WordCount example Key: FLINK-16517 URL: https://issues.apache.org/jira/browse/FLINK-16517 Project: Flink Issue Type: Improvement

Re: [DISCUSS] FLIP-85: Delayed Job Graph Generation

2020-03-09 Thread Yang Wang
Hi Aljoscha, Kostas, I would be in favour of something like "bin/flink run-application", > maybe we should even have "run-job" in the future to differentiate. I have no preference for the "-R/--remote-deploy" option of "flink run" or new introduced "flink run-application". If we always bind the

[jira] [Created] (FLINK-16518) Stateful Function's KafkaSinkProvider should use `setProperty` instead of `put` for resolving client properties

2020-03-09 Thread Tzu-Li (Gordon) Tai (Jira)
Tzu-Li (Gordon) Tai created FLINK-16518: --- Summary: Stateful Function's KafkaSinkProvider should use `setProperty` instead of `put` for resolving client properties Key: FLINK-16518 URL: https://issues.apache.

[jira] [Created] (FLINK-16519) CheckpointCoordinatorFailureTest logs LinkageErrors

2020-03-09 Thread godfrey he (Jira)
godfrey he created FLINK-16519: -- Summary: CheckpointCoordinatorFailureTest logs LinkageErrors Key: FLINK-16519 URL: https://issues.apache.org/jira/browse/FLINK-16519 Project: Flink Issue Type: B

Re: [DISCUSS] Update on Flink Stateful Functions & what are the next steps?

2020-03-09 Thread Tzu-Li (Gordon) Tai
Thanks for the feedback so far! @Stephan Ewen I don't have an objection with 2.0 being the next version to release. My only concern is that at this point it's slightly hard to tell if the APIs are stable enough (especially the Python SDK), and it could be arguable that it is better to wait until

Re: [DISCUSS] Use SET statement to set table config in Flink SQL and implement a unified SQL call method

2020-03-09 Thread DONG, Weike
Hi Timo, After carefully read FLIP-91 (SQL Client Gateway), I have found that it still focuses on ad-hoc (or realtime) queries of batch data, which is quite different from the streaming case. Here I suppose if we could combine some features in FLIP-84 (generic all-purpose executeStatement() ) wit

Re: [PROPOSAL] Reverse the dependency from flink-streaming-java to flink-client

2020-03-09 Thread tison
Thanks for your attention! Best, tison. Aljoscha Krettek 于2020年3月9日周一 下午10:08写道: > On 09.03.20 03:15, tison wrote: > > > So far, there is a PR[1] that implements the proposal in this thread. > > > > I look forward to your reviews or start a vote if required. > > Nice, I'll try and get to revie

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

2020-03-09 Thread Danny Chan
To Weike: About the Error Handing To be consistent with other SQL vendors, the default is to log warnings and if there is any error (invalid hint name or options), the hint is just ignored. I have already addressed in the wiki. To Timo: About the PROPERTIES Table Hint • The properties hints is