Re: [VOTE] Apache Flink CDC Release 3.1.0, release candidate #3

2024-05-16 Thread Jark Wu
+1 (binding) - checked signatures - checked hashes - checked release notes - reviewed the release web PR - checked the jars in the staging repo - build and compile the source code locally with jdk8 Best, Jark On Wed, 15 May 2024 at 16:05, gongzhongqiang wrote: > +1 (non-binding) > > -

Re: [DISCUSSION] FLIP-457: Improve Table/SQL Configuration for Flink 2.0

2024-05-16 Thread Jark Wu
Hi Jane, Thanks for the proposal. +1 from my side. Best, Jark On Thu, 16 May 2024 at 10:28, Xuannan Su wrote: > Hi Jane, > > Thanks for driving this effort! And +1 for the proposed changes. > > I have one comment on the migration plan. > > For options to be moved to another module/package, I

Re: Re: [VOTE] FLIP-448: Introduce Pluggable Workflow Scheduler Interface for Materialized Table

2024-05-09 Thread Jark Wu
+1 (binding) Best, Jark On Thu, 9 May 2024 at 21:27, Lincoln Lee wrote: > +1 (binding) > > Best, > Lincoln Lee > > > Feng Jin 于2024年5月9日周四 19:45写道: > > > +1 (non-binding) > > > > > > Best, > > Feng > > > > > > On Thu, May 9, 2024 at 7:37 PM Xuyang wrote: > > > > > +1 (non-binding) > > > > >

Re: [VOTE] FLIP-436: Introduce Catalog-related Syntax

2024-04-26 Thread Jark Wu
cat2 | > > > | Type | > > > generic_in_memory | > > > | Comment | > > > | > > > | Properties | ('default-database','db'), > > > ('type','generic_in_memory') | > > >

Re: [VOTE] FLIP-435: Introduce a New Materialized Table for Simplifying Data Pipelines

2024-04-17 Thread Jark Wu
+1 (binding) Best, Jark On Wed, 17 Apr 2024 at 20:52, Leonard Xu wrote: > +1(binding) > > Best, > Leonard > > > 2024年4月17日 下午8:31,Lincoln Lee 写道: > > > > +1(binding) > > > > Best, > > Lincoln Lee > > > > > > Ferenc Csaky 于2024年4月17日周三 19:58写道: > > > >> +1 (non-binding) > >> > >> Best, > >>

[ANNOUNCE] New Apache Flink PMC Member - Jing Ge

2024-04-12 Thread Jark Wu
Hi everyone, On behalf of the PMC, I'm very happy to announce that Jing Ge has joined the Flink PMC! Jing has been contributing to Apache Flink for a long time. He continuously works on SQL, connectors, Source, and Sink APIs, test, and document modules while contributing lots of code and

[ANNOUNCE] New Apache Flink PMC Member - Lincoln Lee

2024-04-12 Thread Jark Wu
Hi everyone, On behalf of the PMC, I'm very happy to announce that Lincoln Lee has joined the Flink PMC! Lincoln has been an active member of the Apache Flink community for many years. He mainly works on Flink SQL component and has driven /pushed many FLIPs around SQL, including

Re: [DISCUSS] FLIP-435: Introduce a New Dynamic Table for Simplifying Data Pipelines

2024-04-09 Thread Jark Wu
> > > >>>> >> 3. For the framework, when we perform a certain kind of Alter > > Table > > > >>>> >> behavior for a table, for the table created by specifying > > FRESHNESS > > > >>>> and did > > &g

Re: [VOTE] FLIP-437: Support ML Models in Flink SQL

2024-04-02 Thread Jark Wu
+1 (binding) Best, Jark On Tue, 2 Apr 2024 at 15:12, Timo Walther wrote: > +1 (binding) > > Thanks, > Timo > > On 29.03.24 17:30, Hao Li wrote: > > Hi devs, > > > > I'd like to start a vote on the FLIP-437: Support ML Models in Flink > > SQL [1]. The discussion thread is here [2]. > > > > The

Re: Re: [ANNOUNCE] Apache Paimon is graduated to Top Level Project

2024-03-30 Thread Jark Wu
Congratulations! Best, Jark On Fri, 29 Mar 2024 at 12:08, Yun Tang wrote: > Congratulations to all Paimon guys! > > Glad to see a Flink sub-project has been graduated to an Apache top-level > project. > > Best > Yun Tang > > > From: Hangxiang Yu > Sent:

Re: [DISCUSS] FLIP-437: Support ML Models in Flink SQL

2024-03-28 Thread Jark Wu
Thanks, Hao, Sounds good to me. Best, Jark On Thu, 28 Mar 2024 at 01:02, Hao Li wrote: > Hi Jark, > > I think we can start with supporting popular model providers such as > openai, azureml, sagemaker for remote models. > > Thanks, > Hao > > On Tue, Mar 26, 202

Re: [DISCUSS] FLIP-437: Support ML Models in Flink SQL

2024-03-26 Thread Jark Wu
gt; replaced by `getComment()` which is sufficient. > > >>> > > >>> 2) CREATE TEMPORARY MODEL is not supported. > > >>> > > >>> This is an unnecessary restriction. We should support temporary > > versions > > >>> of these catalog

Re: [DISCUSS] FLIP-437: Support ML Models in Flink SQL

2024-03-22 Thread Jark Wu
Sorry, I mean we can bump the Calcite version if needed in Flink 1.20. On Fri, 22 Mar 2024 at 22:19, Jark Wu wrote: > Hi Timo, > > Introducing user-defined PTF is very useful in Flink, I'm +1 for this. > But I think the ML model FLIP is not blocked by this, because we > can intro

Re: [DISCUSS] FLIP-437: Support ML Models in Flink SQL

2024-03-22 Thread Jark Wu
e proposal per our offline discussions: > > 1. Model will be treated as a new relation in FlinkSQL. > > 2. Include the common ML predict and evaluate functions into the open > > source flink to complete the user journey. > > And we should be able to extend the calcite SqlTableFuncti

Re: [DISCUSS] Planning Flink 1.20

2024-03-22 Thread Jark Wu
Thanks for kicking this off. +1 for the volunteered release managers (Weijie Guo, Rui Fan) and the targeting date (feature freeze: June 15). Best, Jark On Fri, 22 Mar 2024 at 14:00, Rui Fan <1996fan...@gmail.com> wrote: > Thanks Leonard for this feedback and help! > > Best, > Rui > > On

Re: [ANNOUNCE] Donation Flink CDC into Apache Flink has Completed

2024-03-20 Thread Jark Wu
Congratulations and welcome! Best, Jark On Thu, 21 Mar 2024 at 10:35, Rui Fan <1996fan...@gmail.com> wrote: > Congratulations! > > Best, > Rui > > On Thu, Mar 21, 2024 at 10:25 AM Hang Ruan wrote: > > > Congrattulations! > > > > Best, > > Hang > > > > Lincoln Lee 于2024年3月21日周四 09:54写道: > > >

Re: [VOTE] FLIP-436: Introduce Catalog-related Syntax

2024-03-19 Thread Jark Wu
+1 (binding) Best, Jark On Tue, 19 Mar 2024 at 19:05, Yuepeng Pan wrote: > Hi, Yubin > > > Thanks for driving it ! > > +1 non-binding. > > > > > > > > Best, > Yuepeng Pan. > > > > > > > > > At 2024-03-19 17:56:42, "Yubin Li" wrote: > >Hi everyone, > > > >Thanks for all the feedback, I'd like

Re: [DISCUSS] FLIP-437: Support ML Models in Flink SQL

2024-03-18 Thread Jark Wu
pecify > all of them in model creation. > > [1]: > > https://cloud.google.com/bigquery/docs/reference/standard-sql/bigqueryml-syntax-predict > [2]: > > https://cloud.google.com/bigquery/docs/reference/standard-sql/bigqueryml-syntax-evaluate > > Thanks, > Hao >

Re: [ANNOUNCE] Apache Flink 1.19.0 released

2024-03-18 Thread Jark Wu
Congrats! Thanks Lincoln, Jing, Yun and Martijn driving this release. Thanks all who involved this release! Best, Jark On Mon, 18 Mar 2024 at 16:31, Rui Fan <1996fan...@gmail.com> wrote: > Congratulations, thanks for the great work! > > Best, > Rui > > On Mon, Mar 18, 2024 at 4:26 PM Lincoln

Re: Re: [DISCUSS] FLIP-436: Introduce "SHOW CREATE CATALOG" Syntax

2024-03-18 Thread Jark Wu
myKey attribute of a certain table to the default > value. For catalogs, > it might be better to use ALTER CATALOG xxx RESET ('mykey') for the sake of > design > consistency. > > WDYT? Looking forward to your suggestions. > > Best, > Yubin > > [1] > > https://nig

Re: Re: [DISCUSS] FLIP-436: Introduce "SHOW CREATE CATALOG" Syntax

2024-03-17 Thread Jark Wu
> >>>>>> SQL parser implementation in the FLIP > > >> >>>>>> for the SQL syntax might be a bit confusing. Also, the formal > > >> >>>>>> definition > > >> >>>>>> is > &g

Re: [DISCUSS] FLIP-437: Support ML Models in Flink SQL

2024-03-14 Thread Jark Wu
learning/tutorials/quickstart-python-train-score-model?view=azuresqldb-mi-current > > Thanks, > Hao > > On Wed, Mar 13, 2024 at 8:57 PM Jark Wu wrote: > > > Hi Mingge, Hao, > > > > Thanks for your replies. > > > > > PTF is actually the ideal a

Re: [DISCUSS] Add "Special Thanks" Page on the Flink Website

2024-03-14 Thread Jark Wu
The pull request has been merged. Thank you for the discussion and reviewing. The page is live now: https://flink.apache.org/what-is-flink/special-thanks/ Best, Jark On Tue, 12 Mar 2024 at 17:44, Jark Wu wrote: > I have created a JIRA issue and opened a pull request for this: >

Re: [DISCUSS] FLIP-437: Support ML Models in Flink SQL

2024-03-13 Thread Jark Wu
t out as a built-in function. Maybe we can separate that as a > follow up if we want to have it built-in and focus on the model syntax for > this FLIP? > > Thanks, > Hao > > On Tue, Mar 12, 2024 at 10:33 PM Jark Wu wrote: > > > Hi Minge, Chris, Hao, > > > >

Re: [DISCUSS] FLIP-436: Introduce "SHOW CREATE CATALOG" Syntax

2024-03-13 Thread Jark Wu
FLINK/FLIP-295%3A+Support+lazy+initialization+of+catalogs+and+persistence+of+catalog+configurations > > > > > > > > Best, > > > > Jingsong > > > > > > > > On Wed, Mar 13, 2024 at 5:09 PM Yubin Li wrote: > > > > > > > > >

Re: [DISCUSS] FLIP-437: Support ML Models in Flink SQL

2024-03-12 Thread Jark Wu
Hi Minge, Chris, Hao, Thanks for proposing this interesting idea. I think this is a nice step towards the AI world for Apache Flink. I don't know much about AI/ML, so I may have some stupid questions. 1. Could you tell more about why polymorphism table function (PTF) doesn't work and do we have

Re: [DISCUSS] Add "Special Thanks" Page on the Flink Website

2024-03-12 Thread Jark Wu
I have created a JIRA issue and opened a pull request for this: https://github.com/apache/flink-web/pull/725. Best, Jark On Tue, 12 Mar 2024 at 16:56, Jark Wu wrote: > Thank you all for your feedback. If there are no other concerns or > objections, > I'm going to create a pull reque

[jira] [Created] (FLINK-34654) Add "Special Thanks" Page on the Flink Website

2024-03-12 Thread Jark Wu (Jira)
Jark Wu created FLINK-34654: --- Summary: Add "Special Thanks" Page on the Flink Website Key: FLINK-34654 URL: https://issues.apache.org/jira/browse/FLINK-34654 Project: Flink Issue Type: N

Re: [DISCUSS] Add "Special Thanks" Page on the Flink Website

2024-03-12 Thread Jark Wu
, Mar 8, 2024 at 11:55 AM Martijn Visser > wrote: > > > > Hi all, > > > > I'm +1 on it. As long as we follow the ASF rules on this, we can thank > > those that are/have made contributions. > > > > Best regards, > > > > Martijn > > >

Re: [DISCUSS] Add "Special Thanks" Page on the Flink Website

2024-03-06 Thread Jark Wu
e how insufficient CI machines would impact the development > > progress, especially when the feature freeze date is close. And I believe > > that most guys who contributed to the community would not know Alibaba > and > > Ververica had ever donated several machines to make the communi

Re: [DISCUSS] Add "Special Thanks" Page on the Flink Website

2024-03-05 Thread Jark Wu
proposal! > > > > > > > > Best regards, > > > > Jing > > > > > > > > On Tue, Mar 5, 2024 at 10:26 AM tison wrote: > > > > > > > > > I like this idea, so +1 for setting up. > > > > > > > &g

Re: [DISCUSS] Add "Special Thanks" Page on the Flink Website

2024-03-05 Thread Jark Wu
Sorry, I posted the wrong [7] link. The Flink benchmark ML link is: https://lists.apache.org/thread/bkw6ozoflgltwfwmzjtgx522hyssfko6 On Tue, 5 Mar 2024 at 16:56, Jark Wu wrote: > Hi all, > > > > I want to propose adding a "Special Thanks" page to our Apache Flink we

[DISCUSS] Add "Special Thanks" Page on the Flink Website

2024-03-05 Thread Jark Wu
Hi all, I want to propose adding a "Special Thanks" page to our Apache Flink website [1] to honor and appreciate the companies and organizations that have sponsored machines or services for our project. The establishment of such a page serves as a public acknowledgment of our sponsors'

Re: Re: [VOTE] FLIP-377: Support fine-grained configuration to control filter push down for Table/SQL Sources

2024-01-18 Thread Jark Wu
+1 (binding) Best, Jark On Tue, 16 Jan 2024 at 18:01, Xuyang wrote: > +1 (non-binding) > > > -- > > Best! > Xuyang > > > > > > 在 2024-01-16 17:52:38,"Leonard Xu" 写道: > >+1 (binding) > > > >Best, > >Leonard > > > >> 2024年1月16日 下午5:40,Hang Ruan 写道: > >> > >> +1 (non-binding) > >> > >>

Re: [VOTE] Accept Flink CDC into Apache Flink

2024-01-08 Thread Jark Wu
+1 (binding) Best, Jark On Tue, 9 Jan 2024 at 15:31, Benchao Li wrote: > +1 (non-binding) > > Feng Wang 于2024年1月9日周二 15:29写道: > > > > +1 non-binding > > Regards, > > Feng > > > > On Tue, Jan 9, 2024 at 3:05 PM Leonard Xu wrote: > > > > > Hello all, > > > > > > This is the official vote

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

2024-01-08 Thread Jark Wu
Thanks Jing for driving this. +1 (binding) - Build and compile the source code locally: *OK* - Verified signatures and hashes: *OK* - Checked no missing artifacts in the staging area: *OK* - Reviewed the website release PR: *OK* - Went through the quick start: *OK* * Started a cluster and ran

Re: [DISCUSS] Release Flink 1.18.1

2023-12-11 Thread Jark Wu
Thanks Jing for driving 1.18.1. +1 for this. Best, Jark On Mon, 11 Dec 2023 at 16:59, Hong Liang wrote: > +1. Thanks Jing for driving this. > > Hong > > On Mon, Dec 11, 2023 at 2:27 AM Yun Tang wrote: > > > Thanks Jing for driving 1.18.1 release, +1 for this. > > > > > > Best > > Yun Tang > >

Re: [PROPOSAL] Contribute Flink CDC Connectors project to Apache Flink

2023-12-06 Thread Jark Wu
+1 for adding this to Apache Flink! I think this can further extend the ability of Apache Flink and a lot of users would be interested to try this out. Best, Jark On Thu, 7 Dec 2023 at 12:06, Samrat Deb wrote: > That's really cool :) > +1 for the great addition > > Bests, > Samrat > > On Thu,

[jira] [Created] (FLINK-33600) Print cost time for batch queries in SQL Client

2023-11-20 Thread Jark Wu (Jira)
Jark Wu created FLINK-33600: --- Summary: Print cost time for batch queries in SQL Client Key: FLINK-33600 URL: https://issues.apache.org/jira/browse/FLINK-33600 Project: Flink Issue Type: New

Re: [VOTE] FLIP-378: Support Avro timestamp with local timezone

2023-11-15 Thread Jark Wu
+1 (binding) Best, Jark On Thu, 16 Nov 2023 at 12:41, Leonard Xu wrote: > +1(binding) > > Best, > Leonard > > > 2023年11月16日 下午12:13,Mingliang Liu 写道: > > > > +1 (non-binding) > > > > On Wed, Nov 15, 2023 at 3:38 PM Peter Huang > > wrote: > > > >> Hi Devs, > >> > >> I'd like to start a vote

Re: [DISCUSS] FLIP-378: Support Avro timestamp with local timezone

2023-11-13 Thread Jark Wu
+1 I think we can mark the temporary config option as deprecated when we introduce it. So we can remove it after 2 minor releases (1.19, 1.20), i.e., drop in 2.0 release. A minor comment about the config option, I would suggest to use "avro.timestamp-mapping.legacy" instead of

Re: [DISCUSS] FLIP-376: Add DISTRIBUTED BY clause

2023-10-31 Thread Jark Wu
gt; Timo > >> > >> > >> On 30.10.23 10:44, Timo Walther wrote: > >>> Hi Jark, > >>> > >>> my intention was to avoid too complex syntax in the first version. In > >>> the past years, we could enable use cases also witho

Re: [DISCUSS] Planning Flink 1.19

2023-10-29 Thread Jark Wu
+1 for the proposed release managers and feature freeze on Jan 26 sounds good to me. Best. Jark On Mon, 30 Oct 2023 at 12:15, Xintong Song wrote: > Thanks for kicking this off. > > +1 for the proposed release managers (Lincoln, Yun, Jing and Martijn) and > targeting date (feature freeze: Jan

Re: [DISCUSS] FLIP-377: Support configuration to disable filter push down for Table/SQL Sources

2023-10-27 Thread Jark Wu
upportsXXXPushDown interface, the optimizer should push the corresponding > predicates to the Source. I am not sure in which case this configuration > would be used. Any ideas @Jark Wu ? > > Thanks, > > Jiangjie (Becket) Qin > > > On Wed, Oct 25, 2023 at 11:55 PM Jiabao Sun

Re: [DISCUSS] FLIP-376: Add DISTRIBUTED BY clause

2023-10-27 Thread Jark Wu
Hi Timo, Thanks for starting this discussion. I really like it! The FLIP is already in good shape, I only have some minor comments. 1. Could we also support HASH and RANGE distribution kind on the DDL syntax? I noticed that HASH and UNKNOWN are introduced in the Java API, but not in the syntax.

Re: [ANNOUNCE] Apache Flink 1.18.0 released

2023-10-26 Thread Jark Wu
Congratulations and thanks release managers and everyone who has contributed! Best, Jark On Fri, 27 Oct 2023 at 12:25, Hang Ruan wrote: > Congratulations! > > Best, > Hang > > Samrat Deb 于2023年10月27日周五 11:50写道: > > > Congratulations on the great release > > > > Bests, > > Samrat > > > > On

Re: [VOTE] FLIP-373: Support Configuring Different State TTLs using SQL Hint

2023-10-25 Thread Jark Wu
+1 (binding) Best, Jark On Wed, 25 Oct 2023 at 16:27, Jiabao Sun wrote: > Thanks Jane for driving this. > > +1 (non-binding) > > Best, > Jiabao > > > > 2023年10月25日 16:22,Lincoln Lee 写道: > > > > +1 (binding) > > > > Best, > > Lincoln Lee > > > > > > Zakelly Lan 于2023年10月23日周一 14:15写道: > > >

Re: [DISCUSS] FLIP-377: Support configuration to disable filter push down for Table/SQL Sources

2023-10-24 Thread Jark Wu
>> > >>>> Thanks Jark, > >>>> > >>>> If we only add configuration without adding the enableFilterPushDown > >>>> method in the SupportsFilterPushDown interface, > >>>> each connector would have to handle the same logic in the applyFilt

Re: [DISCUSS] FLIP-377: Support configuration to disable filter push down for Table/SQL Sources

2023-10-24 Thread Jark Wu
Hi JIabao, I think the current interface can already satisfy your requirements. The connector can reject all the filters by returning the input filters as `Result#remainingFilters`. So maybe we don't need to introduce a new method to disable pushdown, but just introduce an option for the

[ANNOUNCE] New Apache Flink Committer - Jane Chan

2023-10-15 Thread Jark Wu
was in the early days. Please join me in congratulating Jane Chan for becoming a Flink Committer! Best, Jark Wu (on behalf of the Flink PMC)

[ANNOUNCE] New Apache Flink Committer - Ron Liu

2023-10-15 Thread Jark Wu
me in congratulating Ron Liu for becoming a Flink Committer! Best, Jark Wu (on behalf of the Flink PMC)

Re: [DISCUSS] FLIP-328: Allow source operators to determine isProcessingBacklog based on watermark lag

2023-09-18 Thread Jark Wu
mment on whether the current design looks good? > > I plan to start voting in 3 days if there is no follow-up comment. > > Thanks, > Dong > > > On Fri, Sep 15, 2023 at 2:01 PM Jark Wu wrote: > > > Hi Dong, > > > > > Note that we can not simply en

Re: [DISCUSS] [FLINK-32873] Add a config to allow disabling Query hints

2023-09-15 Thread Jark Wu
GUID-D62CA6D8-D0D8-4A20-93EA-EEB4B3144347 > [2] > > https://www.mssqltips.com/sqlservertip/4175/disabling-sql-server-optimizer-rules-with-queryruleoff/ > > Best regards, > > Martijn > > On Fri, Sep 15, 2023 at 10:53 AM Jark Wu wrote: > > > Hi Martijn, > > >

Re: [DISCUSS] [FLINK-32873] Add a config to allow disabling Query hints

2023-09-15 Thread Jark Wu
Jark! > > > > I can definitely appreciate the desire to have fewer configurations. > > > > Do you have a suggested alternative for platform providers to limit or > > restrict the hints that Bonnie is talking about? > > > > As one possibility, maybe

Re: [DISCUSS] FLIP-328: Allow source operators to determine isProcessingBacklog based on watermark lag

2023-09-15 Thread Jark Wu
ot; first, but the watermark lag exceeds the configured value. This is the conflict I'm concerned about. Best, Jark On Fri, 15 Sept 2023 at 12:00, Dong Lin wrote: > Hi Jark, > > Please see my comments inline. > > On Fri, Sep 15, 2023 at 10:35 AM Jark Wu wrote: > > >

Re: [DISCUSS] FLIP-328: Allow source operators to determine isProcessingBacklog based on watermark lag

2023-09-14 Thread Jark Wu
s right > now), we can add such a config in the future with a follow-up FLIP. The > job-level config will still be useful as it makes users' configuration > simpler for common scenarios. > > If it is OK, can we agree to make incremental progress for Flink and start > a vot

Re: [DISCUSS] FLIP-328: Allow source operators to determine isProcessingBacklog based on watermark lag

2023-09-11 Thread Jark Wu
ong Lin wrote: > Hi Jark, > > Thanks for the comments. Please see my comments inline. > > On Sat, Sep 9, 2023 at 4:13 PM Jark Wu wrote: > > > Hi Xuannan, > > > > I leave my comments inline. > > > > > In the case where a user wants to > >

Re: [DISCUSS] [FLINK-32873] Add a config to allow disabling Query hints

2023-09-09 Thread Jark Wu
ause > > > most > > > > >> of them have no effect for streaming and long-term we could also > set > > > > >> some hints via the CompiledPlan. And if you have multiple teams, > > > > >> non-skilled users should not play around with hin

Re: [DISCUSS] FLIP-328: Allow source operators to determine isProcessingBacklog based on watermark lag

2023-09-09 Thread Jark Wu
gt;>>>>>> Xuannan > > >>>>>>>> > > >>>>>>>> [1] > > >>>>>>>> > > >>>>>> > > >>>> > > >> > https://cwiki.apache.org/co

Re: [DISCUSS][FLINK-31788][FLINK-33015] Add back Support emitUpdateWithRetract for TableAggregateFunction

2023-09-07 Thread Jark Wu
+1 to fix it first. I also agree to deprecate it if there are few people using it, but this should be another discussion thread within dev+user ML. In the future, we are planning to introduce user-defined-operator based on the TVF functionality which I think can fully subsume the UDTAG, cc @Timo

Re: [DISCUSS] FLIP-328: Allow source operators to determine isProcessingBacklog based on watermark lag

2023-09-07 Thread Jark Wu
> > > >>>> watermarks. However, after considering the > pending-record-based > > > > > >> solution, I > > > > > >>>> believe the current solution is superior for the target use > case > > > as it > > > > > >> is >

Re: [DISCUSS] Update Flink Roadmap

2023-09-01 Thread Jark Wu
Thank you all for helping with the roadmap documentation. I have merged the roadmap pull request. Cheers, Jark On Wed, 23 Aug 2023 at 15:26, Jing Ge wrote: > Thanks Jark, +1 for the OLAP :-) > > Best regards, > Jing > > On Sun, Aug 20, 2023 at 5:04 PM Jark Wu wrote: >

Re: [VOTE] FLIP-356: Support Nested Fields Filter Pushdown

2023-09-01 Thread Jark Wu
+1 (binding) Best, Jark > 2023年8月30日 02:40,Venkatakrishnan Sowrirajan 写道: > > Hi everyone, > > Thank you all for your feedback on FLIP-356. I'd like to start a vote. > > Discussion thread: > https://lists.apache.org/thread/686bhgwrrb4xmbfzlk60szwxos4z64t7 > FLIP: >

Re: [VOTE] FLIP-348: Make expanding behavior of virtual metadata columns configurable

2023-08-31 Thread Jark Wu
+1 (binding) Best, Jark > 2023年8月31日 18:54,Jing Ge 写道: > > +1(binding) > > On Thu, Aug 31, 2023 at 11:22 AM Sergey Nuyanzin > wrote: > >> +1 (binding) >> >> On Thu, Aug 31, 2023 at 9:28 AM Benchao Li wrote: >> >>> +1 (binding) >>> >>> Martijn Visser 于2023年8月31日周四 15:24写道: +1

Re: [DISCUSS] FLIP-356: Support Nested Fields Filter Pushdown

2023-08-29 Thread Jark Wu
xArray* we can just do away with *fieldNames *array that > includes fieldName at every level for the nested field. > > Updated the FLIP-357 > < > https://cwiki.apache.org/confluence/display/FLINK/FLIP-356%3A+Support+Nested+Fields+Filter+Pushdown > > > wiki as well. > > Regar

Re: [DISCUSS] FLIP-356: Support Nested Fields Filter Pushdown

2023-08-29 Thread Jark Wu
; > > > > > > > > > 3. Finally, I think we need to look at the costs and benefits of > > unifying > > > > > > the SupportsFilterPushDown and SupportsProjectionPushDown (or > > others) > > > > > from &

Re: [DISCUSS] FLIP-356: Support Nested Fields Filter Pushdown

2023-08-24 Thread Jark Wu
have to introduce two different > applyProjections() > > methods for FieldReferenceExpression / NestedFieldReferenceExpression > > respectively. This issue further extends to implementation in addition to > > public API. A single FieldReferenceExpression might help simplify the

Re: [DISCUSS] FLIP-356: Support Nested Fields Filter Pushdown

2023-08-22 Thread Jark Wu
Given the size of Flink, keeping APIs in the same style over > time is important. The migration is also not that complicated. > > Thanks, > > Jiangjie (Becket) Qin > > > On Tue, Aug 22, 2023 at 2:23 PM Jark Wu wrote: > > > Hi Venkat, > > > > Thanks for th

Re: [DISCUSS] FLIP-356: Support Nested Fields Filter Pushdown

2023-08-22 Thread Jark Wu
Hi Venkat, Thanks for the proposal. I have some minor comments about the FLIP. 1. I think we don't need to add SupportsFilterPushDown#supportsNestedFilters() method, because connectors can skip nested filters by putting them in Result#remainingFilters(). And this is backward-compatible because

Re: [DISCUSS] FLIP-328: Allow source operators to determine isProcessingBacklog based on watermark lag

2023-08-20 Thread Jark Wu
Hi Xuannan, Thanks for opening this discussion. This current proposal may work in the mentioned watermark cases. However, it seems this is not a general solution for sources to determine "isProcessingBacklog". >From my point of view, there are 3 limitations of the current proposal: 1. It doesn't

Re: [DISCUSS] Update Flink Roadmap

2023-08-20 Thread Jark Wu
lve you and Xingtong > > on the document after updating. > > > > Best, > > Shammon FY > > > > > > On Mon, Aug 14, 2023 at 10:39 PM Jark Wu wrote: > > > >> Hi Shammon, > >> > >> Sure, could you help to draft a subsection about t

Re: [DISCUSS] FLIP-348: Support System Columns in SQL and Table API

2023-08-17 Thread Jark Wu
;> > >>> 1. system columns provided by external systems via catalogs, such > >>> as ROWID from the original system. > >>> 2. system columns generated by Flink, such as ROWID generated by > >>> Flink itself. > >>> > &g

Re: [DISCUSS] [FLINK-32873] Add a config to allow disabling Query hints

2023-08-17 Thread Jark Wu
Sorry, I still don't understand why we need to disable the query hint. It doesn't have the security problems as options hint. Bonnie said it could affect performance, but that depends on users using it explicitly. If there is any performance problem, users can remove the hint. If we want to

Re: [DISCUSS] Update Flink Roadmap

2023-08-14 Thread Jark Wu
Song, could we add a subsection in `Towards Streaming Warehouses` > or `Performance` that the short-lived query in Flink Session Cluster is one > of the future directions for Flink? > > Best, > Shammon FY > > On Mon, Aug 14, 2023 at 8:03 PM Jark Wu <mailto:imj..

Re: [DISCUSS] FLIP-330: Support specifying record timestamp requirement

2023-08-14 Thread Jark Wu
Hi Becket, > I kind of think that we can restrain the scope to just batch mode, and only for StreamRecord class. That means only in batch mode, the timestamp in the StreamRecord will be dropped when the config is enabled. However, IIUC, dropping timestamp in StreamRecord has been supported. This

Re: [DISCUSS] Update Flink Roadmap

2023-08-14 Thread Jark Wu
bit late. > > > > I will put a draft for the long-term vision for the state as well as > > large-scale state support into the roadmap. > > > > Best > > Yuan > > > > On Mon, Jul 17, 2023 at 10:34 AM Jark Wu wrote: > > > >> Hi Jiabao

Re: [ANNOUNCE] New Apache Flink Committer - Weihua Hu

2023-08-04 Thread Jark Wu
Congratulations, Weihua! Best, Jark On Fri, 4 Aug 2023 at 14:48, Yuxin Tan wrote: > Congratulations Weihua! > > Best, > Yuxin > > > Junrui Lee 于2023年8月4日周五 14:28写道: > > > Congrats, Weihua! > > Best, > > Junrui > > > > Geng Biao 于2023年8月4日周五 14:25写道: > > > > > Congrats, Weihua! > > > Best, >

Re: [ANNOUNCE] New Apache Flink PMC Member - Matthias Pohl

2023-08-04 Thread Jark Wu
Congratulations, Matthias! Best, Jark On Fri, 4 Aug 2023 at 14:59, Weihua Hu wrote: > Congratulations, Matthias! > > Best, > Weihua > > > On Fri, Aug 4, 2023 at 2:49 PM Yuxin Tan wrote: > > > Congratulations, Matthias! > > > > Best, > > Yuxin > > > > > > Sergey Nuyanzin 于2023年8月4日周五

Re: [ANNOUNCE] New Apache Flink Committer - Hong Teoh

2023-08-04 Thread Jark Wu
Congratulations, Hong! Best, Jark On Fri, 4 Aug 2023 at 14:24, Sergey Nuyanzin wrote: > Congratulations, Hong! > > On Fri, Aug 4, 2023 at 7:25 AM Shammon FY wrote: > > > Congratulations, Hong! > > > > Best, > > Shammon FY > > > > On Fri, Aug 4, 2023 at 12:33 PM Jing Ge > > wrote: > > > > >

Re: FLINK-20767 - Support for nested fields filter push down

2023-08-02 Thread Jark Wu
Hi, I agree with Becket that we may need to extend FieldReferenceExpression to support nested field access (or maybe a new NestedFieldReferenceExpression). But I have some concerns about evolving the SupportsProjectionPushDown.applyProjection. A projection is much simpler than Filter Expression

Re: [DISCUSS] FLIP-348: Support System Columns in SQL and Table API

2023-07-31 Thread Jark Wu
Hi Timo, Thanks for your proposal. I think this is a nice feature for users and I prefer option 3. I only have one concern about the concept of pseudo-column or system-column, because this is the first time we introduce it in Flink SQL. The confusion is similar to the question of Benchao and

Re: [DISCUSS] FLIP-330: Support specifying record timestamp requirement

2023-07-31 Thread Jark Wu
Hi Yunfeng, I think this is a great idea to improve the serialization performance, especially for batch jobs. I'm not sure whether you have considered or tested this optimization for batch jobs. IMO, this optimization can be enabled by default for batch jobs, because they don't have watermarks

Re: [DISCUSS] Add missing visibility annotation for Table APIs

2023-07-31 Thread Jark Wu
; > > between > > > > > a public class with no annotation and with the @Internal? > > > > > > > > > > Best regards, > > > > > Jing > > > > > > > > > > > > > > > On Tue, Jul 25, 2023 at 11:06 AM Lincol

Re: [VOTE] Release 2.0 must-have work items - Round 2

2023-07-26 Thread Jark Wu
+1 (binding) Thanks Xintong for driving this. Thanks all for finalizing the SourceFunction conclusion. Best, Jark On Wed, 26 Jul 2023 at 22:28, Alexander Fedulov wrote: > +1 (non-binding), assuming SourceFunction gets added back to the > doc as a "nice-to-have". I am glad we've reached a

Re: [DISCUSS] Add missing visibility annotation for Table APIs

2023-07-24 Thread Jark Wu
Hi Jane, Thanks for kicking off this work and collecting the detailed list. +1 to add the missing annotation. This often confuses me whether the class can be modified without breaking the compatibility when looking at classes in table-common and table-api. Explicitly mark the visibility can be

Re: [ANNOUNCE] New Apache Flink Committer - Yong Fang

2023-07-24 Thread Jark Wu
Congratulations, Yong Fang! Best, Jark On Mon, 24 Jul 2023 at 22:11, Wencong Liu wrote: > Congratulations! > > Best, > Wencong Liu > > > > > > > > > > > > > > > > 在 2023-07-24 11:03:30,"Paul Lam" 写道: > >Congrats, Shammon! > > > >Best, > >Paul Lam > > > >> 2023年7月24日 10:56,Jingsong Li 写道: >

Re: [VOTE] FLIP-309: Support using larger checkpointing interval when source is processing backlog

2023-07-18 Thread Jark Wu
+1 (binding) Best, Jark On Tue, 18 Jul 2023 at 20:30, Piotr Nowojski wrote: > +1 (binding) > > Piotrek > > wt., 18 lip 2023 o 08:51 Jing Ge napisał(a): > > > +1(binding) > > > > Best regards, > > Jing > > > > On Tue, Jul 18, 2023 at 8:31 AM Rui Fan <1996fan...@gmail.com> wrote: > > > > >

Re: [DISCUSS] Update Flink Roadmap

2023-07-16 Thread Jark Wu
gt; There are two suggestions about the Table API: > > - Add the JSON type to adapt to the no sql database type. > - Remove changelog normalize operator for upsert stream. > > > Best, > Jiabao > > > > 2023年7月13日 下午3:49,Jark Wu 写道: > > > > Hi all, >

Re: [DISCUSS] Update Flink Roadmap

2023-07-13 Thread Jark Wu
ested. Thanks! > > Best regards, > Jing > > On Thu, Jun 1, 2023 at 6:00 PM Jark Wu wrote: > > > Hi Jing, > > > > This thread is for discussing the roadmap for versions 1.18, 2.0, and > even > > more. > > One of the outcomes of this discussion will

Re: [VOTE] Release 2.0 must-have work items

2023-07-09 Thread Jark Wu
+1 (binding) Thanks for driving this. Looking forward to starting the 2.0 works. Best, Jark On Fri, 7 Jul 2023 at 17:31, Xintong Song wrote: > Hi all, > > I'd like to start the VOTE for the must-have work items for release 2.0 > [1]. The corresponding discussion thread is [2]. > > Please

Re: [ANNOUNCE] Apache Flink has won the 2023 SIGMOD Systems Award

2023-07-03 Thread Jark Wu
Congrats everyone! Best, Jark > 2023年7月3日 22:37,Yuval Itzchakov 写道: > > Congrats team! > > On Mon, Jul 3, 2023, 17:28 Jing Ge via user > wrote: >> Congratulations! >> >> Best regards, >> Jing >> >> >> On Mon, Jul 3, 2023 at 3:21 PM yuxia >

Re: [DISCUSS] Release 2.0 Work Items

2023-06-29 Thread Jark Wu
Hi, I think one more thing we need to consider to do in 2.0 is changing the default value of configuration to improve out-of-box user experience. Currently, in order to run a Flink job, users may need to set a bunch of configurations, such as minibatch, checkpoint interval, exactly-once,

Re: [VOTE] FLIP-309: Support using larger checkpointing interval when source is processing backlog

2023-06-29 Thread Jark Wu
+1 (binding) Best, Jark > 2023年6月29日 18:12,Jing Ge 写道: > > +1(binding) > > On Thu, Jun 29, 2023 at 7:47 AM Leonard Xu wrote: > >> +1 (binding) >> >> Best, >> Leonard >> >>> On Jun 29, 2023, at 1:25 PM, Jingsong Li wrote: >>> >>> +1 binding >>> >>> On Thu, Jun 29, 2023 at 11:03 AM Dong

Re: [DISCUSS] FLIP-309: Enable operators to trigger checkpoints dynamically

2023-06-27 Thread Jark Wu
Thank you Dong for driving this FLIP. The new design looks good to me! Best, Jark > 2023年6月27日 14:38,Dong Lin 写道: > > Thank you Leonard for the review! > > Hi Piotr, do you have any comments on the latest proposal? > > I am wondering if it is OK to start the voting thread this week. > >

[jira] [Created] (FLINK-32444) Enable object reuse for Flink SQL jobs by default

2023-06-27 Thread Jark Wu (Jira)
Jark Wu created FLINK-32444: --- Summary: Enable object reuse for Flink SQL jobs by default Key: FLINK-32444 URL: https://issues.apache.org/jira/browse/FLINK-32444 Project: Flink Issue Type: New

Re: [DISCUSS] Persistent SQL Gateway

2023-06-26 Thread Jark Wu
cancel all the jobs. So that would > be the next step I think. Any work or thought regarding this aspect? We are > definitely willing to help out on this front. > > Cheers, > F > > > --- Original Message --- > On Sunday, June 25th, 2023 at 06:23, Jark Wu

Re: Async I/O: preserve stream order for requests on key level

2023-06-25 Thread Jark Wu
This is not supported by the current Async I/O API. But I do think this is a very useful feature and we should support it. Just as Jingsong said this allows changelog stream can also use Async Lookup Join. The rough idea is just like a mixture of the ordered and unordered modes of async operator.

Re: [DISCUSS] FLIP-321: Introduce an API deprecation process

2023-06-25 Thread Jark Wu
I agree with Jingsong and Becket. Look at the legacy SourceFunction (a small part of DataStream API), the SourceFunction is still not and can't be marked deprecated[1] until now after the new Source was released 2 years ago, because the new Source still can't fully consume the abilities of legacy

Re: [DISCUSS] Persistent SQL Gateway

2023-06-24 Thread Jark Wu
Hi Ferenc, Making SQL Gateway to be an easy-to-use platform infrastructure of Flink SQL is one of the important roadmaps [1]. The persistence ability of the SQL Gateway is a major work in 1.18 release. One of the persistence demand is that the registered catalogs are currently kept in memory and

Re: [VOTE] FLIP-324: Introduce Runtime Filter for Flink Batch Jobs

2023-06-24 Thread Jark Wu
+1 (binding) Best, Jark > 2023年6月25日 10:04,Xia Sun 写道: > > +1 (non-binding) > > Best Regards, > > Xia > > yuxia 于2023年6月25日周日 09:23写道: > >> +1 (binding) >> Thanks Lijie driving it. >> >> Best regards, >> Yuxia >> >> - 原始邮件 - >> 发件人: "Yuepeng Pan" >> 收件人: "dev" >> 发送时间: 星期六,

  1   2   3   4   5   6   7   8   9   10   >