Re: [DISCUSS] FLIP-54: Evolve ConfigOption and Configuration

2019-08-29 Thread Stephan Ewen
@Becket One thing that may be non-obvious is that the Configuration class also defines serialization / persistence logic at the moment. So it needs to know the set of types it supports. That stands in the way of an arbitrary generic map type. @Timo I agree though that it seems a bit inconsistent t

[jira] [Created] (FLINK-13888) Translate "How To Contribute" page into Chinese

2019-08-29 Thread Lord i Will (Jira)
Lord i Will created FLINK-13888: --- Summary: Translate "How To Contribute" page into Chinese Key: FLINK-13888 URL: https://issues.apache.org/jira/browse/FLINK-13888 Project: Flink Issue Type: Tas

[jira] [Created] (FLINK-13889) sql client fetch result oom

2019-08-29 Thread richt richt (Jira)
richt richt created FLINK-13889: --- Summary: sql client fetch result oom Key: FLINK-13889 URL: https://issues.apache.org/jira/browse/FLINK-13889 Project: Flink Issue Type: Bug Component

[jira] [Created] (FLINK-13890) HiveCatalogUseBlinkITCase failed to get metastore connection

2019-08-29 Thread Piotr Nowojski (Jira)
Piotr Nowojski created FLINK-13890: -- Summary: HiveCatalogUseBlinkITCase failed to get metastore connection Key: FLINK-13890 URL: https://issues.apache.org/jira/browse/FLINK-13890 Project: Flink

Re: [DISCUSS] FLIP-49: Unified Memory Configuration for TaskExecutors

2019-08-29 Thread Stephan Ewen
When computing the values in the JVM process after it started, how would you deal with values like Max Direct Memory, Metaspace size. native memory reservation (reduce heap size), etc? All the values that are parameters to the JVM process and that need to be supplied at process startup? On Wed, Au

Re: [DISCUSS] FLIP-55: Introduction of a Table API Java Expression DSL

2019-08-29 Thread Aljoscha Krettek
Overall, this is a very nice development that should also simplify the code base once we deprecate the expression parser! Regarding method names, I agree with Seth that values/literals should use something like “lit()”. I also think that for column references we could use “col()” to make it cle

[jira] [Created] (FLINK-13891) Increment flink-shaded version

2019-08-29 Thread Chesnay Schepler (Jira)
Chesnay Schepler created FLINK-13891: Summary: Increment flink-shaded version Key: FLINK-13891 URL: https://issues.apache.org/jira/browse/FLINK-13891 Project: Flink Issue Type: Improvemen

[jira] [Created] (FLINK-13892) HistoryServerTest failed on Travis

2019-08-29 Thread Chesnay Schepler (Jira)
Chesnay Schepler created FLINK-13892: Summary: HistoryServerTest failed on Travis Key: FLINK-13892 URL: https://issues.apache.org/jira/browse/FLINK-13892 Project: Flink Issue Type: Improv

[jira] [Created] (FLINK-13893) S3 tests are failing due to missing jaxb dependency

2019-08-29 Thread Chesnay Schepler (Jira)
Chesnay Schepler created FLINK-13893: Summary: S3 tests are failing due to missing jaxb dependency Key: FLINK-13893 URL: https://issues.apache.org/jira/browse/FLINK-13893 Project: Flink I

Re: [DISCUSS] FLIP-54: Evolve ConfigOption and Configuration

2019-08-29 Thread Timo Walther
Hi Becket, let me try to clarify some of your questions: 1. For every option, we also needed to think about how to represent it in a human readable format. We do not want to allow arbitrary nesting because that would easily allow to bypass the flattened hierarchy of config options (`session.m

Re: [DISCUSS] FLIP-49: Unified Memory Configuration for TaskExecutors

2019-08-29 Thread Till Rohrmann
My understanding was that before starting the Flink process we call a utility which calculates these values. I assume that this utility will do the calculation based on a set of configured values (process memory, flink memory, network memory etc.). Assuming that these values don't differ from the v

Re: [DISCUSS] FLIP-55: Introduction of a Table API Java Expression DSL

2019-08-29 Thread Timo Walther
I'm fine with `lit()`. Regarding `col()`, I initially suggested `ref()` but I think Fabian and Dawid liked single char methods for the most commonly used expressions. Btw, what is your opinion on the names of commonly used methods such as `isEqual`, `isGreaterOrEqual`? Are we fine with the cur

[jira] [Created] (FLINK-13894) Web Ui add log url for subtask of vertex

2019-08-29 Thread lining (Jira)
lining created FLINK-13894: -- Summary: Web Ui add log url for subtask of vertex Key: FLINK-13894 URL: https://issues.apache.org/jira/browse/FLINK-13894 Project: Flink Issue Type: Improvement

[jira] [Created] (FLINK-13895) Client does not exit when bin/yarn-session.sh come fail

2019-08-29 Thread Yu Wang (Jira)
Yu Wang created FLINK-13895: --- Summary: Client does not exit when bin/yarn-session.sh come fail Key: FLINK-13895 URL: https://issues.apache.org/jira/browse/FLINK-13895 Project: Flink Issue Type: Imp

Re: [DISCUSS] FLIP-55: Introduction of a Table API Java Expression DSL

2019-08-29 Thread Fabian Hueske
Hi, IMO, we should define what we would like to optimize for: 1) easy-to-get-started experience or 2) productivity and ease-of-use While 1) is certainly important, I think we should put more emphasis on goal 2). That's why I favor as short as possible names for commonly used methods like column r

[jira] [Created] (FLINK-13896) Scala 2.11 maven compile should target Java 1.8

2019-08-29 Thread Terry Wang (Jira)
Terry Wang created FLINK-13896: -- Summary: Scala 2.11 maven compile should target Java 1.8 Key: FLINK-13896 URL: https://issues.apache.org/jira/browse/FLINK-13896 Project: Flink Issue Type: Bug

Re: [DISCUSS] FLIP-49: Unified Memory Configuration for TaskExecutors

2019-08-29 Thread Stephan Ewen
I see. Under the assumption of strict determinism that should work. The original proposal had this point "don't compute inside the TM, compute outside and supply a full config", because that sounded more intuitive. On Thu, Aug 29, 2019 at 12:15 PM Till Rohrmann wrote: > My understanding was tha

Re: [DISCUSS] FLIP-49: Unified Memory Configuration for TaskExecutors

2019-08-29 Thread Till Rohrmann
I think our goal should be that the configuration is fully specified when the process is started. By considering the internal calculation step to be rather validate existing values and calculate missing ones, these two proposal shouldn't even conflict (given determinism). Since we don't want to ch

Re: [DISCUSS] FLIP-49: Unified Memory Configuration for TaskExecutors

2019-08-29 Thread Till Rohrmann
What I forgot to add is that we could tackle specifying the configuration fully in an incremental way and that the full specification should be the desired end state. On Thu, Aug 29, 2019 at 1:33 PM Till Rohrmann wrote: > I think our goal should be that the configuration is fully specified when

[jira] [Created] (FLINK-13897) OSS FS NOTICE file is placed in wrong directory

2019-08-29 Thread Chesnay Schepler (Jira)
Chesnay Schepler created FLINK-13897: Summary: OSS FS NOTICE file is placed in wrong directory Key: FLINK-13897 URL: https://issues.apache.org/jira/browse/FLINK-13897 Project: Flink Issue

Re: [CODE-STYLE] Builder pattern

2019-08-29 Thread Gyula Fóra
Hi all, Thank you all for the valuable feedback, let me try to summarize the rough agreement: If there is a builder for class A, then A should: - Have only private ctor -> force usage of the builder - Be immutable (no setters) - Have a public static .builder(required params if not too many) m

[jira] [Created] (FLINK-13898) Migrate restart strategy config constants to RestartStrategyOptions

2019-08-29 Thread Till Rohrmann (Jira)
Till Rohrmann created FLINK-13898: - Summary: Migrate restart strategy config constants to RestartStrategyOptions Key: FLINK-13898 URL: https://issues.apache.org/jira/browse/FLINK-13898 Project: Flink

[jira] [Created] (FLINK-13899) Add SQL DDL for Elasticsearch 5.X version

2019-08-29 Thread limbo (Jira)
limbo created FLINK-13899: - Summary: Add SQL DDL for Elasticsearch 5.X version Key: FLINK-13899 URL: https://issues.apache.org/jira/browse/FLINK-13899 Project: Flink Issue Type: Improvement

Re: [DISCUSS] FLIP-50: Spill-able Heap Keyed State Backend

2019-08-29 Thread Zili Chen
Hi Yu, Notice that the wiki is still marked as "*Under Discussion*" state. I think you can update it correspondingly. Best, tison. Yu Li 于2019年8月20日周二 下午10:28写道: > Sorry for the lag but since we've got a consensus days ago, I started a > vote thread which will have a result by EOD, thus I'm

[jira] [Created] (FLINK-13900) Add a built-in "console" sink to print results in console

2019-08-29 Thread Jark Wu (Jira)
Jark Wu created FLINK-13900: --- Summary: Add a built-in "console" sink to print results in console Key: FLINK-13900 URL: https://issues.apache.org/jira/browse/FLINK-13900 Project: Flink Issue Type: N

Re: [VOTE] FLIP-54: Evolve ConfigOption and Configuration

2019-08-29 Thread Timo Walther
I converted the mentioned Google doc into a wiki page: https://cwiki.apache.org/confluence/display/FLINK/FLIP-54%3A+Evolve+ConfigOption+and+Configuration The core semantics have not changed. Happy voting, Timo On 29.08.19 04:30, Zili Chen wrote: The design looks good to me. +1 go ahead! Bes

Re: [DISCUSS] Best practice to run flink on kubernetes

2019-08-29 Thread Thomas Weise
Till had already summed it up, but I want to emphasize that Flink as project only needs to provide #1 (reactive mode) and #3 (active mode, which necessarily is tied to the cluster manager of choice). The latter would be needed for Flink jobs to be elastic (in the future), although we may want to di

Re: Flink operators for Kubernetes

2019-08-29 Thread Thomas Weise
In case anyone comes across this thread in the archives, the FlinkK8sOperator is now available here: https://github.com/lyft/flinkk8soperator The community is invited to check it out, provide feedback (use github issues) or even better, join and contribute to it. Thomas On Fri, Nov 2, 2018 at

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

2019-08-29 Thread Dawid Wysakowicz
Hi, I wanted to propose a new, additional way of configuring execution parameters that can currently be set only on such objects like ExecutionConfig, CheckpointConfig and StreamExecutionEnvironment. This poses problems such as: * no easy way to configure those from a file * there is no easy

[PROPOSAL] Force rebase on master before merge

2019-08-29 Thread Zili Chen
Hi devs, GitHub provides a mechanism which is able to require branches to be up to date before merged[1](point 6). I can see several advantages enabling it. Thus propose our project to turn on this switch. Below are my concerns. Looking forward to your insights. 1. Avoid CI failures in pr which f

Re: [DISCUSS] FLIP-54: Evolve ConfigOption and Configuration

2019-08-29 Thread Becket Qin
Hi Timo and Stephan, Thanks for the detail explanation. 1. I agree that each config should be in a human readable format. My concern is that the current List looks going a little too far from what the configuration is supposed to do. They are essentially creating some Configurable objects instead

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

2019-08-29 Thread Gyula Fóra
Hi! Huuuge +1 from me, this has been an operational pain for years. This would also introduce a nice and simple way to extend it in the future if we need. Ship it! Gyula On Thu, Aug 29, 2019 at 5:05 PM Dawid Wysakowicz wrote: > Hi, > > I wanted to propose a new, additional way of configuring

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

2019-08-29 Thread Gyula Fóra
What we could also add to make this a bit more generic and extensible is to create some interfaces for reconfiguring the StreamExecutionEnvironment, ExecutionConfig etc and let users specify a class the implements the reconfiguration logic based on the flink configuration. This could be executed a

How to handle Flink Job with 400MB+ Uberjar with 800+ containers ?

2019-08-29 Thread Elkhan Dadashov
Dear Flink developers, Having difficulty of getting a Flink job started. The job's uberjar/fat jar is around 400MB, and I need to kick 800+ containers. The default HDFS replication is 3. *The Yarn queue is empty, and 800 containers are allocated almost immediately by Yarn RM.* It takes v

Re: [DISCUSS] FLIP-57 - Rework FunctionCatalog

2019-08-29 Thread Bowen Li
Thanks everyone for the feedback. I have updated the document accordingly. Here're the summary of changes: - clarify the concept of temporary functions, to facilitate deciding function resolution order - provide two options to support Hive built-in functions, with the 2nd one being preferred - ad

Re: [DISCUSS] Enhance Support for Multicast Communication Pattern

2019-08-29 Thread Yun Gao
Hi all, Very thanks Jark for the new scenarios. Based on the these new scenarios, I think these scenarios and iteration should be able to represent a type of scenarios that requires broadcasting events. I also totally agree with Piotr that all the scenarios we have discussed should be clearly

Re: [PROPOSAL] Force rebase on master before merge

2019-08-29 Thread Kurt Young
Hi Zili, Thanks for the proposal, I had similar confusion in the past with your point #2. Force rebase to master before merging can solve some problems, but it also introduces new problem. Given the CI testing time is quite long (couple of hours) now, it's highly possible that before your test whi

Re: [VOTE] FLIP-58: Flink Python User-Defined Function for Table API

2019-08-29 Thread Wei Zhong
Hi Dian, +1 non-binding Thanks for driving this! Best, Wei > 在 2019年8月29日,09:25,Hequn Cheng 写道: > > Hi Dian, > > +1 > Thanks a lot for driving this. > > Best, Hequn > > On Wed, Aug 28, 2019 at 2:01 PM jincheng sun > wrote: > >> Hi Dian, >> >> +1, Thanks for your great job! >> >> Best, >

Re: [VOTE] FLIP-58: Flink Python User-Defined Function for Table API

2019-08-29 Thread Xingbo Huang
Hi Dian, +1, Thanks a lot for driving this. Best, Xingbo > 在 2019年8月30日,上午9:39,Wei Zhong 写道: > > Hi Dian, > > +1 non-binding > Thanks for driving this! > > Best, Wei > >> 在 2019年8月29日,09:25,Hequn Cheng 写道: >> >> Hi Dian, >> >> +1 >> Thanks a lot for driving this. >> >> Best, Hequn >> >

Re: How to handle Flink Job with 400MB+ Uberjar with 800+ containers ?

2019-08-29 Thread SHI Xiaogang
Hi Datashov, We faced similar problems in our production clusters. Now both lauching and stopping of containers are performed in the main thread of YarnResourceManager. As containers are launched and stopped one after another, it usually takes long time to boostrap large jobs. Things get worse wh

[jira] [Created] (FLINK-13901) Documentation links check errors in release-1.9

2019-08-29 Thread Jark Wu (Jira)
Jark Wu created FLINK-13901: --- Summary: Documentation links check errors in release-1.9 Key: FLINK-13901 URL: https://issues.apache.org/jira/browse/FLINK-13901 Project: Flink Issue Type: Bug

Re: [VOTE] FLIP-58: Flink Python User-Defined Function for Table API

2019-08-29 Thread Jark Wu
+1 Thanks for the great work! On Fri, 30 Aug 2019 at 10:04, Xingbo Huang wrote: > Hi Dian, > > +1, > Thanks a lot for driving this. > > Best, > Xingbo > > 在 2019年8月30日,上午9:39,Wei Zhong 写道: > > > > Hi Dian, > > > > +1 non-binding > > Thanks for driving this! > > > > Best, Wei > > > >> 在 2019年8月

[jira] [Created] (FLINK-13902) Can not use index to convert FieldReferenceExpression to RexNode

2019-08-29 Thread Jingsong Lee (Jira)
Jingsong Lee created FLINK-13902: Summary: Can not use index to convert FieldReferenceExpression to RexNode Key: FLINK-13902 URL: https://issues.apache.org/jira/browse/FLINK-13902 Project: Flink

[jira] [Created] (FLINK-13903) Support Hive version 2.3.6

2019-08-29 Thread Xuefu Zhang (Jira)
Xuefu Zhang created FLINK-13903: --- Summary: Support Hive version 2.3.6 Key: FLINK-13903 URL: https://issues.apache.org/jira/browse/FLINK-13903 Project: Flink Issue Type: Improvement Co

Re: [PROPOSAL] Force rebase on master before merge

2019-08-29 Thread Zili Chen
Hi Kurt, Thanks for your reply! I find two concerns about the downside from your email. Correct me if I misunderstanding. 1. Rebase times. Typically commits are independent one another, rebase just fast-forward changes so that contributors rarely resolve conflicts by himself. Reviews doesn't get

Re: [VOTE] FLIP-58: Flink Python User-Defined Function for Table API

2019-08-29 Thread Terry Wang
+1. That would be very helpful. Best, Terry Wang > 在 2019年8月30日,上午10:18,Jark Wu 写道: > > +1 > > Thanks for the great work! > > On Fri, 30 Aug 2019 at 10:04, Xingbo Huang wrote: > >> Hi Dian, >> >> +1, >> Thanks a lot for driving this. >> >> Best, >> Xingbo >>> 在 2019年8月30日,上午9:39,Wei Zhon

Re: [VOTE] FLIP-58: Flink Python User-Defined Function for Table API

2019-08-29 Thread Yu Li
+1 (non-binding) Thanks for driving this! Best Regards, Yu On Fri, 30 Aug 2019 at 11:01, Terry Wang wrote: > +1. That would be very helpful. > Best, > Terry Wang > > > > > 在 2019年8月30日,上午10:18,Jark Wu 写道: > > > > +1 > > > > Thanks for the great work! > > > > On Fri, 30 Aug 2019 at 10:04, Xin

Using Avro SpecficRecord serialization instead of slower ReflectDatumWriter/GenericDatumWriter

2019-08-29 Thread Roshan Naik
Noticing that Flink takes very long inside collect(..) due to Avro serialization that relies on  ReflectDatumWriter & GenericDatumWriter.   The object being serialized here is an Avro object that implements SpecificRecordBase. It is somewhat about large (~50Kb) and complex.  Looking for a way t

Re: [VOTE] FLIP-58: Flink Python User-Defined Function for Table API

2019-08-29 Thread Jeff Zhang
+1, very looking forward this feature in flink 1.10 Yu Li 于2019年8月30日周五 上午11:08写道: > +1 (non-binding) > > Thanks for driving this! > > Best Regards, > Yu > > > On Fri, 30 Aug 2019 at 11:01, Terry Wang wrote: > > > +1. That would be very helpful. > > Best, > > Terry Wang > > > > > > > > > 在 201

[jira] [Created] (FLINK-13904) Avoid competition between different rounds of checkpoint triggering

2019-08-29 Thread Biao Liu (Jira)
Biao Liu created FLINK-13904: Summary: Avoid competition between different rounds of checkpoint triggering Key: FLINK-13904 URL: https://issues.apache.org/jira/browse/FLINK-13904 Project: Flink

Re: [DISCUSS] Best practice to run flink on kubernetes

2019-08-29 Thread Yang Wang
Hi Zhenghua, You are right. For per-job cluster, the taskmanagers will be allocated dynamically by KubernetesResourceManager. For session cluster, we hope taskmangers could be pre-allocated even though it does not work now. Please navigate to the doc[1] for more details. Hi Thomas, We have

[jira] [Created] (FLINK-13905) Separate checkpoint triggering into stages

2019-08-29 Thread Biao Liu (Jira)
Biao Liu created FLINK-13905: Summary: Separate checkpoint triggering into stages Key: FLINK-13905 URL: https://issues.apache.org/jira/browse/FLINK-13905 Project: Flink Issue Type: Sub-task