Re: [DISCUSS] Enhance Support for Multicast Communication Pattern

2019-08-26 Thread Kurt Young
>From SQL's perspective, distributed cross join is a valid feature but not very urgent. Actually this discuss reminds me about another useful feature (sorry for the distraction): when doing broadcast in batch shuffle mode, we can make each producer only write one copy of the output data, but not f

Re: [DISCUSS] Setup a bui...@flink.apache.org mailing list for travis builds

2019-08-26 Thread Kurt Young
Thanks for the updates, Jark! I have subscribed the ML and everything looks good now. Best, Kurt On Mon, Aug 26, 2019 at 11:17 AM Jark Wu wrote: > Hi all, > > Sorry it take so long to get back. I have some good news. > > After some investigation and development and the help from Chesnay, we >

Re: [DISCUSS] Enhance Support for Multicast Communication Pattern

2019-08-26 Thread Kurt Young
_state.html > < > https://ci.apache.org/projects/flink/flink-docs-stable/dev/stream/state/broadcast_state.html > > > [2] https://github.com/apache/flink/pull/7713 < > https://github.com/apache/flink/pull/7713> > > > On 26 Aug 2019, at 09:35, Kurt Young wrote: > > > &

Re: [DISCUSS] Setup a bui...@flink.apache.org mailing list for travis builds

2019-08-28 Thread Kurt Young
one suggestion: we could also filter all notifications about *Cancelled* builds. Best, Kurt On Tue, Aug 27, 2019 at 10:53 AM jincheng sun wrote: > Great Job Jark :) > Best, Jincheng > > Kurt Young 于2019年8月26日周一 下午6:38写道: > > > Thanks for the updates, Jark! I have

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: [DISCUSS] FLIP-60: Restructure the Table API & SQL documentation

2019-09-02 Thread Kurt Young
+1 to the general idea and thanks for driving this. I think the new structure is more clear than the old one, and i have some suggestions: 1. How about adding a "Architecture & Internals" chapter? This can help developers or users who want to contribute more to have a better understanding about Ta

Re: [DISCUSS] FLIP-53: Fine Grained Resource Management

2019-09-02 Thread Kurt Young
Thanks Xingtong for driving this effort, I haven't finished the whole document yet, but have couple of questions: 1. Regarding to network memory, the document said it will be derived by framework automatically. I'm wondering whether we should delete this dimension from user- facing API? 2. Regard

Re: [DISCUSS] FLIP-57 - Rework FunctionCatalog

2019-09-03 Thread Kurt Young
Thanks Bowen for driving this. +1 for the general idea. It makes the function resolved behavior more clear and deterministic. Besides, the user can use all hive built-in functions, which is a great feature. I only have one comment, but maybe it may touch your design so I think it would make sense

Re: [DISCUSS] FLIP-57 - Rework FunctionCatalog

2019-09-03 Thread Kurt Young
Does this only affect the functions and operations we currently have in SQL and have no effect on tables, right? Looks like this is an orthogonal concept with Catalog? If the answer are both yes, then the catalog function will be a weird concept? Best, Kurt On Tue, Sep 3, 2019 at 8:10 PM Danny C

Re: [DISCUSS] FLIP-57 - Rework FunctionCatalog

2019-09-03 Thread Kurt Young
catalog". > >>> > >>> Yes, I've unified #3 and #4 but it seems I didn't update some part of > >>> the doc. I've modified those sections, and they are up to date now. > >>> > >>> In short, now built-in function of e

Re: [DISCUSS] Support JSON functions in Flink SQL

2019-09-04 Thread Kurt Young
+1 to add JSON support to Flink. We also see lots of requirements for JSON related functions in our internal platform. Since these are already SQL standard, I think it's a good time to add them to Flink. Best, Kurt On Thu, Sep 5, 2019 at 10:37 AM Qi Luo wrote: > We also see strong demands from

Re: [ANNOUNCE] Kostas Kloudas joins the Flink PMC

2019-09-06 Thread Kurt Young
Congratulations Klou! Best, Kurt On Sat, Sep 7, 2019 at 2:37 PM ying wrote: > Congratulations Kostas! > > On Fri, Sep 6, 2019 at 11:21 PM Gary Yao wrote: > > > Congratulations Klou! > > > > On Sat, Sep 7, 2019 at 6:21 AM Thomas Weise wrote: > > > > > Congratulations! > > > > > > > > > On Fri

Re: [VOTE] FLIP-53: Fine Grained Operator Resource Management

2019-09-06 Thread Kurt Young
+1 for FLIP-53. I would like to raise one minor concern regarding to implementing request absolute amount of memory case. Currently, it will be translated to a memory fraction during compile, and translate back to absolute value during execution. There is a risk that the user might get less than h

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

2019-09-09 Thread Kurt Young
+1 (binding) - build from source and passed all tests locally - checked the difference between 1.8.1 and 1.8.2, no legal risk found - went through all commits checked in between 1.8.1 and 1.8.2, make sure all the issues set the proper "fixVersion" property Best, Kurt On Mon, Sep 9, 2019 at 8:45

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

2019-09-16 Thread Kurt Young
+1 to this feature, I left some comments on google doc. Another comment is I think we should do some reorganize about the content when you converting this to a cwiki page. I will have some offline discussion with you. Since this feature seems to be a fairly big efforts, so I suggest we can settle

Re: How stable is FlinkSQL.

2019-09-16 Thread Kurt Young
Hi Srikanth, AFAIK, there are quite some companies already using Flink streaming SQL to back their production systems, like realtime data warehouse. If you met some issues when trying streaming sql, I would suggest you to send the problem to user@ml, where you can receive some helps. Best, Kurt

Re: [DISCUSS] FLIP-66: Support time attribute in SQL DDL

2019-09-16 Thread Kurt Young
After some review and discussion in the google document, I think it's time to convert this design to a cwiki flip page and start voting process. Best, Kurt On Mon, Sep 9, 2019 at 7:46 PM Jark Wu wrote: > Hi all, > > Thanks all for so much feedbacks received in the doc so far. > I saw a general

Re: How stable is FlinkSQL.

2019-09-16 Thread Kurt Young
t;> >> Many of them are cases of Flink-SQL. >> >> >> Best, >> >> Forward >> >> srikanth flink 于2019年9月16日周一 下午9:39写道: >> >> > Hi Kurt, >> > >> > thanks for quick response. Is the email user@ml? >> > >

Re: [DISCUSS] FLIP-60: Restructure the Table API & SQL documentation

2019-09-16 Thread Kurt Young
ke this? > > > > SQL > > - Overview > > - Data Manipulation Statements (all operations available in SQL) > > - Data Definition Statements (DDL syntaxes) > > - Pattern Matching > > > > It renames "Full Reference" to "Data Manipulatio

Re: Add Bucket File System Table Sink

2019-09-16 Thread Kurt Young
Hi Jun, Thanks for bringing this up, in general I'm +1 on this feature. As you might know, there is another ongoing efforts about such kind of table sink, which covered in newly proposed partition support reworking[1]. In this proposal, we also want to introduce a new file system connector, which

Re: Add Bucket File System Table Sink

2019-09-16 Thread Kurt Young
Kurt: > thank you very much. > I will take a closer look at the FLIP-63. > > I develop this PR, the underlying is StreamingFileSink, not > BuckingSink, but I gave him a name, called Bucket. > > > On 09/17/2019 10:57,Kurt Young > wrote: > > Hi Ju

Re: Add Bucket File System Table Sink

2019-09-16 Thread Kurt Young
3 and see if there is a better solution to > combine these two functions. I am very willing to join this development. > > > > -- 原始邮件 -- > *发件人:* "Kurt Young"; > *发送时间:* 2019年9月17日(星期二) 中午11:19 > *收件人:* "Jun Zhang"<825875..

Re: [DISCUSS] FLIP-64: Support for Temporary Objects in Table module

2019-09-18 Thread Kurt Young
Hi all, Sorry to join this party late. Big +1 to this flip, especially for the dropping "registerTableSink & registerTableSource" part. These are indeed legacy and we should try to unify them through CatalogTable after we introduce the concept of Catalog. >From my understanding, what we can regis

Re: [DISCUSS] FLIP-66: Support time attribute in SQL DDL

2019-09-18 Thread Kurt Young
-SecocBqzUh7zY6HBYcfMlG_0z-JAcuZkCvsmN3LrOw/edit?ts=5d8258cd > > On Mon, 16 Sep 2019 at 16:12, Kurt Young wrote: > > > After some review and discussion in the google document, I think it's > time > > to > > convert this design to a cwiki flip page and start voting pro

Re: [DISCUSS] FLIP-64: Support for Temporary Objects in Table module

2019-09-19 Thread Kurt Young
IIUC it's good to see that both serializable (tables description from DDL) and unserializable (tables with DataStream underneath) tables are treated unify with CatalogTable. Can I also assume functions that either come from a function class (from DDL) or function objects (newed by user) will also

Re: [DISCUSS] FLIP-57 - Rework FunctionCatalog

2019-09-19 Thread Kurt Young
Looks like I'm the only person who is willing to +1 to #2 for now :-) But I would suggest to change the keyword from GLOBAL to something like BUILTIN. I think #2 and #3 are almost the same proposal, just with different format to indicate whether it want to override built-in functions. My biggest

Re: [DISCUSS] FLIP-57 - Rework FunctionCatalog

2019-09-19 Thread Kurt Young
And let me make my vote complete: -1 for #1 +1 for #2 with different keyword -0 for #3 Best, Kurt On Thu, Sep 19, 2019 at 4:40 PM Kurt Young wrote: > Looks like I'm the only person who is willing to +1 to #2 for now :-) > But I would suggest to change the keyword from GLOBAL to

Re: [DISCUSS] FLIP-57 - Rework FunctionCatalog

2019-09-19 Thread Kurt Young
;> > >> So I think it is a completely new thing, and the direct way to deal with > >> new things is to add new grammar. So, > >> +1 for #2, +0 for #3, -1 for #1 > >> > >> Best, > >> Jingsong Lee > >> > >> > >> ---

Re: Add Bucket File System Table Sink

2019-09-20 Thread Kurt Young
Hi Jun, Thanks for your understanding. If we all agree adding this functionality into FLIP-63 is a good idea, I would suggest you also help reviewing the FLIP-63 design document to see if current design meet your requirements. You can also raise some comments to the design document if you have som

Re: [VOTE] FLIP-57: Rework FunctionCatalog

2019-09-23 Thread Kurt Young
+1 Best, Kurt On Tue, Sep 24, 2019 at 2:30 AM Bowen Li wrote: > Hi all, > > I'd like to start a voting thread for FLIP-57 [1], which we've reached > consensus in [2]. > > This voting will be open for minimum 3 days till 6:30pm UTC, Sep 26. > > Thanks, > Bowen > > [1] > > https://cwiki.apache.o

Re: [DISCUSS] Releasing Flink 1.9.1

2019-09-23 Thread Kurt Young
+1 for the 1.9.1 release and for Jark being the RM. Thanks Jark for the volunteering. Best, Kurt On Mon, Sep 23, 2019 at 9:17 PM Till Rohrmann wrote: > +1 for the 1.9.1 release and for Jark being the RM. I'll help with the > review of FLINK-14010. > > Cheers, > Till > > On Mon, Sep 23, 2019 at

Re: [DISCUSS] FLIP-66: Support time attribute in SQL DDL

2019-09-23 Thread Kurt Young
ield - > INTERVAL > > > 'string' timeUnit". > > > 2. Preserve Watermark From Source, the strategy can be > > > "SYSTEM_WATERMARK()". > > > > > > ## Proctime Attribute > > > > > > CREATE TABLE table_name (

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

2019-09-23 Thread Kurt Young
Looks like the wiki is not aligned with latest google doc, could you update it first? Best, Kurt On Tue, Sep 24, 2019 at 10:19 AM Jingsong Lee wrote: > Hi Flink devs, after another round of discussion. > > I would like to re-start the voting for FLIP-63 > Rework table partition support. > > FL

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

2019-09-24 Thread Kurt Young
GY/edit?usp=sharing > > Best, > Jingsong Lee > > On Tue, Sep 24, 2019 at 11:43 AM Jingsong Lee > wrote: > > > Thank you for your reminder. > > Updated. > > > > Best, > > Jingsong Lee > > > > On Tue, Sep 24, 2019 at 11:36 AM Kur

Re: [VOTE] FLIP-56: Dynamic Slot Allocation

2019-09-24 Thread Kurt Young
If it's possible, I would suggest to add one sector in this doc to emphasize that current design has a prerequisite that each job should either has all its operators using unknown resource profile or all using specified amount of resource. This would make this document easier to understand. (I was

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

2019-09-26 Thread Kurt Young
+1 Best, Kurt On Thu, Sep 26, 2019 at 11:52 AM Jark Wu wrote: > 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,

Re: [DISCUSS] FLIP-64: Support for Temporary Objects in Table module

2019-10-10 Thread Kurt Young
ight revisit that some time in the future if we > >> find that it makes more sense. > >> > >> @All I updated the FLIP page with some more details from the outcome of > >> the discussions around FLIP-57. Please take a look. I would like to > >> start a

Re: [DISCUSS] FLIP-64: Support for Temporary Objects in Table module

2019-10-10 Thread Kurt Young
JavaDocs and IDE support. But I agree > that it needs some rework in the future, once we have finalized the DDL > to ensure that both concepts are in sync. > > Regards, > Timo > > > On 10.10.19 16:08, Kurt Young wrote: > > Regarding to ConnectTableDescriptor, if in the e

Re: [DISCUSS] FLIP-64: Support for Temporary Objects in Table module

2019-10-10 Thread Kurt Young
P-64 tomorrow morning, unless there are some objections until then. > > Best, > > Dawid > > > On 10/10/2019 16:16, Kurt Young wrote: > > Thanks for the clarification Timo, that's sounds good to me. Let's continue > to discuss other things. > > Best, > Ku

Re: [VOTE] FLIP-64: Support for Temporary Objects in Table module

2019-10-14 Thread Kurt Young
+1 Best, Kurt On Fri, Oct 11, 2019 at 1:39 PM Dawid Wysakowicz wrote: > Hi everyone, > I would like to start a vote on FLIP-64. The discussion seems to have > reached an agreement. > > Please vote for the following design document: > > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-6

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

2019-10-14 Thread Kurt Young
+1 - Verify that the source archives do not contains any binaries - Start the cluster locally and ran some examples successfully Best, Kurt On Mon, Oct 14, 2019 at 4:32 AM Jark Wu wrote: > Thanks @Hequn and @Yun Tang, I set the fixVersion of FLINK-14385 to 1.8.3 > and 1.9.2. > > Btw, I would

Re: [VOTE] FLIP-77: Introduce ConfigOptions with Data Types

2019-10-15 Thread Kurt Young
+1 Best, Kurt On Tue, Oct 15, 2019 at 9:30 AM Dawid Wysakowicz wrote: > Hi everyone, > I would like to start a vote on FLIP-77. > > Please vote for the following design document: > > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-77%3A+Introduce+ConfigOptions+with+Data+Types > > > Th

Re: [DISCUSS] Rename the SQL ANY type to OPAQUE type

2019-10-21 Thread Kurt Young
OPAQUE seems to be a little bit advanced to a lot non-english speakers (including me). I think Xuefu raised a good alternative: UNKNOWN. What do you think about it? Best, Kurt On Mon, Oct 21, 2019 at 3:49 PM Aljoscha Krettek wrote: > I prefer OPAQUE compared to ANY because any is often the roo

Re: [VOTE] Accept Stateful Functions into Apache Flink

2019-10-21 Thread Kurt Young
+1 (binding) Best, Kurt On Tue, Oct 22, 2019 at 12:56 AM Fabian Hueske wrote: > +1 (binding) > > Am Mo., 21. Okt. 2019 um 16:18 Uhr schrieb Thomas Weise : > > > +1 (binding) > > > > > > On Mon, Oct 21, 2019 at 7:10 AM Timo Walther wrote: > > > > > +1 (binding) > > > > > > Thanks, > > > Timo >

Re: [DISCUSS] Rename the SQL ANY type to OPAQUE type

2019-10-21 Thread Kurt Young
gt;> types are > >> actually “known” to users, and users just want to leave them out of > Flink > >> type system. > >> > >> +1 for `RAW`, for it's more intuitive than `OPAQUE`. > >> > >> Best, > >> Paul Lam > >> >

Re: [DISCUSS] FLIP-70 - Support Computed Column for Flink SQL

2019-10-23 Thread Kurt Young
Hi Danny, Thanks for preparing this design document. IMO It's a very useful feature, especially combined with time attribute support to specify watermark in DDL. The design doc looks quite good, but I would suggest to reduce the scope of the first version. Like we don't have to support "STORED" i

Re: [DISCUSS] FLIP-70 - Support Computed Column for Flink SQL

2019-10-24 Thread Kurt Young
t. We can add them when user > requires. > > Others looks good to me in general. > > > > Thanks, > > Jark > > > > > >> 在 2019年10月24日,14:58,Kurt Young 写道: > >> > >> Hi Danny, > >> > >> Thanks for preparing this design docu

Re: [DISCUSS] FLIP-70 - Support Computed Column for Flink SQL

2019-10-25 Thread Kurt Young
he reviewers for the design doc, I have resolved all the > questions/suggestions in the doc at this time. > > I will kick off a voting thread shortly as there were no comments in this > thread so far, so I would assume there are no objections :) > > Best, > Danny Chan > 在 201

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

2019-10-28 Thread Kurt Young
+1 (binding) Best, Kurt 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 > > reache

Re: [DISCUSS] FLIP-84: Improve & Refactor API of Table Module

2019-10-30 Thread Kurt Young
Thanks Terry for bringing this up. TableEnv's interface is really critical not only to users, but also for components built upon it like SQL CLI. Your proposal solved some pain points we currently have, so +1 to the proposal. I left some comments in the document. Best, Kurt On Thu, Oct 31, 2019

Re: [DISCUSS] FLIP-84: Improve & Refactor API of Table Module

2019-11-04 Thread Kurt Young
cc @Fabian here, thought you might be interesting to review this. Best, Kurt On Thu, Oct 31, 2019 at 1:39 PM Kurt Young wrote: > Thanks Terry for bringing this up. TableEnv's interface is really critical > not only > to users, but also for components built upon it like

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

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

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

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

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

2019-11-07 Thread Kurt Young
y in the future. Best, Kurt On Fri, Nov 8, 2019 at 2:16 PM Terry Wang wrote: > Hi, Kurt~ > > Thanks for your vote and pointing out some deficiency of this flip. I’ll > try to avoid making similar mistakes. > > Best, > Terry Wang > > > > > 2019年11月8日 11:28,Kurt Y

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

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

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

2019-11-08 Thread Kurt Young
Congrats Jark, well deserved! Best, Kurt On Fri, Nov 8, 2019 at 5:53 PM Paul Lam wrote: > Congrats Jark! > > Best, > Paul Lam > > > 在 2019年11月8日,17:51,jincheng sun 写道: > > > > Hi all, > > > > On behalf of the Flink PMC, I'm happy to announce that Jark Wu is now > > part of the Apache Flink Pr

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

2019-11-09 Thread Kurt Young
+1 (binding) Best, Kurt On Sun, Nov 10, 2019 at 12:25 PM Peter Huang wrote: > Hi Yu, > > Thanks for your reminder about the timeline of delivering the basic > function DDL in release 1.10. > As I replied to Xuefu, the "CREATE FUNCTION" and "DROP FUNCTION" can > relatively easy achieve by revis

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

2019-11-14 Thread Kurt Young
+1 to the general idea of this FLIP. Regarding to connector properties, IIUC it would be divided into 2 parts: 1. connector.key1 = value1, these are interpreted by Flink framework and do whatever needed for such property. 2. connector.properties.xxx.key = value. There items are prefixed with "conn

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

2019-11-20 Thread Kurt Young
+1 (binding) Best, Kurt On Wed, Nov 20, 2019 at 6:56 PM Aljoscha Krettek wrote: > +1 (binding) > > Best, > Aljoscha > > > On 20. Nov 2019, at 11:36, Jingsong Li wrote: > > > > +1 (non-binding) > > Thanks Jark for driving this. > > > > Best, > > Jingsong Lee > > > > On Wed, Nov 20, 2019 at 6:2

Re: [DISCUSS] Primary keys in Table API

2019-11-20 Thread Kurt Young
Hi all, Thanks Dawid for bringing this up, this is very important property of SQL and I'm big +1 to have this. But before the discussion going deeply, I want to first mention that according to SQL standard, any unique key constraint which including primary key, should be always be enforced. Plea

Re: [DISCUSS] Primary keys in Table API

2019-11-20 Thread Kurt Young
anual+DDL#LanguageManualDDL-Constraints > > > What do you think? > > Best, > > Dawid > > > On 20/11/2019 13:51, Kurt Young wrote: > > Hi all, > > > > Thanks Dawid for bringing this up, this is very important property of SQL > > and > >

Re: [DISCUSS] Primary keys in Table API

2019-11-21 Thread Kurt Young
. Best, Kurt On Thu, Nov 21, 2019 at 10:25 AM Kurt Young wrote: > Hi Dawid, > > Actually I'm not 100% sure about both choices: always enforce primary key > by Flink and not enforce at all. > > As you said, always enforce primary key is hard to achieve since Flink > doesn

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

2019-11-22 Thread Kurt Young
+1 to disable, we also need to highlight this in 1.10 release notes. Best, Kurt On Fri, Nov 22, 2019 at 5:56 PM Zhenghua Gao wrote: > Hi, > > I wanted to bring up the discuss of Disable conversion between TIMESTAMP > and Long in parameters and results of UDXs. > > Since FLINK-12253[1] introduc

Re: FLIP 87: Primary key constraints in Table API

2019-11-22 Thread Kurt Young
+1 Best, Kurt On Fri, Nov 22, 2019 at 8:51 PM Dawid Wysakowicz wrote: > Hi everyone, > > I would like to start a vote on FLIP-87. > > Please vote for the following design document: > https://cwiki.apache.org/confluence/display/FLINK/FLIP+87%3A+Primary+key+constraints+in+Table+API > > The discu

Re: [DISCUSS] Voting from apache.org addresses

2019-12-04 Thread Kurt Young
+1 (from my apache email ;-)) Best, Kurt On Wed, Dec 4, 2019 at 7:22 PM Jark Wu wrote: > I'm +1 on this proposal. > > Regarding to the case that Dian mentioned, we can reminder the > committer/PMC to vote again use the apache email, > and of course the non-apache vote is counted as non-binding

Re: [DISCUSS] Migrate build infrastructure from Travis CI to Azure Pipelines

2019-12-04 Thread Kurt Young
Thanks Robert for driving this. There is another big pain point of current travis, which is its cache mechanism will fail from time to time. Almost around 50% of the build fails are caused by cache problem. I opened this issue to travis but got no response yet. So big +1 from my side. Just one com

Re: [DISCUSS] Add N-Ary Stream Operator

2019-12-05 Thread Kurt Young
During implementing n-ary input operator in table, please keep this pattern in mind: Build1 ---+ | +---> HshJoin1 --—> HashAgg ---+ | | Probe1 ---+ +---> HashJoin2

Re: [VOTE] Release 1.8.0, release candidate #4

2019-03-25 Thread Kurt Young
+1 (non-binding) Checked items: - checked checksums and GPG files - verified that the source archives do not contains any binaries - checked that all POM files point to the same version - build from source successfully Best, Kurt On Tue, Mar 26, 2019 at 10:57 AM Shaoxuan Wang wrote: > +1 (non

Re: [DISCUSS] FLIP-37: Rework of the Table API Type System (Part 1)

2019-03-28 Thread Kurt Young
Big +1 to this! I left some comments in google doc. Best, Kurt On Wed, Mar 27, 2019 at 11:32 PM Timo Walther wrote: > Hi everyone, > > some of you might have already read FLIP-32 [1] where we've described an > approximate roadmap of how to handle the big Blink SQL contribution and > how we can

Re: How can I use 1.9-SNAPSHOT Blink version for MapBundleFunction?

2019-04-15 Thread Kurt Young
Replied in user mailing list. Best, Kurt On Mon, Apr 15, 2019 at 11:48 PM Felipe Gutierrez < felipe.o.gutier...@gmail.com> wrote: > Hi, > > I am trying to use the Blink implementation for "MapBundleFunction > < > https://github.com/felipegutierrez/explore-blink/blob/master/src/main/java/org/sen

Re: SQL CLI and JDBC

2019-04-17 Thread Kurt Young
Also +1 to support JDBC. Best, Kurt On Wed, Apr 17, 2019 at 7:38 PM Stephan Ewen wrote: > I think this problem sounds fixable. Having proper JDBC support through the > SQL client would be really cool! > > Adding Timo and Shaoxuan here: > > Let's assume that the "collect()" call supports large

Re: [DISCUSS] Clean up and reorganize the JIRA components

2019-05-15 Thread Kurt Young
+1 to add benchmark component. Best, Kurt On Wed, May 15, 2019 at 6:13 PM Piotr Nowojski wrote: > Hi, > > I would like to propose two changes: > > 1. Renaming “Runtime / Operators” to “Runtime / Task” or something like > “Runtime / Processing”. “Runtime / Operators” was confusing me, since it

Re: [DISCUSS] Ground Source and Sink Concepts in Flink SQL

2019-05-27 Thread Kurt Young
Thanks Jark for bringing this topic. I think proper concepts is very important for users who are using Table API & SQL. Especially for them to have a clear understanding about the behavior of the SQL job. Also this is essential for connector developers to have a better understanding why we abstract

Re: [DISCUSS] Features for Apache Flink 1.9.0

2019-05-27 Thread Kurt Young
Thanks Gordon for bringing this up. I'm glad to say that blink planner merge work is almost done, and i will follow up the work of integrating blink planner with Table API to co-exist with current flink planner. In addition to this, the following features: 1. FLIP-32: Restructure flink-table for

Re: [DISCUSS] Adopting a Code Style and Quality Guide

2019-06-13 Thread Kurt Young
Big +1 and thanks for preparing this. I think wha't more important is making sure most all the contributors can follow the same guide, a clear document is definitely a great start. Committers can first try to follow the guide by self, and spread the standard during code reviewing. Best, Kurt On

Re: [DISCUSS] FLIP-44: Support Local Aggregation in Flink

2019-06-17 Thread Kurt Young
Hi Vino, Thanks for the proposal, I like the general idea and IMO it's very useful feature. But after reading through the document, I feel that we may over design the required operator for proper local aggregation. The main reason is we want to have a clear definition and behavior about the "local

Re: [DISCUSS] FLIP-44: Support Local Aggregation in Flink

2019-06-17 Thread Kurt Young
nfigure >the trigger threshold (maybe memory availability?), this design cannot >guarantee a deterministic semantics (it will bring trouble for testing > and >debugging). > - if the implementation depends on the timing of checkpoint, it would >affect the che

Re: [DISCUSS] FLIP-44: Support Local Aggregation in Flink

2019-06-17 Thread Kurt Young
exact API in DataStream named localKeyBy, about the pre-aggregation we need > to define the trigger mechanism of local aggregation, so we find reused > window API and operator is a good choice. This is a reasoning link from > design to implementation. > > What do you think? >

Something wrong with travis?

2019-06-17 Thread Kurt Young
Hi dev, I noticed that all the travis tests triggered by pull request are failed with the same error: "Cached flink dir /home/travis/flink_cache/x/flink does not exist. Exiting build." Anyone have a clue on what happened and how to fix this? Best, Kurt

Re: [DISCUSS] FLIP-44: Support Local Aggregation in Flink

2019-06-18 Thread Kurt Young
, which can >benefit from the asnyc snapshot and incremental checkpoint. IMO, the >performance is not a problem, and we also do not find the performance > issue >in our production. > > [1]: > > http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCU

Re: [DISCUSS] FLIP-44: Support Local Aggregation in Flink

2019-06-18 Thread Kurt Young
would be probably a better choice. > > Because of that, I think we should eventually provide both versions and in > the initial version we should at least make the “local aggregation engine” > abstract enough, that one could easily provide different implementation > strategy. > >

Re: [DISCUSS] FLIP-44: Support Local Aggregation in Flink

2019-06-19 Thread Kurt Young
; > > Thanks for your comments. > > > > > > I agree that we can provide a better abstraction to be compatible with > > two > > > different implementations. > > > > > > First of all, I think we should consider what kind of scenarios we need > > to >

Re: [DISCUSS] Start a user...@flink.apache.org mailing list for the Chinese-speaking community?

2019-06-19 Thread Kurt Young
Is there any chance that we can contact Apache infra team to find out why apache mails are blocked by qq.com? QQ mails are very popular in Chinese. Best, Kurt On Thu, Jun 20, 2019 at 12:01 PM Hequn Cheng wrote: > Hi Gordon, > > Thanks a lot for providing the valuable information! > As I carry

Re: [DISCUSS] Start a user...@flink.apache.org mailing list for the Chinese-speaking community?

2019-06-20 Thread Kurt Young
(see a similar > example, also mentioning qq.com: > https://issues.apache.org/jira/browse/INFRA-18249) > > > > On Thu, Jun 20, 2019 at 6:23 AM Kurt Young wrote: > > > Is there any chance that we can contact Apache infra team to find out why > > apache mails are blocked by

Re: [DISCUSS] Start a user...@flink.apache.org mailing list for the Chinese-speaking community?

2019-06-20 Thread Kurt Young
20, 2019 at 10:23 PM Kurt Young wrote: > Thanks Robert, I left a comment in the JIRA you gave and see what will > happen. > > Best, > Kurt > > > On Thu, Jun 20, 2019 at 9:04 PM Robert Metzger > wrote: > >> Thank you all for working on this! >> >>

Re: [DISCUSS] Start a user...@flink.apache.org mailing list for the Chinese-speaking community?

2019-06-20 Thread Kurt Young
e. >> >> Best, >> Vino. >> >> Robert Metzger 于2019年6月20日周四 下午10:59写道: >> >>> Thanks a lot! >>> >>> qq.com belongs to Tencent, right? >>> As far as I know, we have some active contributors working at Tencent >>> (Vino

Re: [DISCUSS] FLIP-44: Support Local Aggregation in Flink

2019-06-24 Thread Kurt Young
ic is very complicated and optimization does not matter, I > > think it's a better choice to provide a relatively low-level and > canonical > > interface. > > > > The composited interface, on the other side, may be a good choice in > > declarative interfaces, including

Re: [DISCUSS] FLIP-44: Support Local Aggregation in Flink

2019-06-24 Thread Kurt Young
In API level, we have answered your question about pass an > AggregateFunction to do the aggregation. No matter introduce localKeyBy API > or not, we can support AggregateFunction. > > So what's your different opinion now? Can you share it with us? > > Best, > Vino > >

Re: [DISCUSS] FLIP-44: Support Local Aggregation in Flink

2019-06-24 Thread Kurt Young
Hi vino, One thing to add, for a), I think use one or two examples like how to do local aggregation on a sliding window, and how do we do local aggregation on an unbounded aggregate, will do a lot help. Best, Kurt On Mon, Jun 24, 2019 at 6:06 PM Kurt Young wrote: > Hi vino, > >

Re: [ANNOUNCE] Jincheng Sun is now part of the Flink PMC

2019-06-24 Thread Kurt Young
Congratulations Jincheng! Best, Kurt On Tue, Jun 25, 2019 at 9:56 AM LakeShen wrote: > Congratulations! Jincheng Sun > > Best, > LakeShen > > Robert Metzger 于2019年6月24日周一 下午11:09写道: > > > Hi all, > > > > On behalf of the Flink PMC, I'm happy to announce that Jincheng Sun is > now > > part of

Re: [DISCUSS] solve unstable build capacity problem on TravisCI

2019-06-24 Thread Kurt Young
Hi Bowen, Thanks for bringing this up. We actually have discussed about this, and I think Till and George have already spend sometime investigating it. I have cced both of them, and maybe they can share their findings. Best, Kurt On Tue, Jun 25, 2019 at 10:08 AM Jark Wu wrote: > Hi Bowen, > >

Re: [DISCUSS] solve unstable build capacity problem on TravisCI

2019-06-24 Thread Kurt Young
(Forgot to cc George) Best, Kurt On Tue, Jun 25, 2019 at 10:16 AM Kurt Young wrote: > Hi Bowen, > > Thanks for bringing this up. We actually have discussed about this, and I > think Till and George have > already spend sometime investigating it. I have cced both of them, and &

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

2019-06-25 Thread Kurt Young
Hi Aljoscha, I also feel an additional week can make the remaining work more easy. At least we don't have to check in lots of commits in both branches (master & release-1.9). Best, Kurt On Tue, Jun 25, 2019 at 8:27 PM Aljoscha Krettek wrote: > A few threads are converging around supporting th

Re: [DISCUSS] META-FLIP: Sticking (or not) to a strict FLIP voting process

2019-06-26 Thread Kurt Young
+1 for sticking to the lazy majority voting. Especially for the reason that if all committers don't have time capacity to help discuss and review the changes which bring up by the FLIP, it will be meaningless for this FLIP to be considered as accepted. I don't have much suggestions about the scope

Re: [DISCUSS]Support Upsert mode for Streaming Non-window FlatAggregate

2019-06-30 Thread Kurt Young
Hi, I have a question about the key information of TableAggregateFunction. IIUC, you need to define something like primary key or unique key in the result table of TableAggregateFunction, and also need a way to let user configure this through the API. My question is, will that effect the logic of

Re: [DISCUSS]Support Upsert mode for Streaming Non-window FlatAggregate

2019-06-30 Thread Kurt Young
ssing does not involve semantic > changes. The definition of keys is to support non-window flatAggregate on > upsert mode. (The upsert mode is already supported in the flink framework. > The current discussion only needs to inform the framework that the keys > information, which is the `

Re: [ANNOUNCE] Apache Flink 1.8.1 released

2019-07-02 Thread Kurt Young
Thanks for being the release manager and great job! @Jincheng Best, Kurt On Wed, Jul 3, 2019 at 10:19 AM Tzu-Li (Gordon) Tai wrote: > Thanks for being the release manager @jincheng sun > :) > > On Wed, Jul 3, 2019 at 10:16 AM Dian Fu wrote: > >> Awesome! Thanks a lot for being the release ma

Re: [DISCUSS] FLIP-44: Support Local Aggregation in Flink

2019-07-03 Thread Kurt Young
gregate: > >>>>>>> > input.localKeyBy(..).aggregate(agg1).keyBy(..).aggregate(agg2) > >>>>>>> **NOT > >>>>>>> > SUPPORT** > >>>>>>> > b) For windowed aggregate: > >>>>>>> > >

Re: [VOTE] Migrate to sponsored Travis account

2019-07-04 Thread Kurt Young
t;>>>>>> harder to > >>>>>>>>>> accomplish in a short period of time and may deserve > >>>its own > >>>>>>> separate > >>>>>>>>>> discussion". Thus I didn't incl

Re: [DISCUSS]Support Upsert mode for Streaming Non-window FlatAggregate

2019-07-04 Thread Kurt Young
as. Let me know if this makes sense to you. Best, Kurt On Thu, Jul 4, 2019 at 4:32 PM jincheng sun wrote: > Hi All, > > @Kurt Young one user-defined table aggregate function > can be used in both with(out) keys case, and we do not introduce any other > aggregations. just li

  1   2   3   4   5   >