Re: Re: [DISCUSS] Contribute Pulsar Flink connector back to Flink

2019-09-25 Thread Sijie Guo
+1 for starting with pulsar sink and catalog. Sijie On Tue, Sep 24, 2019 at 3:36 PM Jark Wu wrote: > I agree that we can start from Pulsar sink and Pulsar Catalog. > When we finish that, maybe FLIP-27 is ready then. > > Best, > Jark > > On Tue, 24 Sep 2019 at 17:06, Becket Qin wrote: > >> Hi T

[jira] [Created] (FLINK-14202) Optimize the execution plan for Python Calc when there is a condition

2019-09-25 Thread Dian Fu (Jira)
Dian Fu created FLINK-14202: --- Summary: Optimize the execution plan for Python Calc when there is a condition Key: FLINK-14202 URL: https://issues.apache.org/jira/browse/FLINK-14202 Project: Flink

How to corretly use checkstyle in IntelliJ IDEA

2019-09-25 Thread Felipe Gutierrez
Hi, is there another way to use the checkstyle.xml with IntelliJ IDEA that is different from the official documentation [1]? I imported flink source code and I developed my own function on the code. After that, I run the check style feature on Intellij IDEA 2019 and it points a lot of errors rega

Re: Confluence permission for FLIP creation

2019-09-25 Thread Fabian Hueske
Hi, I gave you the permissions for the wiki. Thanks, Fabian Am Mi., 25. Sept. 2019 um 05:04 Uhr schrieb Yadong Xie : > Hi all > > I'm proposing to improve the Flink user experience in the Web UI and I have > created a draft proposal doc. > > Could anyone grant the permission to convert the doc

Re: [COMMITTER] repo locked due to synchronization issues

2019-09-25 Thread Till Rohrmann
I think it makes sense to state a recommendation in the wiki. If we decide on something I would be in favour of pushing to Github because settings credentials up (private keys and signing keys) can happen in one place. Moreover, one can reuse these credentials to push to contributors PR branches w

Re: How to corretly use checkstyle in IntelliJ IDEA

2019-09-25 Thread Till Rohrmann
Hi Felipe, Flink's checkstyle prohibits the direct usage of Guava. Please import the shaded Guava version `import org.apache.flink.shaded.guava18.com.google.common.base.Strings;`. Cheers, Till On Wed, Sep 25, 2019 at 9:31 AM Felipe Gutierrez < felipe.o.gutier...@gmail.com> wrote: > Hi, > > is t

Re: FLINK-14108

2019-09-25 Thread Till Rohrmann
Hi Lasse, thanks for starting this discussion. I think this feature could be helpful. I've pulled in Aljoscha who's working most with Flink's formats module these days to see what he thinks. Cheers, Till On Wed, Sep 25, 2019 at 8:00 AM Lasse Nedergaard wrote: > Hi. > > I have create the jira

Re: [DISCUSS] FLIP-73: Introducing Executors for job submission

2019-09-25 Thread Kostas Kloudas
Hi Thomas and Zili, As you both said the Executor is a new addition so there are no compatibility concerns. Backwards compatibility comes into play on the (Stream)ExecutionEnvironment#execute(). This method has to stay and keep having the same (blocking) semantics, but we can add a new one, sth a

Re: How to corretly use checkstyle in IntelliJ IDEA

2019-09-25 Thread Terry Wang
Hi Felipe, If you must use guava directly, you can modify the config of /tools/maven/suppressions.xml like the Cassandra connectors’s config. As for checkStyle plugin not worked well your dev-environment, you should solve the errors line by line according to the error msg . Hope it helps you~ B

[jira] [Created] (FLINK-14203) add an abstract class for mappers with rich model.

2019-09-25 Thread Xu Yang (Jira)
Xu Yang created FLINK-14203: --- Summary: add an abstract class for mappers with rich model. Key: FLINK-14203 URL: https://issues.apache.org/jira/browse/FLINK-14203 Project: Flink Issue Type: Sub-task

[jira] [Created] (FLINK-14204) Add the model mapper for Gaussian Mixture model.

2019-09-25 Thread Xu Yang (Jira)
Xu Yang created FLINK-14204: --- Summary: Add the model mapper for Gaussian Mixture model. Key: FLINK-14204 URL: https://issues.apache.org/jira/browse/FLINK-14204 Project: Flink Issue Type: Sub-task

Re: Confluence permission for FLIP creation

2019-09-25 Thread Yadong Xie
Thanks a lot! Fabian Hueske 于2019年9月25日周三 下午3:55写道: > Hi, > > I gave you the permissions for the wiki. > > Thanks, Fabian > > Am Mi., 25. Sept. 2019 um 05:04 Uhr schrieb Yadong Xie < > vthink...@gmail.com > >: > > > Hi all > > > > I'm proposing to improve the Flink user experience in the Web UI

[jira] [Created] (FLINK-14205) Distinguish duplicate job submissions from other job submission errors

2019-09-25 Thread Till Rohrmann (Jira)
Till Rohrmann created FLINK-14205: - Summary: Distinguish duplicate job submissions from other job submission errors Key: FLINK-14205 URL: https://issues.apache.org/jira/browse/FLINK-14205 Project: Fli

Re: [DISCUSS] Add ARM CI build to Flink (information-only)

2019-09-25 Thread Xiyuan Wang
Hi Till Thanks for your response. All ARM related work is triggered here: https://issues.apache.org/jira/browse/FLINK-13448 and I have created some PRs already. After do some hacking locally, E2E tests runs well now. I have added them into OpenLab alreay. The POC log: http://status.openla

[jira] [Created] (FLINK-14206) Make fullRestart metric to count fine grained restarts as well

2019-09-25 Thread Zhu Zhu (Jira)
Zhu Zhu created FLINK-14206: --- Summary: Make fullRestart metric to count fine grained restarts as well Key: FLINK-14206 URL: https://issues.apache.org/jira/browse/FLINK-14206 Project: Flink Issue T

[jira] [Created] (FLINK-14207) YarnMiniCluster failed to start in YARNSessionFIFOSecuredITCase

2019-09-25 Thread Paul Lin (Jira)
Paul Lin created FLINK-14207: Summary: YarnMiniCluster failed to start in YARNSessionFIFOSecuredITCase Key: FLINK-14207 URL: https://issues.apache.org/jira/browse/FLINK-14207 Project: Flink Issu

[jira] [Created] (FLINK-14208) Support Python UDF with parameters of constant values

2019-09-25 Thread Dian Fu (Jira)
Dian Fu created FLINK-14208: --- Summary: Support Python UDF with parameters of constant values Key: FLINK-14208 URL: https://issues.apache.org/jira/browse/FLINK-14208 Project: Flink Issue Type: Sub-t

[DISCUSS] Expose multiple level clients

2019-09-25 Thread Zili Chen
Hi all, While drafting FLIP-74 I notice that a job level client(called JobClient) is always retrieved from a Flink application cluster level client(called ClusterClient), which is then always retrieved from a extern cluster(YARN, mesos, k8s, etc.) level client(called ClusterDescriptor). A Flink j

[DISCUSS] FLIP-74: Flink JobClient API

2019-09-25 Thread Zili Chen
Hi all, Summary from the discussion about introducing Flink JobClient API[1] we draft FLIP-74[2] to gather thoughts and towards a standard public user-facing interfaces. This discussion thread aims at standardizing job level client API. But I'd like to emphasize that how to retrieve JobClient pos

[jira] [Created] (FLINK-14209) Add jackson-dataformat-xml to flink-shaded

2019-09-25 Thread Flavio Pompermaier (Jira)
Flavio Pompermaier created FLINK-14209: -- Summary: Add jackson-dataformat-xml to flink-shaded Key: FLINK-14209 URL: https://issues.apache.org/jira/browse/FLINK-14209 Project: Flink Issue

Re: [VOTE] FLIP-68: Extend Core Table System with Modular Plugins

2019-09-25 Thread Bowen Li
Hi, I'd like to withdraw the vote for the moment. From offline feedback I got, the community is currently running out of bandwidth to review and vote this FLIP. I'd hold back this effort a little bit, On Tue, Sep 24, 2019 at 3:26 PM Xuefu Z wrote: > +1, LGTM > > On Mon, Sep 23, 2019 at 10:26 AM

Re: [COMMITTER] repo locked due to synchronization issues

2019-09-25 Thread Jark Wu
+1 to sticking with Github repo. I have encountered once that pushing to ASF repo is blocked in China, but GitHub always works for me and has better performance. And this keeps the awesome green button for merging PRs from GitHub UI. Best, Jark On Wed, 25 Sep 2019 at 15:57, Till Rohrmann wrote:

Re: [VOTE] FLIP-57: Rework FunctionCatalog

2019-09-25 Thread Dawid Wysakowicz
Ad. 1 I wouldn't say it is hacky. Moreover, how do you want ensure that the dB always exists when a temporary object is used?( in this particular case function). Do you want to query for the database existence whenever e.g a temporary function is used? I think important aspect here is that the data

Re: [DISCUSS] Releasing Flink 1.9.1

2019-09-25 Thread Jark Wu
Hi all, I am here to update the progress of the issue that needs to be tracked: - FLINK-14010 (merged) - FLINK-14118 (under discussion whether we should back port it to 1.9) - FLINK-13386 (Daryl reviewed and Dawid will verify the functionality again) - FLINK-13708 (under reviewing) - FLINK-14145

[jira] [Created] (FLINK-14210) influxdb reporter add connect timeout and write timeout config options for decrease timeout exception

2019-09-25 Thread ouyangwulin (Jira)
ouyangwulin created FLINK-14210: --- Summary: influxdb reporter add connect timeout and write timeout config options for decrease timeout exception Key: FLINK-14210 URL: https://issues.apache.org/jira/browse/FLINK-1421

Re: [DISCUSS] FLIP-73: Introducing Executors for job submission

2019-09-25 Thread Aljoscha Krettek
Hi, I’m fine with either signature for the new execute() method but I think we should focus on the executor discovery and executor configuration part in this FLIP while FLIP-74 is about the evolution of the method signature to return a future. I understand that it’s a bit weird, that this FLIP

Re: [DISCUSS] FLIP-74: Flink JobClient API

2019-09-25 Thread Aljoscha Krettek
Hi Tison, Thanks for proposing the document! I had some comments on the document. I think the only complex thing that we still need to figure out is how to get a JobClient for a job that is already running. As you mentioned in the document. Currently I’m thinking that its ok to add a method to

Re: [DISCUSS] FLIP-73: Introducing Executors for job submission

2019-09-25 Thread Kostas Kloudas
Hi, @Aljoscha, I believe that it is better to be done like this so that we do not step on each-other's feet. If the executor already "knew" about the JobClient, then we should also know about how we expect the JobExecutionResult is retrieved (which is part of FLIP-74). I think it is nice to have e

[jira] [Created] (FLINK-14211) Add jobManager address configuration for SqlClient

2019-09-25 Thread zhangwei (Jira)
zhangwei created FLINK-14211: Summary: Add jobManager address configuration for SqlClient Key: FLINK-14211 URL: https://issues.apache.org/jira/browse/FLINK-14211 Project: Flink Issue Type: Improv

[jira] [Created] (FLINK-14212) Support Python UDFs without arguments

2019-09-25 Thread Dian Fu (Jira)
Dian Fu created FLINK-14212: --- Summary: Support Python UDFs without arguments Key: FLINK-14212 URL: https://issues.apache.org/jira/browse/FLINK-14212 Project: Flink Issue Type: Sub-task Co

[jira] [Created] (FLINK-14213) Link from Flink website to Getting Started Overview page

2019-09-25 Thread Fabian Hueske (Jira)
Fabian Hueske created FLINK-14213: - Summary: Link from Flink website to Getting Started Overview page Key: FLINK-14213 URL: https://issues.apache.org/jira/browse/FLINK-14213 Project: Flink Is

Re: [DISCUSS] FLIP-74: Flink JobClient API

2019-09-25 Thread Kostas Kloudas
Hi Tison, Thanks for the FLIP and launching the discussion! As a first note, big +1 on providing/exposing a JobClient to the users! Some points that would be nice to be clarified: 1) You mention that we can get rid of the DETACHED mode: I agree that at a high level, given that everything will no

Re: [DISCUSS] Expose multiple level clients

2019-09-25 Thread Kostas Kloudas
Hi Tison, I shared some thoughts on this topic in the FLIP-74 thread, as some of the questions are related to that. Cheers, Kostas On Wed, Sep 25, 2019 at 12:30 PM Zili Chen wrote: > > Hi all, > > While drafting FLIP-74 I notice that a job level client(called JobClient) is > always retrieved f

[jira] [Created] (FLINK-14214) Performance regression in TwoInputStreamOperator

2019-09-25 Thread Piotr Nowojski (Jira)
Piotr Nowojski created FLINK-14214: -- Summary: Performance regression in TwoInputStreamOperator Key: FLINK-14214 URL: https://issues.apache.org/jira/browse/FLINK-14214 Project: Flink Issue Ty

[jira] [Created] (FLINK-14215) Add Docs for TM and JM Environment Variable Setting

2019-09-25 Thread Zhenqiu Huang (Jira)
Zhenqiu Huang created FLINK-14215: - Summary: Add Docs for TM and JM Environment Variable Setting Key: FLINK-14215 URL: https://issues.apache.org/jira/browse/FLINK-14215 Project: Flink Issue T

Re: [SURVEY] How many people are using customized RestartStrategy(s)

2019-09-25 Thread Steven Wu
Zhu Zhu, that is correct. On Tue, Sep 24, 2019 at 8:04 PM Zhu Zhu wrote: > Hi Steven, > > As a conclusion, since we will have a meter metric[1] for restarts, > customized restart strategy is not needed in your case. > Is that right? > > [1] https://issues.apache.org/jira/browse/FLINK-14164 > > T

Re: [SURVEY] How many people are using customized RestartStrategy(s)

2019-09-25 Thread Zhu Zhu
We will then keep the decision that we do not support customized restart strategy in Flink 1.10. Thanks Steven for the inputs! Thanks, Zhu Zhu Steven Wu 于2019年9月26日周四 上午12:13写道: > Zhu Zhu, that is correct. > > On Tue, Sep 24, 2019 at 8:04 PM Zhu Zhu wrote: > >> Hi Steven, >> >> As a conclusio

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

2019-09-25 Thread Xuefu Z
Actually catalogs are more of system settings than of user objects that a user might create or drop constantly. Thus, it's probably sufficient to set up catalog information in the config file, at least for now. Thanks, Xuefu On Tue, Sep 24, 2019 at 7:10 PM Terry Wang wrote: > Thanks Bowen for y

Re: [VOTE] FLIP-57: Rework FunctionCatalog

2019-09-25 Thread Bowen Li
Re 1) As described in the FLIP, a temp function lookup will first make sure the db exists. If the db doesn't exist, a lazy drop is triggered to remove that temp function. I agree Hive doesn't handle it consistently, and we are not copying Hive. IMHO, allowing registering temp functions in nonexis

[jira] [Created] (FLINK-14216) introduce temp system functions and APIs to FunctionCatalog

2019-09-25 Thread Bowen Li (Jira)
Bowen Li created FLINK-14216: Summary: introduce temp system functions and APIs to FunctionCatalog Key: FLINK-14216 URL: https://issues.apache.org/jira/browse/FLINK-14216 Project: Flink Issue Ty

[jira] [Created] (FLINK-14217) introduce temp functions and APIS to replace "userFunctions" in FunctionCatalog

2019-09-25 Thread Bowen Li (Jira)
Bowen Li created FLINK-14217: Summary: introduce temp functions and APIS to replace "userFunctions" in FunctionCatalog Key: FLINK-14217 URL: https://issues.apache.org/jira/browse/FLINK-14217 Project: Flin

[jira] [Created] (FLINK-14218) support precise function reference

2019-09-25 Thread Bowen Li (Jira)
Bowen Li created FLINK-14218: Summary: support precise function reference Key: FLINK-14218 URL: https://issues.apache.org/jira/browse/FLINK-14218 Project: Flink Issue Type: Sub-task Com

[jira] [Created] (FLINK-14219) support ambiguous function reference

2019-09-25 Thread Bowen Li (Jira)
Bowen Li created FLINK-14219: Summary: support ambiguous function reference Key: FLINK-14219 URL: https://issues.apache.org/jira/browse/FLINK-14219 Project: Flink Issue Type: Sub-task C

[jira] [Created] (FLINK-14220) support drop and alter temp functions

2019-09-25 Thread Bowen Li (Jira)
Bowen Li created FLINK-14220: Summary: support drop and alter temp functions Key: FLINK-14220 URL: https://issues.apache.org/jira/browse/FLINK-14220 Project: Flink Issue Type: Sub-task

[jira] [Created] (FLINK-14221) support drop and alter temp system functions

2019-09-25 Thread Bowen Li (Jira)
Bowen Li created FLINK-14221: Summary: support drop and alter temp system functions Key: FLINK-14221 URL: https://issues.apache.org/jira/browse/FLINK-14221 Project: Flink Issue Type: Sub-task

[jira] [Created] (FLINK-14222) Optimize for Python UDFs with all parameters are constant values

2019-09-25 Thread Dian Fu (Jira)
Dian Fu created FLINK-14222: --- Summary: Optimize for Python UDFs with all parameters are constant values Key: FLINK-14222 URL: https://issues.apache.org/jira/browse/FLINK-14222 Project: Flink Issue

Re: [DISCUSS] FLIP-63: Rework table partition support

2019-09-25 Thread JingsongLee
After offline discussion with Jark, the current grammar for creating partition tables is limited to hive dialect, and the Flink built-in grammar for creating partition tables is treated as further discussion, it will be determined by voting after a period of time (Need more thinking). Best, Jin

Re: [VOTE] FLIP-63: Rework table partition support

2019-09-25 Thread Jark Wu
Hi, After some discussion with Jinsong, the FLIP is slightly updated with the following 2 changes, which will not take too much time to agree on. 1) the proposed create partition table grammar is **limited to hive dialect** (via `TableConfig.setSqlDialect(HIVE)`). Flink's create partition tab

Re: CiBot Update

2019-09-25 Thread Zili Chen
I notice that flinkbot now includes an information about ci command Mention the bot in a comment to re-run the automated checks. however, there are many times I forget whether it is @flinkbot run travis or @flinkbot travis run It should be no harm we include the exact content there. Besides,

[VOTE] FLIP-66: Support Time Attribute in SQL DDL (#2)

2019-09-25 Thread Jark Wu
Hi all, I would like to start the vote for FLIP-66 [1], which is discussed and reached a consensus in the discussion thread[2]. The vote will be open for at least 72 hours. I'll try to close it after Oct. 01 08:00 UTC, unless there is an objection or not enough votes. Thanks, Jark [1]: https://

[DISCUSS] Flink Python UDF Environment and Dependency Management

2019-09-25 Thread Wei Zhong
Hi everyone, In FLIP-58 [1] we have a plan to support Python UDF. As a critical part of python UDF, the environment and dependency management of users' python code has not been fully discussed. I'd like to start a discussion on "Flink Python UDF Environment and Dependency Management". Here is

Re: [DISCUSS] Flink Python UDF Environment and Dependency Management

2019-09-25 Thread jincheng sun
Thanks for bring up the discussion, Wei. Overall the design doc looks good. I have left a few comments. BTW: Dependency Management is very important for Python UDFs, welcome anyone left your suggestions! Best, Jincheng Wei Zhong 于2019年9月26日周四 上午11:59写道: > Hi everyone, > > In FLIP-58 [1] we hav