Re: [VOTE] FLIP-95: New TableSource and TableSink interfaces

2020-03-30 Thread Jingsong Li
+1 Best, Jingsong Lee On Mon, Mar 30, 2020 at 4:41 PM Kurt Young wrote: > +1 > > Best, > Kurt > > > On Mon, Mar 30, 2020 at 4:08 PM Benchao Li wrote: > > > +1 (non-binding) > > > > Jark Wu 于2020年3月30日周一 下午3:57写道: > > > > > +1 from my side. > > > > > > Thanks Timo for driving this. > > > > > >

Re: [ANNOUNCE] Flink on Zeppelin (Zeppelin 0.9 is released)

2020-03-30 Thread Jingsong Li
Thanks Jeff very much, that is very impressive. Zeppelin is very convenient development platform. Best, Jingsong Lee On Tue, Mar 31, 2020 at 11:58 AM Zhijiang wrote: > > Thanks for the continuous efforts for engaging in Flink ecosystem Jeff! > Glad to see the progressive achievement. Wish more

Re: [DISCUSS] FLIP-110: Support LIKE clause in CREATE TABLE

2020-03-31 Thread Jingsong Li
QL standard, I think it's fine for vendors to > >>> define > >>> theire semantics. > >>> > >>>> Standard also allows declaring the clause after the schema part. We > >>>> can > >>> also do it. > >>> Is that true? I didn't find it in SQL st

Re: [VOTE] FLIP-113: Supports Dynamic Table Options for Flink SQL

2020-03-31 Thread Jingsong Li
Hi Danny, If I haven't missed anything, I don't know how to support the dynamic options of format. Connector don't know format information in TableFactory before obtains real properties, so it can not list any format `supportedHintOptions`. And I think it is important for support format dynamic o

Re: [DISCUSS] Change default planner to blink planner in 1.11

2020-03-31 Thread Jingsong Li
+1 In 1.10, we have set default planner for SQL Client to Blink planner[1]. Looks good. [1] http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-Set-default-planner-for-SQL-Client-to-Blink-planner-in-1-10-release-td36379.html Best, Jingsong Lee On Wed, Apr 1, 2020 at 11:39 AM

Re: [ANNOUNCE] New Committers and PMC member

2020-04-01 Thread Jingsong Li
Congratulations! Konstantin, Dawid, Zhijiang. Well deserved. Best, Jingsong Lee On Wed, Apr 1, 2020 at 4:52 PM Stephan Ewen wrote: > Hi all! > > Happy to announce that over the last few weeks, several people in the > community joined in new roles: > > - Konstantin Knauf joined as a committer.

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

2020-04-01 Thread Jingsong Li
Hi Dawid, > When a factory is instantiated it has access to the CatalogTable, therefore it has access to all the original properties. In turn it knows the original format and can call FormatFactory#supportedHintOptions(). Factory can only get CatalogTable when creating source or sink, right? IIUC

Re: [VOTE] FLIP-122: New Connector Property Keys for New Factory

2020-04-02 Thread Jingsong Li
+1 Best, Jingsong Lee On Fri, Apr 3, 2020 at 9:50 AM Benchao Li wrote: > +1 (non-binding) > > Dawid Wysakowicz 于2020年4月3日周五 上午12:33写道: > > > +1 > > > > Best, > > > > Dawid > > > > On 02/04/2020 18:28, Timo Walther wrote: > > > +1 > > > > > > Thanks, > > > Timo > > > > > > On 02.04.20 17:22, Ja

Re: [ANNOUNCE] New Flink committer: Seth Wiesman

2020-04-06 Thread Jingsong Li
Congratulations Seth! Best, Jingsong Lee On Tue, Apr 7, 2020 at 2:46 PM Dawid Wysakowicz wrote: > Congratulations Seth. Happy to have you in the community! > > Best, > > Dawid > > On 07/04/2020 08:43, Dian Fu wrote: > > Congratulations! > > > >> 在 2020年4月7日,下午2:35,Konstantin Knauf 写道: > >> > >

Re: [DISCUSS] FLIP-123: DDL and DML compatibility for Hive connector

2020-04-09 Thread Jingsong Li
Thanks Rui for diving. +1 for this proposal. There are still lots of people who love Hive SQL. And I have seen some people support HQL on presto. Presto, as a famous computing engine, also supports ANSI SQL as we do. This is quite different from HQL. Do you think we must need import `FlinkHiveSq

Re: [VOTE] FLIP-71: E2E View support in Flink SQL

2020-04-09 Thread Jingsong Li
+1 Hope we can finish it in 1.11, we have been looking forward to it for a long time. Best, Jingsong Lee On Thu, Apr 9, 2020 at 9:23 PM Timo Walther wrote: > +1 (binding) > > Thanks for your efforts. > > Regards, > Timo > > > On 09.04.20 14:46, Zhenghua Gao wrote: > > Hi all, > > > > I'd like

Re: [DISCUSS] FLIP-123: DDL and DML compatibility for Hive connector

2020-04-13 Thread Jingsong Li
ate parser only requires minimum change in the planner, which I > think is acceptable compared to the benefits it brings us. > > On Thu, Apr 9, 2020 at 10:42 PM Jingsong Li > wrote: > > > Thanks Rui for diving. > > > > +1 for this proposal. > > > > Th

Re: [VOTE] FLIP-105: Support to Interpret and Emit Changelog in Flink SQL

2020-04-13 Thread Jingsong Li
+1 Best, Jingsong Lee On Tue, Apr 14, 2020 at 1:46 PM Jark Wu wrote: > +1 (binding) > > Best, > Jark > > On Sun, 12 Apr 2020 at 09:24, Benchao Li wrote: > > > +1 (non-binding) > > > > Jark Wu 于2020年4月11日周六 上午11:31写道: > > > > > Hi all, > > > > > > I would like to start the vote for FLIP-105 [1

Re: flink-orc or flink-orc-nohive

2020-04-14 Thread Jingsong Li
Hi, Maybe you should use flink-orc. And use orc-core instead of orc-core with nohive classifier. We can provide nohive version in the future. Because orc and hive are so close, orc still relies on some classes of hive currently. Apache orc with nohive classifier is for create a variant of core an

Re: [VOTE] FLIP-123: DDL and DML compatibility for Hive connector

2020-04-14 Thread Jingsong Li
+1 Best, Jingsong Lee On Tue, Apr 14, 2020 at 9:29 PM Kurt Young wrote: > +1 > > Best, > Kurt > > > On Mon, Apr 13, 2020 at 9:26 PM Rui Li wrote: > > > Hi all, > > > > I'd like to start the vote for FLIP-123[1], which has been discussed in > > thread[2]. > > > > The vote will be open for 72h,

Re: flink-orc or flink-orc-nohive

2020-04-14 Thread Jingsong Li
slightly newer version - maybe 1.5.x or even 1.6.0? > > - > Sivaprasanna > > On Tue, Apr 14, 2020 at 1:42 PM Jingsong Li > wrote: > > > Hi, > > > > Maybe you should use flink-orc. And use orc-core instead of orc-core with > > nohive classifier. We can

Re: [DISCUSS] Releasing "fat" and "slim" Flink distributions

2020-04-15 Thread Jingsong Li
Big +1. I like "fat" and "slim". For csv and json, like Jark said, they are quite small and don't have other dependencies. They are important to kafka connector, and important to upcoming file system connector too. So can we move them to both "fat" and "slim"? They're so important, and they're so

Re: [DISCUSS] Releasing "fat" and "slim" Flink distributions

2020-04-15 Thread Jingsong Li
> > > connectors and formats > > > into our "lib" directory, like kafka, csv, json metioned above, and > still > > > leave some other connectors out of it. > > > If this is the case, then why not we just provide this distribution to > > > user

Re: [PROPOSAL] Contribute training materials to Apache Flink

2020-04-15 Thread Jingsong Li
+1. It's very useful for Flink newcomers. Best, Jingsong Lee On Wed, Apr 15, 2020 at 10:23 PM Yun Tang wrote: > +1 for this idea. > > I think there would existed many details to discuss once community ready > to host the materials: > >1. How to judge whether a lab exercise should be added?

Re: [DISCUSS] Generating java code?

2020-04-15 Thread Jingsong Li
Hi Till, +1 to define an interface and load it at runtime if we can do. No disrupting the workflows of devs and throw an exception with good description look good to me. This also force us to do a good dependent class abstract. Best, Jingsong Lee On Wed, Apr 15, 2020 at 10:31 PM Till Rohrmann w

Re: [ANNOUNCE] New Apache Flink PMC Member - Hequn Chen

2020-04-19 Thread Jingsong Li
Congratulations Hequn! Best, Jingsong Lee On Mon, Apr 20, 2020 at 9:52 AM jincheng sun wrote: > Congratulations and welcome on board Hequn! > > Best, > Jincheng > > > > Zhijiang 于2020年4月19日周日 下午10:47写道: > > > Congratulations, Hequn! > > > > Best, > > Zhijiang > > > > > > --

Re: Reading a single input file in parallel?

2020-04-23 Thread Jingsong Li
Hi Niels, Thanks for start this discussion. Share some thought about your questions/proposals. > Judge whether splittable for each individual file Looks good to me. > BZIP2 support splittable Looks good to me. > the Flink implementation is controlled by the number of splits Can you check again?

Re: [DISCUSS] Exact feature freeze date

2020-04-23 Thread Jingsong Li
+1 Best, Jingsong Lee On Fri, Apr 24, 2020 at 2:27 AM Zhu Zhu wrote: > +1 for extending the code freeze date. > FLIP-119 could benefit from it. > May 15th sounds reasonable. > > Thanks, > Zhu Zhu > > Jark Wu 于2020年4月24日周五 上午12:01写道: > > > +1 > > > > Thanks, > > Jark > > > > On Thu, 23 Apr 2020

Re: [ANNOUNCE] Apache Flink 1.9.3 released

2020-04-26 Thread Jingsong Li
Thanks Dian for managing this release! Best, Jingsong Lee On Sun, Apr 26, 2020 at 7:17 PM Jark Wu wrote: > Thanks Dian for being the release manager and thanks all who make this > possible. > > Best, > Jark > > On Sun, 26 Apr 2020 at 18:06, Leonard Xu wrote: > > > Thanks Dian for the release a

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

2019-09-24 Thread Jingsong Li
Just to clarify: FLIP wiki: https://cwiki.apache.org/confluence/display/FLINK/FLIP-63%3A+Rework+table+partition+support Discussion thread: http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-FLIP-63-Rework-table-partition-support-td32770.html Google Doc: https://docs.googl

Re: Re: [DISCUSS] FLIP-65: New type inference for Table API UDFs

2019-10-08 Thread Jingsong Li
Thanks Timo for your pretty nice proposal, big +1 to the FLIP. Left some minor comments. A minor concern about flink-planner, precision things maybe cannot be supported. Best, Jingsong Lee On Tue, Oct 8, 2019 at 5:58 PM zha...@lenovocloud.com < zha...@lenovocloud.com> wrote: > unsubscribe > > >

Re: [VOTE] FLIP-57: Rework FunctionCatalog, latest updated

2019-10-10 Thread Jingsong Li
+1 Best, Jingsong Lee On Thu, Oct 10, 2019 at 3:38 PM Jark Wu wrote: > +1 > > Thanks, > Jark > > On Wed, 9 Oct 2019 at 01:03, Xuefu Z wrote: > > > +1 > > > > On Tue, Oct 8, 2019 at 7:00 AM Aljoscha Krettek > > wrote: > > > > > +1 > > > > > > > On 8. Oct 2019, at 15:35, Timo Walther wrote: >

Re: [PROPOSAL] Contribute Stateful Functions to Apache Flink

2019-10-11 Thread Jingsong Li
Hi Stephan, big +1 for this contribution. It provides another user interface that is easy to use and popular at this time. these functions, It's hard for users to write in SQL/TableApi, while using DataStream is too complex. (We've done some stateFun kind jobs using DataStream before). With statef

Re: [VOTE] Release 1.9.1, release candidate #1

2019-10-12 Thread Jingsong Li
+1 (non-binding) - Check if checksums files match the corresponding release files - Check if GPG files match the corresponding release files - Verify that the source archives do not contains any binaries - Build the source with Maven to ensure all source files have Apache headers - Check that all

Re: Re: [VOTE] Accept Stateful Functions into Apache Flink

2019-10-23 Thread Jingsong Li
+1 (non-binding) Best, Jingsong Lee On Wed, Oct 23, 2019 at 9:02 PM Yu Li wrote: > +1 (non-binding) > > Best Regards, > Yu > > > On Wed, 23 Oct 2019 at 16:56, Haibo Sun wrote: > > > +1 (non-binding)Best, > > Haibo > > > > > > At 2019-10-23 09:07:41, "Becket Qin" wrote: > > >+1 (binding) > > >

Re: [VOTE] FLIP-65: New type inference for Table API UDFs

2019-10-23 Thread Jingsong Li
+1 (non-binding) Best, Jingsong Lee On Mon, Oct 21, 2019 at 11:36 PM Rong Rong wrote: > +1 (binding) > > Thanks Timo for driving this. > > -- > Rong > > On Mon, Oct 21, 2019 at 8:19 AM wrote: > > > +1 (binding) > > > > Best, > > Xingcan > > > > -Original Message- > > From: jincheng sun

Re: [DISUCSS] FLIP-80: Expression String Serializable and Deserializable

2019-10-23 Thread Jingsong Li
Thanks Jark for your proposal. If we introduce a new kind of string presentation for expression, we will have 3 string presentation now: 1. Java expression string api. We have PlannerExpressionParser to parse string to Expressions. 2. Sql string presentation, as you said, we can use calcite classe

Re: [DISUCSS] FLIP-80: Expression String Serializable and Deserializable

2019-10-24 Thread Jingsong Li
des. > > > > Best, > > Jark > > > > [1]: > http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-Introduction-of-a-Table-API-Java-Expression-DSL-td27787.html > < > http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-Int

Re: [Discussion] FLIP-79 Flink Function DDL Support

2019-10-24 Thread Jingsong Li
Hi Peter, Thanks for your proposal. The first thing I care about most is whether it can cover the needs of hive. Hive create function: CREATE FUNCTION [db_name.]function_name AS class_name [USING JAR|FILE|ARCHIVE 'file_uri' [, JAR|FILE|ARCHIVE 'file_uri'] ]; Hive support a list of resources, a

Re: [VOTE] FLIP-70: Flink SQL Computed Column Design

2019-10-28 Thread Jingsong Li
+1 (non-binding) Best, Jingsong Lee On Mon, Oct 28, 2019 at 2:48 PM Jark Wu wrote: > Thanks for driving this Danny, > > +1 (binding) > > Best, > Jark > > > On Mon, 28 Oct 2019 at 14:26, Danny Chan wrote: > > > Hi all, > > > > I would like to start the vote for FLIP-70[1] which is discussed and

Re: [ANNOUNCE] Becket Qin joins the Flink PMC

2019-10-28 Thread Jingsong Li
Congratulations Becket! Best, Jingsong Lee On Tue, Oct 29, 2019 at 10:18 AM Terry Wang wrote: > Congratulations, Becket! > > Best, > Terry Wang > > > > > 2019年10月29日 10:12,OpenInx 写道: > > > > Congratulations Becket! > > > > On Tue, Oct 29, 2019 at 10:06 AM Zili Chen wrote: > > > >> Congratula

Re: [DISCUSS] Move flink-orc to flink-formats from flink-connectors

2019-10-29 Thread Jingsong Li
Hi bowen, I think the reason is that flink-orc was developed earlier than flink-formats. Thanks the responses of everyone, I will create JIRA to do it. On Wed, Oct 30, 2019 at 10:27 AM Danny Chan wrote: > +1 to move to flink-format. > > Best, > Danny Chan > 在 2019年10月29日 +0800 AM11:10,dev@flink

Re: [DISCUSS] Move flink-orc to flink-formats from flink-connectors

2019-10-29 Thread Jingsong Li
> [1]: > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=120731026 > > On Wed, 30 Oct 2019 at 11:12, Jingsong Li wrote: > > > Hi bowen, > > I think the reason is that flink-orc was developed earlier than > > flink-formats. > > > > Thanks

[VOTE] Move flink-orc to flink-formats from flink-connectors

2019-10-29 Thread Jingsong Li
Hi all: We already have the parent model of formats. we have put other formats(flink-avro, flink-json, flink-parquet, flink-json, flink-csv, flink-sequence-file) to flink-formats. flink-orc is a format too. So we can move it to flink-formats. In theory, there should be no compatibility problem, o

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

2019-11-04 Thread Jingsong Li
+1 for the idea. Thanks Yu for driving this. Just curious about that can we collect the metrics about Job scheduling and task launch. the speed of this part is also important. We can add tests for watch it too. Look forward to more batch test support. Best, Jingsong Lee On Mon, Nov 4, 2019 at 10

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

2019-11-08 Thread Jingsong Li
Thanks Kurt and Terry for bringing this to my attention. Agree each topic of this FLIP touched is super big. But because it's too big, we can't cover everything in one FLIP. So I think there are two ways: 1. Decompose this big thing into an independent topic(FLIP), so that each FLIP can be explain

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

2019-11-08 Thread Jingsong Li
Thanks Terry for driving this. +1 With these features, SQL-CLI could be really used. Best, Jingsong Lee On Fri, Nov 8, 2019 at 2:20 PM Kurt Young wrote: > 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

Re: [ANNOUNCE] Jark Wu is now part of the Flink PMC

2019-11-08 Thread Jingsong Li
Congratulations to Jark. Jark has really contributed a lot to the table layer with a long time. Well deserved. Best, Jingsong Lee On Fri, Nov 8, 2019 at 6:05 PM Yu Li wrote: > Congratulations Jark! Well deserved! > > Best Regards, > Yu > > > On Fri, 8 Nov 2019 at 17:55, OpenInx wrote: > > > Co

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

2019-11-11 Thread Jingsong Li
+1 (non-binding) Best, Jingsong Lee On Tue, Nov 12, 2019 at 9:49 AM vino yang wrote: > +1 (non-binding) > > Best, > Vino > > Xuefu Z 于2019年11月12日周二 上午3:27写道: > > > +1 (non-binding) > > > > On Mon, Nov 11, 2019 at 9:54 AM Shuyi Chen wrote: > > > > > +1 (binding) > > > > > > On Sat, Nov 9, 2019

Re: Flink Read thousands of files with batch

2019-11-12 Thread Jingsong Li
Hi Dominik: I found a problem too that it maybe your root cause.[1] JobConf in HadoopInputSplit may very big, contains hundreds of configurations, if it is serialized by every split, that will significantly reduce performance. Consider thousands of splits, the akka thread of JobMaster will all on

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

2019-11-14 Thread Jingsong Li
+1 (non-binding) Thanks Dawid for driving this. Best, Jingsong Lee On Thu, Nov 14, 2019 at 6:09 PM Gyula Fóra wrote: > +1 (binding) > > On Thu, Nov 14, 2019 at 11:07 AM Jeff Zhang wrote: > > > +1 (non-binding) > > > > Kostas Kloudas 于2019年11月14日周四 下午6:04写道: > > > > > +1 (binding) > > > > > >

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

2019-11-18 Thread Jingsong Li
+1 (non-binding) Best, Jingsong Lee On Mon, Nov 18, 2019 at 7:59 PM Becket Qin wrote: > +1 (binding) on having the test suite. > > BTW, it would be good to have a few more details about the performance > tests. For example: > 1. How do the testing records look like? The size and key distributio

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 use

Re: [VOTE] FLIP-86: Improve Connector Properties

2019-11-20 Thread Jingsong Li
+1 (non-binding) Thanks Jark for driving this. Best, Jingsong Lee On Wed, Nov 20, 2019 at 6:25 PM Jark Wu wrote: > +1 (binding) > > Best, > Jark > > On Wed, 20 Nov 2019 at 18:11, Terry Wang wrote: > > > +1 (non-binding) > > > > Best, > > Terry Wang > > > > > > > > > 2019年11月20日 17:47,Dawid Wys

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

2019-11-21 Thread Jingsong Li
+1 (non-binding) - verified the signature and checksum - checked the maven central artifices - checked "release-9.0-rc1" source code - verified website pull request listing the new release Best, Jingsong Lee On Thu, Nov 21, 2019 at 5:28 PM Dian Fu wrote: > +1 (non-binding) > > - verified the s

Re: [DISCUSS] Disable conversion between TIMESTAMP and Long in parameters and results of UDXs

2019-11-22 Thread Jingsong Li
+1 to disable, It is already introduced by new type system in TimestampType. I think it is time to update document too. Best, Jingsong Lee On Fri, Nov 22, 2019 at 6:05 PM Kurt Young wrote: > +1 to disable, we also need to highlight this in 1.10 release notes. > > Best, > Kurt > > > On Fri, Nov

Re: FLIP 87: Primary key constraints in Table API

2019-11-22 Thread Jingsong Li
+1 thanks dawid for driving this. Best Jingsong Lee Timo Walther : > +1 > > Thanks, > Timo > > > On 23.11.19 00:28, Xuefu Z wrote: > > +1 > > > > On Fri, Nov 22, 2019 at 5:31 AM Kurt Young wrote: > > > >> +1 > >> > >> Best, > >> Kurt > >> > >> > >> On Fri, Nov 22, 2019 at 8:51 PM Dawid Wysakowi

Re: [DISCUSS] Make Managed Memory always off-heap (Adjustment to FLIP-49)

2019-11-26 Thread Jingsong Li
Hi Stephan, +1 to default have off-heap managed memory. >From the perspective of batch, In our long-term performance test and online practice: - There is no significant difference in performance between heap and off-heap memory. If it is a heap object, the JVM has many opportunities to optimize i

Re: [DISCUSS] Multi-topics consuming from KafkaTableSource

2019-11-26 Thread Jingsong Li
Hi leo: Can you share why KafkaSource supports multiple topics? Why can't we use multiple sources and then union them? Best, Jingsong Lee On Wed, Nov 27, 2019 at 8:19 AM leo wrote: > Hi, > > As Flink does not own the data, so it can not forbid this. It's the > responsibility of user to deseria

Re: Re: [DISCUSS] Support JSON functions in Flink SQL

2019-12-01 Thread Jingsong Li
Hi Forward: Document looks good to me. I think you can just start doing this. They all work very independently, so I don't think there's any obvious blocking. Best, Jingsong Lee On Sat, Nov 30, 2019 at 10:59 AM Forward Xu wrote: > Hi everyone, It's been a long time since I started this discuss

Re: [VOTE] FLIP-88: PyFlink User-Defined Function Resource Management

2019-12-01 Thread Jingsong Li
Sorry for bothering your voting. Let's discuss in discussion thread. Best, Jingsong Lee On Mon, Dec 2, 2019 at 3:32 PM Jingsong Lee wrote: > Hi Dian: > > Thanks for your driving. I have some questions: > > - Where should these configurations belong? You have mentioned > tableApi/SQL, so should

Re: [VOTE] Release 1.8.3, release candidate #2

2019-12-02 Thread Jingsong Li
Hi Hequn, +1 (non-binding) Thank you for driving this. - Verified signatures and checksums - Maven build from source skip tests (Scala 2.11 and Scala 2.12) - Start local cluster and web ui is accessible (Scala 2.11 and Scala 2.12) - Submit WordCount example of both batch and streaming, good (Scal

Re: [VOTE] FLIP-88: PyFlink User-Defined Function Resource Management

2019-12-03 Thread Jingsong Li
; > Thanks, > Dian > > > 在 2019年12月2日,下午3:47,Jingsong Li 写道: > > > > Sorry for bothering your voting. > > Let's discuss in discussion thread. > > > > Best, > > Jingsong Lee > > > > On Mon, Dec 2, 2019 at 3:32 PM Jingsong Lee >

Re: I love Flink

2019-12-03 Thread Jingsong Li
Hi boqi, Now, you don't need permission now. For JIRA creation, you can create JIRAs by yourself to report bugs or improvements. For JIRA assginment, only committers can assign JIRA to someone, if you want to contribute some code, you can comment JIRA, and some committers can help you to assign th

Re: [DISCUSS] Voting from apache.org addresses

2019-12-04 Thread Jingsong Li
Hi Dawid, Thanks for you explanation. You mean that "people who vote for binding +1 need to use apache email addresses". Although I can't vote binding +1, I am +1 for this suggestion. Best, Jingsong Lee On Wed, Dec 4, 2019 at 4:04 PM Dawid Wysakowicz wrote: > Hi all, > > Sorry I think I was no

Re: [DISCUSS] Add N-Ary Stream Operator

2019-12-04 Thread Jingsong Li
Hi Piotr, Huge +1 for N-Ary Stream Operator. And I love this Golden Shovel award very much! There are a large number jobs (in production environment) that their TwoInputOperators that can be chained. We used to only watch the last ten tasks transmit data through disk and network, which could have

Re: [DISCUSS] Add N-Ary Stream Operator

2019-12-05 Thread Jingsong Li
. > > > > Provide an N-Ary stream operator to make everything possible. The upper > > layer can do anything. These things can be specific optimization, which > is much > > more natural than the lower layer. > > Do you mean that those Table API/SQL use cases (HashJ

Re: [DISCUSS] Add N-Ary Stream Operator

2019-12-05 Thread Jingsong Li
utput > before `Build2` finished. I don't have a clear solution for now, but > it's a common pattern we will face. > > Best, > Kurt > > > On Thu, Dec 5, 2019 at 4:37 PM Jingsong Li wrote: > > > Hi Piotr, > > > > > a) two input operator X -&g

Re: [DISCUSS] PyFlink User-Defined Function Resource Management

2019-12-05 Thread Jingsong Li
Hi Dian, After [1] and [2], in the batch sql world, we will: - [2] In client/compile side: we use memory weight request memory for Transformation. - [1] In runtime side: we use memory fraction to compute memory size and allocate in StreamOperator. For your information. [1] https://jira.apache.org

Re: [DISCUSS] Drop Kafka 0.8/0.9

2019-12-08 Thread Jingsong Li
Thanks Chesnay, +1 to make it official that we no longer actively develop them but user can still use. Best, Jingsong Lee On Mon, Dec 9, 2019 at 4:47 AM Chesnay Schepler wrote: > Users can continue to use the 1.9 versions of the 0.8/0.9 connectors > with future versions of Flink. > > We just m

Re: [VOTE] Release 1.8.3, release candidate #3

2019-12-08 Thread Jingsong Li
Hi Hequn, +1 (non-binding) Thank you for driving this. - Verified signatures and checksums - Maven build from source skip tests (Scala 2.11 and Scala 2.12) - Start local cluster and web ui is accessible (Scala 2.11 and Scala 2.12) - Submit WordCount example of both batch and streaming, good (Scal

Re: [DISCUSS] Add N-Ary Stream Operator

2019-12-09 Thread Jingsong Li
t’s why I’m not > saying that we do not want to handle this at some point in the future, but > at least not in the first version. > > Piotrek > > > On 5 Dec 2019, at 10:11, Jingsong Li wrote: > > > > Kurt mentioned a very interesting thing, > > > > If we

Re: [DISCUSS] FLIP-27: Refactor Source Interface

2019-12-09 Thread Jingsong Li
Hi all, I think current design is good. My understanding is: For execution mode: bounded mode and continuous mode, It's totally different. I don't think we have the ability to integrate the two models at present. It's about scheduling, memory, algorithms, States, etc. we shouldn't confuse them.

Re: [ANNOUNCE] Feature freeze for Apache Flink 1.10.0 release

2019-12-10 Thread Jingsong Li
Thanks Yu & Gary for your great work. 1.10 is a very important version for batch sql (also streaming sql). It enables many features to be put into production. Looking forward to the 1.10 release. Let's "beat" it by tests. Best, Jingsong Lee On Wed, Dec 11, 2019 at 2:55 PM Wei Zhong wrote: > Hi

Re: [DISCUSS] Register user jar files in {Stream}ExecutionEnvironment

2019-12-10 Thread Jingsong Li
Hi Leo, I think run job with external jars is important too. Have you took a look to PipelineOptions.JARS in configuration? I think this is a way to set external jars. And SQL-CLI need it too. Best, Jingsong Lee On Wed, Dec 11, 2019 at 9:18 AM 50man wrote: > Hi everyone, > > > I propose an imp

[DISCUSS] Overwrite and partition inserting support in 1.10

2019-12-11 Thread Jingsong Li
Hi Dev, After cutting out the branch of 1.10, I tried the following functions of SQL-CLI and found that it does not support: - insert overwrite - PARTITION (partcol1=val1, partcol2=val2 ...) The SQL pattern is: INSERT { INTO | OVERWRITE } TABLE tablename1 [PARTITION (partcol1=val1, partcol2=val2 .

Re: [DISCUSS] Overwrite and partition inserting support in 1.10

2019-12-11 Thread Jingsong Li
> >> alone. It's just unfortunately that we may have forgot to revisit this > >> topic and we apparently missed some test cases on SQL CLI side. Sorry > for > >> that. > >> > >> From a product perspective, SQL CLI is super crit

Re: [DISCUSS] FLIP-27: Refactor Source Interface

2019-12-11 Thread Jingsong Li
method which returns the currently retrieved records and then the > caller > > >> emits them outside of the SourceReader. That way the interface would > not > > >> allow to implement an outputting loop where we never hand back control > > to > > >> the

Re: [ANNOUNCE] Apache Flink 1.8.3 released

2019-12-11 Thread Jingsong Li
Thanks Hequn for your driving, 1.8.3 fixed a lot of issues and it is very useful to users. Great work! Best, Jingsong Lee On Thu, Dec 12, 2019 at 3:25 PM jincheng sun wrote: > Thanks for being the release manager and the great work Hequn :) > Also thanks to the community making this release pos

Re: [DISCUSS] Overwrite and partition inserting support in 1.10

2019-12-12 Thread Jingsong Li
ve the hive dialect limitations for INSERT OVERWRITE and INSERT > > PARTITION > > +0 to add yaml dialect conf to SQL-CLI because FLIP-89 is not finished > > yet, we better do this until FLIP-89 is resolved. > > > > Best, > > Danny Chan > > 在 2019年12月

Re: [DISCUSS] Overwrite and partition inserting support in 1.10

2019-12-12 Thread Jingsong Li
ION are > > under the dialect limitation, but CREATE PARTITION TABLE is not. > > > > So it is indeed a bug which should be fixed in 1.10. > > > > Best, > > Jark > > > > On Thu, 12 Dec 2019 at 16:35, Jingsong Li > wrote: > > > >> Hi Jar

Re: [DISCUSS] Overwrite and partition inserting support in 1.10

2019-12-13 Thread Jingsong Li
has pointed out, the current implementation violates the accepted > > design of FLIP-63, which IMO qualifies as a bug. Given that it's a bug > and > > has great impact on the usability of our Hive integration, do you think > we > > can fix it in 1.10? > >

Re: [DISCUSS] have separate Flink distributions with built-in Hive dependencies

2019-12-13 Thread Jingsong Li
Hi Bowen, Thanks for driving this. +1 for this proposal. Due to our multi version support, users are required to rely on different dependencies, it does break the "out of box" experience. Now that the client has changed to go to child first class loader resolve by default, it puts forward higher

Re: [ANNOUNCE] Zhu Zhu becomes a Flink committer

2019-12-15 Thread Jingsong Li
Congratulations Zhu Zhu! Best, Jingsong Lee On Mon, Dec 16, 2019 at 10:01 AM Yang Wang wrote: > Congratulations, Zhu Zhu! > > wenlong.lwl 于2019年12月16日周一 上午9:56写道: > > > Congratulations, Zhu Zhu! > > > > On Mon, 16 Dec 2019 at 09:14, Leonard Xu wrote: > > > > > Congratulations, Zhu Zhu ! ! > >

Re: NPE in blink planner code gen

2019-12-15 Thread Jingsong Li
Hi Benchao, Thanks for your reporting. As Leonard said, you can create an issue in JIRA. We can go on discussing in JIRA. The answer is #1, Blink's behavior ensures that the job runs as much as possible without interrupting it, so null is returned here to make it possible to continue running. In

Re: [DISCUSS] have separate Flink distributions with built-in Hive dependencies

2019-12-15 Thread Jingsong Li
Thanks all for explaining. I misunderstood the original proposal. -1 to put them in our distributions +1 to have provide hive uber jars as Seth and Aljoscha advice Hive is just a connector no matter how important it is. So I totally agree that we shouldn't put them in our distributions. We can st

Re: [DISCUSS] Overwrite and partition inserting support in 1.10

2019-12-15 Thread Jingsong Li
nd thought or debate on > anything, a vote is required to overturn it. Before that, the original FLIP > holds. > > Thanks, > Xuefu > > > On Fri, Dec 13, 2019 at 1:20 AM Jingsong Li > wrote: > > > Hi Timo, > > > > Thanks for your feedback. > > &

Re: Potential side-effect of connector code to JM/TM

2019-12-18 Thread Jingsong Li
Thanks Yingjie for driving. It is very useful to have this check list. I think we can list all problematic third-party libraries. Including hadoop jar: org.apache.hadoop.fs.FileSystem.StatisticsDataReferenceCleaner. Because there are too many libraries with this problem. And our Yarn mode perJob

Re: [DISCUSS] Correct the terminology of "Time-windowed Join" to "Interval Join" in Table API & SQL

2019-12-22 Thread Jingsong Li
Thanks Jark for bringing this. +1 to use a unify name: "Interval Join" before 1.10 is release. I think maybe "Interval Join" was come from SQL world too in [1]. Another candidate is to use "Range Join", But considering DataStream, I am OK with "Interval". [1] https://issues.apache.org/jira/brows

Re: [DISCUSS] Correct the terminology of "Time-windowed Join" to "Interval Join" in Table API & SQL

2019-12-22 Thread Jingsong Li
Hi Danny, > DatasStream interval join and Table/SQL Time-windowed Join are not equivalent In my opinion, there is no difference between table and DataStream except that outer join is not implemented in DataStream. KeyedStream has defined equivalent conditions. Other conditions can be completed in

Re: [jira] [Created] (FLINK-15017) add a thrift jdbc/odbc server for Flink

2019-12-23 Thread Jingsong Li
Hi gteeice, CC: godfrey and tsreaper. I know they are considering and designing JDBC support. I believe there will be a lot of progress on 1.11. Best, Jingsong Lee On Tue, Dec 24, 2019 at 11:07 AM 唐晨阳 wrote: > Is there any detailed plan or progress regarding jdbc support? > On 12/3/2019 04:41,

Re: Apply for contributor permission

2019-12-24 Thread Jingsong Li
Hi Pu, Only Flink committers can assign issue. So you can ask some committers to assign issue to you. Best, Jingsong Lee On Wed, Dec 25, 2019 at 11:38 AM pu zhang wrote: > it seems not work, i have no right to assign a issue to me, any other > questions? Please help check again ,thank you

Re: What is the suggested way to validate SQL?

2019-12-29 Thread Jingsong Li
Hi Kaibo, > Validate SQL syntax not need to depend on connector jar At present, sql function strongly need jar dependent support too , but the overall approach is still under discussion, and there is no clear plan at present. But you are right, it really important for platform users. Another way i

Re: [DISCUSS] Some feedback after trying out the new FLIP-49 memory configurations

2019-12-29 Thread Jingsong Li
Thank you for your wonderful discussion. +1 for set "taskmanager.memory.flink.size" in the default config. Maybe we can write and explain "taskmanager.memory.process.size" in the comments. Most of the time, - "trying out" users is less in-depth users and use standalone mode. - Production users us

Re: [ANNOUNCE] Progress of Apache Flink 1.10 #3

2020-01-02 Thread Jingsong Li
Thanks Yu and Gary, Table and hive layer have resolved all blockers and critical issues. (38 blockers and 33 critical issues) Look forward to the first RC. Best, Jingsong Lee On Fri, Jan 3, 2020 at 11:16 AM Danny Chan wrote: > Thanks for the great work ~ > > Best, > Danny Chan > 在 2020年1月2日 +

Re: [VOTE] Rename terminology "Time-windowed Join" to "Interval Join" in Table API & SQL

2020-01-02 Thread Jingsong Li
+1 for this documentation change. Hope less confuse to users. Best, Jingsong Lee On Fri, Jan 3, 2020 at 12:09 PM Benchao Li wrote: > +1 > > It's good to align the terminology between Table API & SQL and DataStream. > > Jark Wu 于2020年1月3日周五 下午12:04写道: > > > Hi everyone, > > > > As we discussed

Re: [DISCUSS] Set default planner for SQL Client to Blink planner in 1.10 release

2020-01-03 Thread Jingsong Li
Hi Jark, +1 for default blink planner in SQL-CLI. I believe this new planner can be put into practice in production. We've worked hard for nearly a year, but the old planner didn't move on. And I'd like to cc to u...@flink.apache.org. If anyone finds that blink planner has any significant defects

Re: [DISCUSS] JDBC exactly-once sink

2020-01-06 Thread Jingsong Li
Thanks Roman for driving this. Although Upsert sink can bring some consistency guarantee to JDBC sink, users have strong requirements to append exactly-once sink too. +1 to use XA distribution transaction. For WAL, as far as my experience is concerned, writing JDBC in large quantities when check

Re: [DISCUSS] JDBC exactly-once sink

2020-01-06 Thread Jingsong Li
s it's just in > Table API, right? > > Piotrek > > > On 6 Jan 2020, at 10:43, Jingsong Li wrote: > > > > Thanks Roman for driving this. > > > > Although Upsert sink can bring some consistency guarantee to JDBC sink, > > users have strong requireme

Re: [VOTE] FLIP-90: Support SQL 2016-2017 JSON functions in Flink SQL

2020-01-06 Thread Jingsong Li
+1 non-binding. Thanks Forward for driving this. Considering that it is made up of independent and certain things from SQL standard and Calcite, I think it can be started as soon as possible. Best, Jingsong Lee On Tue, Dec 31, 2019 at 5:09 PM Forward Xu wrote: > Hi all, > > I'd like to start t

Re: [DISCUSS] FLIP-92: JDBC catalog and Postgres catalog

2020-01-08 Thread Jingsong Li
Thanks Bowen for driving this, +1 for this, The DDL schema definition is a headache for users, and catalog is a solution to this problem. I have some questions and suggestions: - We can provide a Builder for Catalog, In my opinion, defaultDatabase, username, pwd can be included in JDBC DB url.

Re: [VOTE] FLIP-92: Add N-Ary Stream Operator in Flink

2020-01-09 Thread Jingsong Li
+1 non-binding to the N-Ary Stream Operator. Thanks Piotr for driving. Looks like the previous FLIP-92 did not change the "Next FLIP Number" in FLIP page. Best, Jingsong Lee On Fri, Jan 10, 2020 at 8:40 AM Benchao Li wrote: > Hi Piotr, > > It seems that we have the 'FLIP-92' already. > see: > >

Re: [DISCUSS] FLIP-92: JDBC catalog and Postgres catalog

2020-01-09 Thread Jingsong Li
t; > > > For the different params besides schema, as we discussed offline, > > unfortunately we can't do anything right now until Flink DDL/DML are able > > to distinguish 3 types of params - external data's metada, source/sink > > runtime params, and Flink semantics

Re: [DISCUSS] A mechanism to validate the precision of columns for connectors

2020-01-09 Thread Jingsong Li
Hi Zhenghua, I think it's not just about precision of type. Connectors not validate the types either. Now there is "SchemaValidator", this validator is just used to validate type properties. But not for connector type support. I think we can have something like "DataTypeValidator" to help connecto

[DISCUSS] Improve TableFactory

2020-01-14 Thread Jingsong Li
Hi dev, I'd like to kick off a discussion on the improvement of TableSourceFactory and TableSinkFactory. Motivation: Now the main needs and problems are: 1.Connector can't get TableConfig [1], and some behaviors really need to be controlled by the user's table configuration. In the era of catalog

<    1   2   3   4   5   6   7   >